You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ja...@apache.org on 2014/01/27 23:15:17 UTC

[01/51] [partial] Initial commit of master branch from github

Updated Branches:
  refs/heads/master 64da9e222 -> 50d523f6e


http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/site/markdown/source.md
----------------------------------------------------------------------
diff --git a/phoenix-core/src/site/markdown/source.md b/phoenix-core/src/site/markdown/source.md
new file mode 100644
index 0000000..a2d5467
--- /dev/null
+++ b/phoenix-core/src/site/markdown/source.md
@@ -0,0 +1,9 @@
+# Source Repository
+
+Source will be moved over to Apache SVN soon! In meanwhile it is available at:
+
+<hr/>
+
+https://github.com/forcedotcom/phoenix
+
+<hr/>


[06/51] [partial] Initial commit of master branch from github

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/util/DateUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/DateUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/DateUtil.java
new file mode 100644
index 0000000..b7460f8
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/DateUtil.java
@@ -0,0 +1,154 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.text.Format;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.TimeZone;
+
+import org.apache.commons.lang.time.FastDateFormat;
+
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.IllegalDataException;
+
+
+
+@SuppressWarnings("serial")
+public class DateUtil {
+    public static final TimeZone DATE_TIME_ZONE = TimeZone.getTimeZone("GMT");
+    public static final String DEFAULT_DATE_FORMAT = "yyyy-MM-dd HH:mm:ss"; // This is the format the app sets in NLS settings for every connection.
+    public static final Format DEFAULT_DATE_FORMATTER = FastDateFormat.getInstance(DEFAULT_DATE_FORMAT, DATE_TIME_ZONE);
+
+    public static final String DEFAULT_MS_DATE_FORMAT = "yyyy-MM-dd HH:mm:ss.SSS";
+    public static final Format DEFAULT_MS_DATE_FORMATTER = FastDateFormat.getInstance(DEFAULT_MS_DATE_FORMAT, DATE_TIME_ZONE);
+
+    private DateUtil() {
+    }
+
+    public static Format getDateParser(String pattern) {
+        SimpleDateFormat format = new SimpleDateFormat(pattern) {
+            @Override
+            public java.util.Date parseObject(String source) throws ParseException {
+                java.util.Date date = super.parse(source);
+                return new java.sql.Date(date.getTime());
+            }
+        };
+        format.setTimeZone(DateUtil.DATE_TIME_ZONE);
+        return format;
+    }
+
+    public static Format getTimeParser(String pattern) {
+        SimpleDateFormat format = new SimpleDateFormat(pattern) {
+            @Override
+            public java.util.Date parseObject(String source) throws ParseException {
+                java.util.Date date = super.parse(source);
+                return new java.sql.Time(date.getTime());
+            }
+        };
+        format.setTimeZone(DateUtil.DATE_TIME_ZONE);
+        return format;
+    }
+
+    public static Format getTimestampParser(String pattern) {
+        SimpleDateFormat format = new SimpleDateFormat(pattern) {
+            @Override
+            public java.util.Date parseObject(String source) throws ParseException {
+                java.util.Date date = super.parse(source);
+                return new java.sql.Timestamp(date.getTime());
+            }
+        };
+        format.setTimeZone(DateUtil.DATE_TIME_ZONE);
+        return format;
+    }
+
+    public static Format getDateFormatter(String pattern) {
+        return DateUtil.DEFAULT_DATE_FORMAT.equals(pattern) ? DateUtil.DEFAULT_DATE_FORMATTER : FastDateFormat.getInstance(pattern, DateUtil.DATE_TIME_ZONE);
+    }
+
+    private static ThreadLocal<Format> dateFormat =
+            new ThreadLocal < Format > () {
+        @Override protected Format initialValue() {
+            return getDateParser(DEFAULT_DATE_FORMAT);
+        }
+    };
+
+    public static Date parseDate(String dateValue) {
+        try {
+            return (Date)dateFormat.get().parseObject(dateValue);
+        } catch (ParseException e) {
+            throw new IllegalDataException(e);
+        }
+    }
+
+    private static ThreadLocal<Format> timeFormat =
+            new ThreadLocal < Format > () {
+        @Override protected Format initialValue() {
+            return getTimeParser(DEFAULT_DATE_FORMAT);
+        }
+    };
+
+    public static Time parseTime(String timeValue) {
+        try {
+            return (Time)timeFormat.get().parseObject(timeValue);
+        } catch (ParseException e) {
+            throw new IllegalDataException(e);
+        }
+    }
+
+    private static ThreadLocal<Format> timestampFormat =
+            new ThreadLocal < Format > () {
+        @Override protected Format initialValue() {
+            return getTimestampParser(DEFAULT_DATE_FORMAT);
+        }
+    };
+
+    public static Timestamp parseTimestamp(String timeValue) {
+        try {
+            return (Timestamp)timestampFormat.get().parseObject(timeValue);
+        } catch (ParseException e) {
+            throw new IllegalDataException(e);
+        }
+    }
+
+    /**
+     * Utility function to work around the weirdness of the {@link Timestamp} constructor.
+     * This method takes the milli-seconds that spills over to the nanos part as part of 
+     * constructing the {@link Timestamp} object.
+     * If we just set the nanos part of timestamp to the nanos passed in param, we 
+     * end up losing the sub-second part of timestamp. 
+     */
+    public static Timestamp getTimestamp(long millis, int nanos) {
+        Timestamp ts = new Timestamp(millis);
+        ts.setNanos(ts.getNanos() + nanos);
+        return ts;
+    }
+
+    /**
+     * Utility function to convert a {@link BigDecimal} value to {@link Timestamp}.
+     */
+    public static Timestamp getTimestamp(BigDecimal bd) {
+        return DateUtil.getTimestamp(bd.longValue(), ((bd.remainder(BigDecimal.ONE).multiply(BigDecimal.valueOf(QueryConstants.MILLIS_TO_NANOS_CONVERTOR))).intValue()));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..a7c975c
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
@@ -0,0 +1,204 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.io.IOException;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import com.google.common.collect.Lists;
+import org.apache.hadoop.hbase.index.ValueGetter;
+import org.apache.hadoop.hbase.index.covered.update.ColumnReference;
+import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.client.KeyValueBuilder;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+import org.apache.phoenix.index.IndexMaintainer;
+import org.apache.phoenix.query.QueryConstants;
+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.PDataType;
+import org.apache.phoenix.schema.PTable;
+
+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);
+
+    private IndexUtil() {
+    }
+
+    // Since we cannot have nullable fixed length in a row key
+    // we need to translate to variable length.
+    public static PDataType getIndexColumnDataType(PColumn dataColumn) throws SQLException {
+        PDataType type = getIndexColumnDataType(dataColumn.isNullable(),dataColumn.getDataType());
+        if (type == null) {
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_INDEX_COLUMN_ON_TYPE).setColumnName(dataColumn.getName().getString())
+            .setMessage("Type="+dataColumn.getDataType()).build().buildException();
+        }
+        return type;
+    }
+    
+    // Since we cannot have nullable fixed length in a row key
+    // we need to translate to variable length. The verification that we have a valid index
+    // row key was already done, so here we just need to covert from one built-in type to
+    // another.
+    public static PDataType getIndexColumnDataType(boolean isNullable, PDataType dataType) {
+        if (dataType == null || !isNullable || !dataType.isFixedWidth() || dataType == PDataType.BINARY) {
+            return dataType;
+        }
+        // for INT, BIGINT
+        if (dataType.isCoercibleTo(PDataType.TIMESTAMP) || dataType.isCoercibleTo(PDataType.DECIMAL)) {
+            return PDataType.DECIMAL;
+        }
+        // for CHAR
+        if (dataType.isCoercibleTo(PDataType.VARCHAR)) {
+            return PDataType.VARCHAR;
+        }
+        throw new IllegalArgumentException("Unsupported non nullable index type " + dataType);
+    }
+    
+
+    public static String getDataColumnName(String name) {
+        return name.substring(name.indexOf(INDEX_COLUMN_NAME_SEP) + 1);
+    }
+
+    public static String getDataColumnFamilyName(String name) {
+        return name.substring(0,name.indexOf(INDEX_COLUMN_NAME_SEP));
+    }
+
+    public static String getDataColumnFullName(String name) {
+        int index = name.indexOf(INDEX_COLUMN_NAME_SEP) ;
+        if (index == 0) {
+            return name.substring(index+1);
+        }
+        return SchemaUtil.getColumnDisplayName(name.substring(0, index), name.substring(index+1));
+    }
+
+    public static String getIndexColumnName(String dataColumnFamilyName, String dataColumnName) {
+        return (dataColumnFamilyName == null ? "" : dataColumnFamilyName) + INDEX_COLUMN_NAME_SEP + dataColumnName;
+    }
+    
+    public static byte[] getIndexColumnName(byte[] dataColumnFamilyName, byte[] dataColumnName) {
+        return ByteUtil.concat(dataColumnFamilyName == null ?  ByteUtil.EMPTY_BYTE_ARRAY : dataColumnFamilyName, INDEX_COLUMN_NAME_SEP_BYTES, dataColumnName);
+    }
+    
+    public static String getIndexColumnName(PColumn dataColumn) {
+        String dataColumnFamilyName = SchemaUtil.isPKColumn(dataColumn) ? null : dataColumn.getFamilyName().getString();
+        return getIndexColumnName(dataColumnFamilyName, dataColumn.getName().getString());
+    }
+
+    public static PColumn getDataColumn(PTable dataTable, String indexColumnName) {
+        int pos = indexColumnName.indexOf(INDEX_COLUMN_NAME_SEP);
+        if (pos < 0) {
+            throw new IllegalArgumentException("Could not find expected '" + INDEX_COLUMN_NAME_SEP +  "' separator in index column name of \"" + indexColumnName + "\"");
+        }
+        if (pos == 0) {
+            try {
+                return dataTable.getPKColumn(indexColumnName.substring(1));
+            } catch (ColumnNotFoundException e) {
+                throw new IllegalArgumentException("Could not find PK column \"" +  indexColumnName.substring(pos+1) + "\" in index column name of \"" + indexColumnName + "\"", e);
+            }
+        }
+        PColumnFamily family;
+        try {
+            family = dataTable.getColumnFamily(indexColumnName.substring(0, pos));
+        } catch (ColumnFamilyNotFoundException e) {
+            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));
+        } catch (ColumnNotFoundException e) {
+            throw new IllegalArgumentException("Could not find column \"" +  indexColumnName.substring(pos+1) + "\" in index column name of \"" + indexColumnName + "\"", e);
+        }
+    }
+
+    private static boolean isEmptyKeyValue(PTable table, ColumnReference ref) {
+        byte[] emptyKeyValueCF = SchemaUtil.getEmptyColumnFamily(table.getColumnFamilies());
+        return (Bytes.compareTo(emptyKeyValueCF, ref.getFamily()) == 0 &&
+                Bytes.compareTo(QueryConstants.EMPTY_COLUMN_BYTES, ref.getQualifier()) == 0);
+    }
+
+    public static List<Mutation> generateIndexData(final PTable table, PTable index,
+            List<Mutation> dataMutations, ImmutableBytesWritable ptr, KeyValueBuilder builder)
+            throws SQLException {
+        try {
+            IndexMaintainer maintainer = index.getIndexMaintainer(table);
+            maintainer.setKvBuilder(builder);
+            List<Mutation> indexMutations = Lists.newArrayListWithExpectedSize(dataMutations.size());
+           for (final Mutation dataMutation : dataMutations) {
+                long ts = MetaDataUtil.getClientTimeStamp(dataMutation);
+                ptr.set(dataMutation.getRow());
+                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
+                        public ImmutableBytesPtr getLatestValue(ColumnReference ref) {
+                            // Always return null for our empty key value, as this will cause the index
+                            // maintainer to always treat this Put as a new row.
+                            if (isEmptyKeyValue(table, ref)) {
+                                return null;
+                            }
+                            Map<byte [], List<KeyValue>> familyMap = dataMutation.getFamilyMap();
+                            byte[] family = ref.getFamily();
+                            List<KeyValue> kvs = familyMap.get(family);
+                            if (kvs == null) {
+                                return null;
+                            }
+                            byte[] qualifier = ref.getQualifier();
+                            for (KeyValue kv : kvs) {
+                                if (Bytes.compareTo(kv.getBuffer(), kv.getFamilyOffset(), kv.getFamilyLength(), family, 0, family.length) == 0 &&
+                                    Bytes.compareTo(kv.getBuffer(), kv.getQualifierOffset(), kv.getQualifierLength(), qualifier, 0, qualifier.length) == 0) {
+                                    return new ImmutableBytesPtr(kv.getBuffer(), kv.getValueOffset(), kv.getValueLength());
+                                }
+                            }
+                            return null;
+                        }
+                        
+                    };
+                    indexMutations.add(maintainer.buildUpdateMutation(valueGetter, ptr, ts));
+                } else {
+                    if (!maintainer.getIndexedColumns().isEmpty()) {
+                        throw new SQLExceptionInfo.Builder(SQLExceptionCode.NO_DELETE_IF_IMMUTABLE_INDEX).setSchemaName(table.getSchemaName().getString())
+                        .setTableName(table.getTableName().getString()).build().buildException();
+                    }
+                    indexMutations.add(maintainer.buildDeleteMutation(ptr, ts));
+                }
+            }
+            return indexMutations;
+        } catch (IOException e) {
+            throw new SQLException(e);
+        }
+    }
+
+    public static boolean isDataPKColumn(PColumn column) {
+        return column.getName().getString().startsWith(INDEX_COLUMN_NAME_SEP);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/util/InstanceResolver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/InstanceResolver.java b/phoenix-core/src/main/java/org/apache/phoenix/util/InstanceResolver.java
new file mode 100644
index 0000000..85cf54d
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/InstanceResolver.java
@@ -0,0 +1,66 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.util.ServiceLoader;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Resolves object instances registered using the JDK 6+ {@link java.util.ServiceLoader}.
+ *
+ * @author aaraujo
+ * @since 2.0
+ */
+public class InstanceResolver {
+    private static final ConcurrentHashMap<Class, Object> RESOLVED_SINGLETONS = new ConcurrentHashMap<Class, Object>();
+
+    private InstanceResolver() {/* not allowed */}
+
+    /**
+     * Resolves an instance of the specified class if it has not already been resolved.
+     * @param clazz The type of instance to resolve
+     * @param defaultInstance The instance to use if a custom instance has not been registered
+     * @return The resolved instance or the default instance provided.
+     *         {@code null} if an instance is not registered and a default is not provided.
+     */
+    @SuppressWarnings("unchecked")
+    public static <T> T getSingleton(Class<T> clazz, T defaultInstance) {
+        Object obj = RESOLVED_SINGLETONS.get(clazz);
+        if(obj != null) {
+            return (T)obj;
+        }
+        if (defaultInstance != null && !clazz.isInstance(defaultInstance)) throw new IllegalArgumentException("defaultInstance is not of type " + clazz.getName());
+        final Object o = resolveSingleton(clazz, defaultInstance);
+        obj = RESOLVED_SINGLETONS.putIfAbsent(clazz, o);
+        if(obj == null) {
+            obj = o;
+        }
+        return (T)obj;
+    }
+    
+    private synchronized static <T> T resolveSingleton(Class<T> clazz, T defaultInstance) {
+        ServiceLoader<T> loader = ServiceLoader.load(clazz);
+        // returns the first registered instance found
+        for (T singleton : loader) {
+            return singleton;
+        }
+        return defaultInstance;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/util/JDBCUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/JDBCUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/JDBCUtil.java
new file mode 100644
index 0000000..eb973cc
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/JDBCUtil.java
@@ -0,0 +1,83 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.sql.SQLException;
+import java.util.Properties;
+
+import javax.annotation.Nullable;
+
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.PName;
+import org.apache.phoenix.schema.PNameFactory;
+
+
+
+/**
+ * Utilities for JDBC
+ *
+ * @author jtaylor
+ * @since 178
+ */
+public class JDBCUtil {
+    
+    private JDBCUtil() {
+    }
+
+    /**
+     * Find the propName by first looking in the url string and if not found,
+     * next in the info properties. If not found, null is returned.
+     * @param url JDBC connection URL
+     * @param info JDBC connection properties
+     * @param propName the name of the property to find
+     * @return the property value or null if not found
+     */
+    public static String findProperty(String url, Properties info, String propName) {
+        String urlPropName = ";" + propName + "=";
+        String propValue = info.getProperty(propName);
+        if (propValue == null) {
+            int begIndex = url.indexOf(urlPropName);
+            if (begIndex >= 0) {
+                int endIndex = url.indexOf(';',begIndex + urlPropName.length());
+                if (endIndex < 0) {
+                    endIndex = url.length();
+                }
+                propValue = url.substring(begIndex + urlPropName.length(), endIndex);
+            }
+        }
+        return propValue;
+    }
+
+    public static Long getCurrentSCN(String url, Properties info) throws SQLException {
+        String scnStr = findProperty(url, info, PhoenixRuntime.CURRENT_SCN_ATTRIB);
+        return (scnStr == null ? null : Long.parseLong(scnStr));
+    }
+
+    public static int getMutateBatchSize(String url, Properties info, ReadOnlyProps props) throws SQLException {
+        String batchSizeStr = findProperty(url, info, PhoenixRuntime.UPSERT_BATCH_SIZE_ATTRIB);
+        return (batchSizeStr == null ? props.getInt(QueryServices.MUTATE_BATCH_SIZE_ATTRIB, QueryServicesOptions.DEFAULT_MUTATE_BATCH_SIZE) : Integer.parseInt(batchSizeStr));
+    }
+
+    public static @Nullable PName getTenantId(String url, Properties info) throws SQLException {
+        String tenantId = findProperty(url, info, PhoenixRuntime.TENANT_ID_ATTRIB);
+        return (tenantId == null ? null : PNameFactory.newName(tenantId));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..657845d
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/KeyValueUtil.java
@@ -0,0 +1,170 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValue.Type;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * 
+ * Utilities for KeyValue. Where there's duplication with KeyValue methods,
+ * these avoid creating new objects when not necessary (primary preventing
+ * byte array copying).
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class KeyValueUtil {
+    private KeyValueUtil() {
+    }
+
+    public static KeyValue newKeyValue(Result r, byte[] cf, byte[] cq, long ts, byte[] value, int valueOffset, int valueLength) {
+        byte[] bytes = ResultUtil.getRawBytes(r);
+        return new KeyValue(bytes, ResultUtil.getKeyOffset(r), ResultUtil.getKeyLength(r),
+                cf, 0, cf.length,
+                cq, 0, cq.length,
+                ts, Type.Put,
+                value, valueOffset, valueLength);
+    }
+
+    public static KeyValue newKeyValue(byte[] key, byte[] cf, byte[] cq, long ts, byte[] value, int valueOffset, int valueLength) {
+        return new KeyValue(key, 0, key.length,
+                cf, 0, cf.length,
+                cq, 0, cq.length,
+                ts, Type.Put,
+                value, valueOffset, valueLength);
+    }
+
+    public static KeyValue newKeyValue(ImmutableBytesWritable key, byte[] cf, byte[] cq, long ts, byte[] value, int valueOffset, int valueLength) {
+        return new KeyValue(key.get(), key.getOffset(), key.getLength(),
+                cf, 0, cf.length,
+                cq, 0, cq.length,
+                ts, Type.Put,
+                value, valueOffset, valueLength);
+    }
+
+    public static KeyValue newKeyValue(byte[] key, int keyOffset, int keyLength, byte[] cf, byte[] cq, long ts, byte[] value, int valueOffset, int valueLength) {
+        return new KeyValue(key, keyOffset, keyLength,
+                cf, 0, cf.length,
+                cq, 0, cq.length,
+                ts, Type.Put,
+                value, valueOffset, valueLength);
+    }
+
+    public static KeyValue newKeyValue(byte[] key, byte[] cf, byte[] cq, long ts, byte[] value) {
+        return newKeyValue(key,cf,cq,ts,value,0,value.length);
+    }
+
+    public static KeyValue newKeyValue(Result r, byte[] cf, byte[] cq, long ts, byte[] value) {
+        return newKeyValue(r,cf,cq,ts,value,0,value.length);
+    }
+
+    /**
+     * Binary search for latest column value without allocating memory in the process
+     * @param kvs
+     * @param family
+     * @param qualifier
+     */
+    public static KeyValue getColumnLatest(List<KeyValue>kvs, byte[] family, byte[] qualifier) {
+        if (kvs.size() == 0) {
+        	return null;
+        }
+        KeyValue row = kvs.get(0);
+        Comparator<KeyValue> comp = new SearchComparator(row.getBuffer(), row.getRowOffset(), row.getRowLength(), family, qualifier);
+        // pos === ( -(insertion point) - 1)
+        int pos = Collections.binarySearch(kvs, null, comp);
+        // never will exact match
+        if (pos < 0) {
+          pos = (pos+1) * -1;
+          // pos is now insertion point
+        }
+        if (pos == kvs.size()) {
+          return null; // doesn't exist
+        }
+    
+        KeyValue kv = kvs.get(pos);
+        if (Bytes.compareTo(kv.getBuffer(), kv.getFamilyOffset(), kv.getFamilyLength(),
+                family, 0, family.length) != 0) {
+            return null;
+        }
+        if (Bytes.compareTo(kv.getBuffer(), kv.getQualifierOffset(), kv.getQualifierLength(),
+                qualifier, 0, qualifier.length) != 0) {
+            return null;
+        }
+        return kv;
+    }
+
+    /*
+     * Special comparator, *only* works for binary search.
+     * Current JDKs only uses the search term on the right side,
+     * Making use of that saves instanceof checks, and allows us
+     * to inline the search term in the comparator
+     */
+	private static class SearchComparator implements Comparator<KeyValue> {
+		private final byte[] row;
+		private final byte[] family;
+		private final byte[] qualifier;
+		private final int rowOff;
+		private final int rowLen;
+
+		public SearchComparator(byte[] r, int rOff, int rLen, byte[] f, byte[] q) {
+			row = r;
+			family = f;
+			qualifier = q;
+			rowOff = rOff;
+			rowLen = rLen;
+		}
+
+		@Override
+        public int compare(final KeyValue l, final KeyValue ignored) {
+			assert ignored == null;
+			final byte[] buf = l.getBuffer();
+			final int rOff = l.getRowOffset();
+			final short rLen = l.getRowLength();
+			// row
+			int val = Bytes.compareTo(buf, rOff, rLen, row, rowOff, rowLen);
+			if (val != 0) {
+				return val;
+			}
+			// family
+			final int fOff = l.getFamilyOffset(rLen);
+			final byte fLen = l.getFamilyLength(fOff);
+			val = Bytes.compareTo(buf, fOff, fLen, family, 0, family.length);
+			if (val != 0) {
+				return val;
+			}
+			// qualifier
+			val = Bytes.compareTo(buf, l.getQualifierOffset(fOff),
+					l.getQualifierLength(rLen, fLen), qualifier, 0, qualifier.length);
+			if (val != 0) {
+				return val;
+			}
+			// want latest TS and type, so we get the first
+			return 1;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..0d8a60a
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
@@ -0,0 +1,233 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.util.SchemaUtil.getVarChars;
+
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.coprocessor.MetaDataProtocol;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.PTableType;
+
+
+public class MetaDataUtil {
+
+    public static boolean areClientAndServerCompatible(long version) {
+        // A server and client with the same major and minor version number must be compatible.
+        // So it's important that we roll the PHOENIX_MAJOR_VERSION or PHOENIX_MINOR_VERSION
+        // when we make an incompatible change.
+        return areClientAndServerCompatible(MetaDataUtil.decodePhoenixVersion(version), MetaDataProtocol.PHOENIX_MAJOR_VERSION, MetaDataProtocol.PHOENIX_MINOR_VERSION);
+    }
+
+    // For testing
+    static boolean areClientAndServerCompatible(int version, int pMajor, int pMinor) {
+        // A server and client with the same major and minor version number must be compatible.
+        // So it's important that we roll the PHOENIX_MAJOR_VERSION or PHOENIX_MINOR_VERSION
+        // when we make an incompatible change.
+        return MetaDataUtil.encodeMaxPatchVersion(pMajor, pMinor) >= version && MetaDataUtil.encodeMinPatchVersion(pMajor, pMinor) <= version;
+    }
+
+    // Given the encoded integer representing the phoenix version in the encoded version value.
+    // The second byte in int would be the major version, 3rd byte minor version, and 4th byte 
+    // patch version.
+    public static int decodePhoenixVersion(long version) {
+        return (int) ((version << Byte.SIZE * 3) >>> Byte.SIZE * 4);
+    }
+    
+    // TODO: generalize this to use two bytes to return a SQL error code instead
+    public static long encodeMutableIndexConfiguredProperly(long version, boolean isValid) {
+        if (!isValid) {
+            return version | 1;
+        }
+        return version;
+    }
+    
+    public static boolean decodeMutableIndexConfiguredProperly(long version) {
+        return (version & 0xF) == 0;
+    }
+
+    // Given the encoded integer representing the client hbase version in the encoded version value.
+    // The second byte in int would be the major version, 3rd byte minor version, and 4th byte 
+    // patch version.
+    public static int decodeHBaseVersion(long version) {
+        return (int) (version >>> Byte.SIZE * 5);
+    }
+
+    public static String decodeHBaseVersionAsString(int version) {
+        int major = (version >>> Byte.SIZE  * 2) & 0xFF;
+        int minor = (version >>> Byte.SIZE  * 1) & 0xFF;
+        int patch = version & 0xFF;
+        return major + "." + minor + "." + patch;
+    }
+
+    public static long encodeHBaseAndPhoenixVersions(String hbaseVersion) {
+        return (((long) encodeVersion(hbaseVersion)) << (Byte.SIZE * 5)) |
+                (((long) encodeVersion(MetaDataProtocol.PHOENIX_MAJOR_VERSION, MetaDataProtocol.PHOENIX_MINOR_VERSION,
+                        MetaDataProtocol.PHOENIX_PATCH_NUMBER)) << (Byte.SIZE * 1));
+    }
+
+    // Encode a version string in the format of "major.minor.patch" into an integer.
+    public static int encodeVersion(String version) {
+        String[] versionParts = splitHBaseVersionString(version);
+        return encodeVersion(versionParts[0], versionParts.length > 1 ? versionParts[1] : null, versionParts.length > 2 ? versionParts[2] : null);
+    }
+
+    public static String[] splitHBaseVersionString(String version) {
+        return version.split("[-\\.]");
+    }
+
+    // Encode the major as 2nd byte in the int, minor as the first byte and patch as the last byte.
+    public static int encodeVersion(String major, String minor, String patch) {
+        return encodeVersion(major == null ? 0 : Integer.parseInt(major), minor == null ? 0 : Integer.parseInt(minor), 
+                        patch == null ? 0 : Integer.parseInt(patch));
+    }
+
+    public static int encodeVersion(int major, int minor, int patch) {
+        int version = 0;
+        version |= (major << Byte.SIZE * 2);
+        version |= (minor << Byte.SIZE);
+        version |= patch;
+        return version;
+    }
+
+    public static int encodeMaxPatchVersion(int major, int minor) {
+        int version = 0;
+        version |= (major << Byte.SIZE * 2);
+        version |= (minor << Byte.SIZE);
+        version |= 0xFF;
+        return version;
+    }
+
+    public static int encodeMinPatchVersion(int major, int minor) {
+        int version = 0;
+        version |= (major << Byte.SIZE * 2);
+        version |= (minor << Byte.SIZE);
+        return version;
+    }
+
+    public static void getTenantIdAndSchemaAndTableName(List<Mutation> tableMetadata, byte[][] rowKeyMetaData) {
+        Mutation m = getTableHeaderRow(tableMetadata);
+        getVarChars(m.getRow(), 3, rowKeyMetaData);
+    }
+    
+    public static byte[] getParentTableName(List<Mutation> tableMetadata) {
+        if (tableMetadata.size() == 1) {
+            return null;
+        }
+        byte[][] rowKeyMetaData = new byte[3][];
+        getTenantIdAndSchemaAndTableName(tableMetadata, rowKeyMetaData);
+        byte[] tableName = rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
+        Mutation m = getParentTableHeaderRow(tableMetadata);
+        getVarChars(m.getRow(), 3, rowKeyMetaData);
+        if (Bytes.compareTo(tableName, rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX]) == 0) {
+            return null;
+        }
+        return rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
+    }
+    
+    public static long getSequenceNumber(Mutation tableMutation) {
+        List<KeyValue> kvs = tableMutation.getFamilyMap().get(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES);
+        if (kvs != null) {
+            for (KeyValue kv : kvs) { // list is not ordered, so search. TODO: we could potentially assume the position
+                if (Bytes.compareTo(kv.getBuffer(), kv.getQualifierOffset(), kv.getQualifierLength(), PhoenixDatabaseMetaData.TABLE_SEQ_NUM_BYTES, 0, PhoenixDatabaseMetaData.TABLE_SEQ_NUM_BYTES.length) == 0) {
+                    return PDataType.LONG.getCodec().decodeLong(kv.getBuffer(), kv.getValueOffset(), null);
+                }
+            }
+        }
+        throw new IllegalStateException();
+    }
+
+    public static long getSequenceNumber(List<Mutation> tableMetaData) {
+        return getSequenceNumber(getTableHeaderRow(tableMetaData));
+    }
+    
+    public static PTableType getTableType(List<Mutation> tableMetaData) {
+        KeyValue kv = getMutationKeyValue(getPutOnlyTableHeaderRow(tableMetaData), PhoenixDatabaseMetaData.TABLE_TYPE_BYTES);
+        return kv == null ? null : PTableType.fromSerializedValue(kv.getBuffer()[kv.getValueOffset()]);
+    }
+    
+    public static long getParentSequenceNumber(List<Mutation> tableMetaData) {
+        return getSequenceNumber(getParentTableHeaderRow(tableMetaData));
+    }
+    
+    public static Mutation getTableHeaderRow(List<Mutation> tableMetaData) {
+        return tableMetaData.get(0);
+    }
+
+    public static byte[] getMutationKVByteValue(Mutation headerRow, byte[] key) {
+        KeyValue kv = getMutationKeyValue(headerRow, key);
+        // FIXME: byte copy
+        return kv == null ? ByteUtil.EMPTY_BYTE_ARRAY : kv.getValue();
+    }
+
+    private static KeyValue getMutationKeyValue(Mutation headerRow, byte[] key) {
+        List<KeyValue> kvs = headerRow.getFamilyMap().get(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES);
+        if (kvs != null) {
+            for (KeyValue kv : kvs) {
+                if (Bytes.compareTo(kv.getBuffer(), kv.getQualifierOffset(), kv.getQualifierLength(), key, 0,
+                        key.length) == 0) { return kv; }
+            }
+        }
+        return null;
+    }
+
+    /**
+     * Returns the first Put element in <code>tableMetaData</code>. There could be leading Delete elements before the
+     * table header row
+     */
+    public static Mutation getPutOnlyTableHeaderRow(List<Mutation> tableMetaData) {
+        for (Mutation m : tableMetaData) {
+            if (m instanceof Put) { return m; }
+        }
+        throw new IllegalStateException("No table header row found in table meatadata");
+    }
+
+    public static Mutation getParentTableHeaderRow(List<Mutation> tableMetaData) {
+        return tableMetaData.get(tableMetaData.size()-1);
+    }
+
+    public static long getClientTimeStamp(List<Mutation> tableMetadata) {
+        Mutation m = tableMetadata.get(0);
+        return getClientTimeStamp(m);
+    }    
+
+    public static long getClientTimeStamp(Mutation m) {
+        Collection<List<KeyValue>> kvs = m.getFamilyMap().values();
+        // Empty if Mutation is a Delete
+        // TODO: confirm that Delete timestamp is reset like Put
+        return kvs.isEmpty() ? m.getTimeStamp() : kvs.iterator().next().get(0).getTimestamp();
+    }    
+
+    public static byte[] getParentLinkKey(String tenantId, String schemaName, String tableName, String indexName) {
+        return ByteUtil.concat(tenantId == null ? ByteUtil.EMPTY_BYTE_ARRAY : Bytes.toBytes(tenantId), QueryConstants.SEPARATOR_BYTE_ARRAY, schemaName == null ? ByteUtil.EMPTY_BYTE_ARRAY : Bytes.toBytes(schemaName), QueryConstants.SEPARATOR_BYTE_ARRAY, Bytes.toBytes(tableName), QueryConstants.SEPARATOR_BYTE_ARRAY, QueryConstants.SEPARATOR_BYTE_ARRAY, Bytes.toBytes(indexName));
+    }
+
+    public static byte[] getParentLinkKey(byte[] tenantId, byte[] schemaName, byte[] tableName, byte[] indexName) {
+        return ByteUtil.concat(tenantId == null ? ByteUtil.EMPTY_BYTE_ARRAY : tenantId, QueryConstants.SEPARATOR_BYTE_ARRAY, schemaName == null ? ByteUtil.EMPTY_BYTE_ARRAY : schemaName, QueryConstants.SEPARATOR_BYTE_ARRAY, tableName, QueryConstants.SEPARATOR_BYTE_ARRAY, QueryConstants.SEPARATOR_BYTE_ARRAY, indexName);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/util/NumberUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/NumberUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/NumberUtil.java
new file mode 100644
index 0000000..ece1104
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/NumberUtil.java
@@ -0,0 +1,54 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.math.BigDecimal;
+
+import org.apache.phoenix.schema.PDataType;
+
+/**
+ * Utility methods for numbers like decimal, long, etc.
+ *
+ * @author elevine
+ * @since 0.1
+ */
+public class NumberUtil {
+    
+    public static final String DEFAULT_NUMBER_FORMAT = "#,##0.###";
+
+    /**
+     * Strip all trailing zeros to ensure that no digit will be zero and
+     * round using our default context to ensure precision doesn't exceed max allowed.
+     * @return new {@link BigDecimal} instance
+     */
+    public static BigDecimal normalize(BigDecimal bigDecimal) {
+        return bigDecimal.stripTrailingZeros().round(PDataType.DEFAULT_MATH_CONTEXT);
+    }
+
+    public static BigDecimal setDecimalWidthAndScale(BigDecimal decimal, int precision, int scale) {
+        // If we could not fit all the digits before decimal point into the new desired precision and
+        // scale, return null and the caller method should handle the error.
+        if (((precision - scale) < (decimal.precision() - decimal.scale()))){
+            return null;
+        }
+        decimal = decimal.setScale(scale, BigDecimal.ROUND_DOWN);
+        return decimal;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..102a0f3
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
@@ -0,0 +1,308 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.Reader;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.StringTokenizer;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.util.Pair;
+
+import com.google.common.collect.Lists;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+
+/**
+ * 
+ * Collection of non JDBC compliant utility methods
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class PhoenixRuntime {
+    /**
+     * Use this connection property to control HBase timestamps
+     * by specifying your own long timestamp value at connection time. All
+     * queries will use this as the upper bound of the time range for scans
+     * and DDL, and DML will use this as t he timestamp for key values.
+     */
+    public static final String CURRENT_SCN_ATTRIB = "CurrentSCN";
+
+    /**
+     * Root for the JDBC URL that the Phoenix accepts accepts.
+     */
+    public final static String JDBC_PROTOCOL = "jdbc:phoenix";
+    public final static char JDBC_PROTOCOL_TERMINATOR = ';';
+    public final static char JDBC_PROTOCOL_SEPARATOR = ':';
+    
+    @Deprecated
+    public final static String EMBEDDED_JDBC_PROTOCOL = PhoenixRuntime.JDBC_PROTOCOL + PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR;
+    
+    /**
+     * Use this connection property to control the number of rows that are
+     * batched together on an UPSERT INTO table1... SELECT ... FROM table2.
+     * It's only used when autoCommit is true and your source table is
+     * different than your target table or your SELECT statement has a 
+     * GROUP BY clause.
+     */
+    public final static String UPSERT_BATCH_SIZE_ATTRIB = "UpsertBatchSize";
+    
+    /**
+     * Use this connection property to help with fairness of resource allocation
+     * for the client and server. The value of the attribute determines the
+     * bucket used to rollup resource usage for a particular tenant/organization. Each tenant
+     * may only use a percentage of total resources, governed by the {@link org.apache.phoenix.query.QueryServices}
+     * configuration properties
+     */
+    public static final String TENANT_ID_ATTRIB = "TenantId";
+
+    /**
+     * Use this as the zookeeper quorum name to have a connection-less connection. This enables
+     * Phoenix-compatible HFiles to be created in a map/reduce job by creating tables,
+     * upserting data into them, and getting the uncommitted state through {@link #getUncommittedData(Connection)}
+     */
+    public final static String CONNECTIONLESS = "none";
+    
+    private static final String TABLE_OPTION = "-t";
+    private static final String HEADER_OPTION = "-h";
+    private static final String STRICT_OPTION = "-s";
+    private static final String CSV_OPTION = "-d";
+    private static final String HEADER_IN_LINE = "in-line";
+    private static final String SQL_FILE_EXT = ".sql";
+    private static final String CSV_FILE_EXT = ".csv";
+    
+    private static void usageError() {
+        System.err.println("Usage: psql [-t table-name] [-h comma-separated-column-names | in-line] [-d field-delimiter-char quote-char escape-char]<zookeeper>  <path-to-sql-or-csv-file>...\n" +
+                "  By default, the name of the CSV file is used to determine the Phoenix table into which the CSV data is loaded\n" +
+                "  and the ordinal value of the columns determines the mapping.\n" +
+                "  -t overrides the table into which the CSV data is loaded\n" +
+                "  -h overrides the column names to which the CSV data maps\n" +
+                "     A special value of in-line indicating that the first line of the CSV file\n" +
+                "     determines the column to which the data maps.\n" +
+                "  -s uses strict mode by throwing an exception if a column name doesn't match during CSV loading.\n" +
+                "  -d uses custom delimiters for CSV loader, need to specify single char for field delimiter, phrase delimiter, and escape char.\n" +
+                "     number is NOT usually a delimiter and shall be taken as 1 -> ctrl A, 2 -> ctrl B ... 9 -> ctrl I. \n" +
+                "Examples:\n" +
+                "  psql localhost my_ddl.sql\n" +
+                "  psql localhost my_ddl.sql my_table.csv\n" +
+                "  psql -t my_table my_cluster:1825 my_table2012-Q3.csv\n" +
+                "  psql -t my_table -h col1,col2,col3 my_cluster:1825 my_table2012-Q3.csv\n" +
+                "  psql -t my_table -h col1,col2,col3 -d 1 2 3 my_cluster:1825 my_table2012-Q3.csv\n"
+        );
+        System.exit(-1);
+    }
+    /**
+     * Provides a mechanism to run SQL scripts against, where the arguments are:
+     * 1) connection URL string
+     * 2) one or more paths to either SQL scripts or CSV files
+     * If a CurrentSCN property is set on the connection URL, then it is incremented
+     * between processing, with each file being processed by a new connection at the
+     * increment timestamp value.
+     */
+    public static void main(String [] args) {
+        if (args.length < 2) {
+            usageError();
+        }
+        PhoenixConnection conn = null;
+        try {
+            String tableName = null;
+            List<String> columns = null;
+            boolean isStrict = false;
+            List<String> delimiter = new ArrayList<String>();
+
+            int i = 0;
+            for (; i < args.length; i++) {
+                if (TABLE_OPTION.equals(args[i])) {
+                    if (++i == args.length || tableName != null) {
+                        usageError();
+                    }
+                    tableName = args[i];
+                } else if (HEADER_OPTION.equals(args[i])) {
+                    if (++i >= args.length || columns != null) {
+                        usageError();
+                    }
+                    String header = args[i];
+                    if (HEADER_IN_LINE.equals(header)) {
+                        columns = Collections.emptyList();
+                    } else {
+                        columns = Lists.newArrayList();
+                        StringTokenizer tokenizer = new StringTokenizer(header,",");
+                        while(tokenizer.hasMoreTokens()) {
+                            columns.add(tokenizer.nextToken());
+                        }
+                    }
+                } else if (STRICT_OPTION.equals(args[i])) {
+                    isStrict = true;
+                } else if (CSV_OPTION.equals(args[i])) {
+                    for(int j=0; j < 3; j++) {
+                        if(args[++i].length()==1){
+                            delimiter.add(args[i]);
+                        } else {
+                            usageError();
+                        }
+                    }
+                } else {
+                    break;
+                }
+            }
+            if (i == args.length) {
+                usageError();
+            }
+            
+            Properties props = new Properties();
+            String connectionUrl = JDBC_PROTOCOL + JDBC_PROTOCOL_SEPARATOR + args[i++];
+            conn = DriverManager.getConnection(connectionUrl, props).unwrap(PhoenixConnection.class);
+            
+            for (; i < args.length; i++) {
+                String fileName = args[i];
+                if (fileName.endsWith(SQL_FILE_EXT)) {
+               		PhoenixRuntime.executeStatements(conn, new FileReader(args[i]), Collections.emptyList());
+                } else if (fileName.endsWith(CSV_FILE_EXT)) {
+                    if (tableName == null) {
+                        tableName = fileName.substring(fileName.lastIndexOf(File.separatorChar) + 1, fileName.length()-CSV_FILE_EXT.length());
+                    }
+                    CSVLoader csvLoader = new CSVLoader(conn, tableName, columns, isStrict, delimiter);
+                    csvLoader.upsert(fileName);
+                } else {
+                    usageError();
+                }
+                Long scn = conn.getSCN();
+                // If specifying SCN, increment it between processing files to allow
+                // for later files to see earlier files tables.
+                if (scn != null) {
+                    scn++;
+                    props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, scn.toString());
+                    conn.close();
+                    conn = DriverManager.getConnection(connectionUrl, props).unwrap(PhoenixConnection.class);
+                }
+            }
+        } catch (Throwable t) {
+            t.printStackTrace();
+        } finally {
+            if(conn != null) {
+                try {
+                    conn.close();
+                } catch (SQLException e) {
+                    //going to shut jvm down anyway. So might as well feast on it.
+                }
+            }
+            System.exit(0);
+        }
+    }
+
+    private PhoenixRuntime() {
+    }
+    
+    /**
+     * Runs a series of semicolon-terminated SQL statements using the connection provided, returning
+     * the number of SQL statements executed. Note that if the connection has specified an SCN through
+     * the {@link org.apache.phoenix.util.PhoenixRuntime#CURRENT_SCN_ATTRIB} connection property, then the timestamp
+     * is bumped up by one after each statement execution.
+     * @param conn an open JDBC connection
+     * @param reader a reader for semicolumn separated SQL statements
+     * @param binds the binds for all statements
+     * @return the number of SQL statements that were executed
+     * @throws IOException
+     * @throws SQLException
+     */
+    public static int executeStatements(Connection conn, Reader reader, List<Object> binds) throws IOException,SQLException {
+        PhoenixConnection pconn = conn.unwrap(PhoenixConnection.class);
+        // Turn auto commit to true when running scripts in case there's DML
+        pconn.setAutoCommit(true);
+        return pconn.executeStatements(reader, binds, System.out);
+    }
+    
+    /**
+     * Get the list of uncommitted KeyValues for the connection. Currently used to write an
+     * Phoenix-compliant HFile from a map/reduce job.
+     * @param conn an open JDBC connection
+     * @return the list of HBase mutations for uncommitted data
+     * @throws SQLException 
+     */
+    @Deprecated
+    public static List<KeyValue> getUncommittedData(Connection conn) throws SQLException {
+        Iterator<Pair<byte[],List<KeyValue>>> iterator = getUncommittedDataIterator(conn);
+        if (iterator.hasNext()) {
+            return iterator.next().getSecond();
+        }
+        return Collections.emptyList();
+    }
+    
+    /**
+     * Get the list of uncommitted KeyValues for the connection. Currently used to write an
+     * Phoenix-compliant HFile from a map/reduce job.
+     * @param conn an open JDBC connection
+     * @return the list of HBase mutations for uncommitted data
+     * @throws SQLException 
+     */
+    public static Iterator<Pair<byte[],List<KeyValue>>> getUncommittedDataIterator(Connection conn) throws SQLException {
+        return getUncommittedDataIterator(conn, false);
+    }
+    
+    /**
+     * Get the list of uncommitted KeyValues for the connection. Currently used to write an
+     * Phoenix-compliant HFile from a map/reduce job.
+     * @param conn an open JDBC connection
+     * @return the list of HBase mutations for uncommitted data
+     * @throws SQLException 
+     */
+    public static Iterator<Pair<byte[],List<KeyValue>>> getUncommittedDataIterator(Connection conn, boolean includeMutableIndexes) throws SQLException {
+        final Iterator<Pair<byte[],List<Mutation>>> iterator = conn.unwrap(PhoenixConnection.class).getMutationState().toMutations(includeMutableIndexes);
+        return new Iterator<Pair<byte[],List<KeyValue>>>() {
+
+            @Override
+            public boolean hasNext() {
+                return iterator.hasNext();
+            }
+
+            @Override
+            public Pair<byte[], List<KeyValue>> next() {
+                Pair<byte[],List<Mutation>> pair = iterator.next();
+                List<KeyValue> keyValues = Lists.newArrayListWithExpectedSize(pair.getSecond().size() * 5); // Guess-timate 5 key values per row
+                for (Mutation mutation : pair.getSecond()) {
+                    for (List<KeyValue> keyValueList : mutation.getFamilyMap().values()) {
+                        for (KeyValue keyValue : keyValueList) {
+                            keyValues.add(keyValue);
+                        }
+                    }
+                }
+                Collections.sort(keyValues, KeyValue.COMPARATOR);
+                return new Pair<byte[], List<KeyValue>>(pair.getFirst(),keyValues);
+            }
+
+            @Override
+            public void remove() {
+                throw new UnsupportedOperationException();
+            }
+            
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java
new file mode 100644
index 0000000..bb0c9cc
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java
@@ -0,0 +1,75 @@
+package org.apache.phoenix.util;
+
+import java.sql.DatabaseMetaData;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+
+public class QueryUtil {
+	
+    /**
+     *  Column family name index within ResultSet resulting from {@link DatabaseMetaData#getColumns(String, String, String, String)}
+     */
+    public static final int COLUMN_FAMILY_POSITION = 1;
+
+ 	/**
+	 *  Column name index within ResultSet resulting from {@link DatabaseMetaData#getColumns(String, String, String, String)}
+	 */
+	public static final int COLUMN_NAME_POSITION = 4;
+	/**
+	 * Data type index within ResultSet resulting from {@link DatabaseMetaData#getColumns(String, String, String, String)}
+	 */
+	public static final int DATA_TYPE_POSITION = 5;
+
+	/**
+	 * Generates the upsert statement based on number of ColumnInfo. If
+	 * ColumnInfo is unavailable, it produces a generic UPSERT query without
+	 * columns information using number of columns.
+	 * 
+	 * @return Upsert Statement
+	 */
+	public static String constructUpsertStatement(ColumnInfo[] columnTypes,
+			String tableName, int numColumns) {
+		if(numColumns <= 0) {
+			throw new RuntimeException("Number of columns in HBase table cannot be less than 1");
+		}
+		StringBuilder sb = new StringBuilder();
+		sb.append("UPSERT INTO ");
+		sb.append(tableName);
+		if (columnTypes != null) {
+			sb.append("(");
+			for (ColumnInfo columnType : columnTypes) {
+				if (columnType != null) {
+					sb.append(columnType.getColumnName());
+					sb.append(",");
+				}
+			}
+			// Remove the trailing comma
+			sb.setLength(sb.length() - 1);
+			sb.append(") ");
+		}
+		sb.append("\n");
+		sb.append("VALUES (");
+		for (short i = 0; i < numColumns - 1; i++) {
+			sb.append("?,");
+		}
+		sb.append("?)");
+
+		return sb.toString();
+	}
+
+	public static String getUrl(String server) {
+		return PhoenixRuntime.JDBC_PROTOCOL + PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR + server;
+	}
+
+    public static String getExplainPlan(ResultSet rs) throws SQLException {
+        StringBuilder buf = new StringBuilder();
+        while (rs.next()) {
+            buf.append(rs.getString(1));
+            buf.append('\n');
+        }
+        if (buf.length() > 0) {
+            buf.setLength(buf.length()-1);
+        }
+        return buf.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/util/ReadOnlyProps.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/ReadOnlyProps.java b/phoenix-core/src/main/java/org/apache/phoenix/util/ReadOnlyProps.java
new file mode 100644
index 0000000..6fcbc3d
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/ReadOnlyProps.java
@@ -0,0 +1,239 @@
+package org.apache.phoenix.util;
+
+import java.util.*;
+import java.util.Map.Entry;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import com.google.common.collect.*;
+
+/**
+ * 
+ * Read-only properties that avoids unnecessary synchronization in
+ * java.util.Properties.
+ *
+ * @author jtaylor
+ * @since 1.2.2
+ */
+public class ReadOnlyProps implements Iterable<Entry<String, String>> {
+    public static final ReadOnlyProps EMPTY_PROPS = new ReadOnlyProps(Iterators.<Entry<String, String>>emptyIterator());
+    private final Map<String, String> props;
+    
+    public ReadOnlyProps(Iterator<Entry<String, String>> iterator) {
+        Map<String, String> map = Maps.newHashMap();
+        while (iterator.hasNext()) {
+            Entry<String,String> entry = iterator.next();
+            map.put(entry.getKey(), entry.getValue());
+        }
+        this.props = ImmutableMap.copyOf(map);
+    }
+
+    public ReadOnlyProps(Map<String, String> props) {
+        this.props = ImmutableMap.copyOf(props);
+    }
+
+    private static Pattern varPat = Pattern.compile("\\$\\{[^\\}\\$\u0020]+\\}");
+    private static int MAX_SUBST = 20;
+
+    private String substituteVars(String expr) {
+        if (expr == null) {
+          return null;
+        }
+        Matcher match = varPat.matcher("");
+        String eval = expr;
+        for(int s=0; s<MAX_SUBST; s++) {
+          match.reset(eval);
+          if (!match.find()) {
+            return eval;
+          }
+          String var = match.group();
+          var = var.substring(2, var.length()-1); // remove ${ .. }
+          String val = null;
+          try {
+            val = System.getProperty(var);
+          } catch(SecurityException se) {
+          }
+          if (val == null) {
+            val = getRaw(var);
+          }
+          if (val == null) {
+            return eval; // return literal ${var}: var is unbound
+          }
+          // substitute
+          eval = eval.substring(0, match.start())+val+eval.substring(match.end());
+        }
+        throw new IllegalStateException("Variable substitution depth too large: " 
+                                        + MAX_SUBST + " " + expr);
+      }
+      
+    /**
+     * Get the value of the <code>name</code> property, without doing
+     * <a href="#VariableExpansion">variable expansion</a>.
+     * 
+     * @param name the property name.
+     * @return the value of the <code>name</code> property, 
+     *         or null if no such property exists.
+     */
+    public String getRaw(String name) {
+      return props.get(name);
+    }
+
+    public String getRaw(String name, String defaultValue) {
+        String value = getRaw(name);
+        if (value == null) {
+            return defaultValue;
+        }
+        return value;
+      }
+
+    /** 
+     * Get the value of the <code>name</code> property. If no such property 
+     * exists, then <code>defaultValue</code> is returned.
+     * 
+     * @param name property name.
+     * @param defaultValue default value.
+     * @return property value, or <code>defaultValue</code> if the property 
+     *         doesn't exist.                    
+     */
+    public String get(String name, String defaultValue) {
+      return substituteVars(getRaw(name, defaultValue));
+    }
+      
+    /**
+     * Get the value of the <code>name</code> property, <code>null</code> if
+     * no such property exists.
+     * 
+     * Values are processed for <a href="#VariableExpansion">variable expansion</a> 
+     * before being returned. 
+     * 
+     * @param name the property name.
+     * @return the value of the <code>name</code> property, 
+     *         or null if no such property exists.
+     */
+    public String get(String name) {
+      return substituteVars(getRaw(name));
+    }
+
+    private String getHexDigits(String value) {
+        boolean negative = false;
+        String str = value;
+        String hexString = null;
+        if (value.startsWith("-")) {
+          negative = true;
+          str = value.substring(1);
+        }
+        if (str.startsWith("0x") || str.startsWith("0X")) {
+          hexString = str.substring(2);
+          if (negative) {
+            hexString = "-" + hexString;
+          }
+          return hexString;
+        }
+        return null;
+      }
+      
+    /** 
+     * Get the value of the <code>name</code> property as a <code>boolean</code>.  
+     * If no such property is specified, or if the specified value is not a valid
+     * <code>boolean</code>, then <code>defaultValue</code> is returned.
+     * 
+     * @param name property name.
+     * @param defaultValue default value.
+     * @return property value as a <code>boolean</code>, 
+     *         or <code>defaultValue</code>. 
+     */
+    public boolean getBoolean(String name, boolean defaultValue) {
+      String valueString = get(name);
+      if ("true".equals(valueString))
+        return true;
+      else if ("false".equals(valueString))
+        return false;
+      else return defaultValue;
+    }
+
+    /** 
+     * Get the value of the <code>name</code> property as an <code>int</code>.
+     *   
+     * If no such property exists, or if the specified value is not a valid
+     * <code>int</code>, then <code>defaultValue</code> is returned.
+     * 
+     * @param name property name.
+     * @param defaultValue default value.
+     * @return property value as an <code>int</code>, 
+     *         or <code>defaultValue</code>. 
+     */
+    public int getInt(String name, int defaultValue) {
+      String valueString = get(name);
+      if (valueString == null)
+        return defaultValue;
+      try {
+        String hexString = getHexDigits(valueString);
+        if (hexString != null) {
+          return Integer.parseInt(hexString, 16);
+        }
+        return Integer.parseInt(valueString);
+      } catch (NumberFormatException e) {
+        return defaultValue;
+      }
+    }
+    
+    /** 
+     * Get the value of the <code>name</code> property as a <code>long</code>.  
+     * If no such property is specified, or if the specified value is not a valid
+     * <code>long</code>, then <code>defaultValue</code> is returned.
+     * 
+     * @param name property name.
+     * @param defaultValue default value.
+     * @return property value as a <code>long</code>, 
+     *         or <code>defaultValue</code>. 
+     */
+    public long getLong(String name, long defaultValue) {
+      String valueString = get(name);
+      if (valueString == null)
+        return defaultValue;
+      try {
+        String hexString = getHexDigits(valueString);
+        if (hexString != null) {
+          return Long.parseLong(hexString, 16);
+        }
+        return Long.parseLong(valueString);
+      } catch (NumberFormatException e) {
+        return defaultValue;
+      }
+    }
+
+    /** 
+     * Get the value of the <code>name</code> property as a <code>float</code>.  
+     * If no such property is specified, or if the specified value is not a valid
+     * <code>float</code>, then <code>defaultValue</code> is returned.
+     * 
+     * @param name property name.
+     * @param defaultValue default value.
+     * @return property value as a <code>float</code>, 
+     *         or <code>defaultValue</code>. 
+     */
+    public float getFloat(String name, float defaultValue) {
+      String valueString = get(name);
+      if (valueString == null)
+        return defaultValue;
+      try {
+        return Float.parseFloat(valueString);
+      } catch (NumberFormatException e) {
+        return defaultValue;
+      }
+    }
+
+    /**
+     * Get the properties as a <code>Map<String,String></code>
+     * 
+     * @return Map<String,String> 
+     */
+    public Map<String,String> asMap() {
+        return props;
+    }
+    
+    @Override
+    public Iterator<Entry<String, String>> iterator() {
+        return props.entrySet().iterator();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..8339cc0
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/ResultUtil.java
@@ -0,0 +1,177 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.util.Arrays;
+import java.util.Comparator;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * Static class for various methods that would be nice to have added to {@link org.apache.hadoop.hbase.client.Result}.
+ * These methods work off of the raw bytes preventing the explosion of Result into object form.
+ * 
+ * @author jtaylor
+ * @since 0.1
+ */
+public class ResultUtil {
+    public static final Result EMPTY_RESULT = new Result() {
+        @Override
+        public final boolean isEmpty() { return true; }
+    };
+    
+    private ResultUtil() {
+    }
+    
+    /**
+     * Return a pointer into a potentially much bigger byte buffer that points to the key of a Result.
+     * @param r
+     */
+    public static ImmutableBytesWritable getKey(Result r) {
+        return getKey(r, 0);
+    }
+    
+    public static void getKey(Result r, ImmutableBytesWritable key) {
+        key.set(r.getRow());
+        //key.set(getRawBytes(r), getKeyOffset(r), getKeyLength(r));
+    }
+    
+    public static void getKey(KeyValue value, ImmutableBytesWritable key) {
+        key.set(value.getBuffer(), value.getRowOffset(), value.getRowLength());
+    }
+    
+    /**
+     * Return a pointer into a potentially much bigger byte buffer that points to the key of a Result.
+     * Use offset to return a subset of the key bytes, for example to skip the organization ID embedded
+     * in all of our keys.
+     * @param r
+     * @param offset offset added to start of key and subtracted from key length (to select subset of key bytes)
+     */
+    public static ImmutableBytesWritable getKey(Result r, int offset) {
+        return new ImmutableBytesWritable(getRawBytes(r), getKeyOffset(r) + offset, getKeyLength(r) - offset);
+    }
+
+    public static void getKey(Result r, int offset, int length, ImmutableBytesWritable key) {
+        key.set(getRawBytes(r), getKeyOffset(r) + offset, length);
+    }
+
+    /**
+     * Comparator for comparing the keys from two Results in-place, without allocating new byte arrays
+     */
+    public static final Comparator<Result> KEY_COMPARATOR = new Comparator<Result>() {
+
+        @Override
+        public int compare(Result r1, Result r2) {
+            byte[] r1Bytes = getRawBytes(r1);
+            byte[] r2Bytes = getRawBytes(r2);
+            return Bytes.compareTo(r1Bytes, getKeyOffset(r1), getKeyLength(r1), r2Bytes, getKeyOffset(r2), getKeyLength(r2));
+        }
+        
+    };
+    
+    /**
+     * Get the offset into the Result byte array to the key.
+     * @param r
+     * @return
+     */
+    static int getKeyOffset(Result r) {
+        // Special case for when Result was instantiated via KeyValue array (no bytes in that case) versus returned from a scanner
+        return (r.getBytes() == null ? r.raw()[0].getOffset() : (r.getBytes().getOffset() + Bytes.SIZEOF_INT /* KV length in Result */)) + KeyValue.ROW_OFFSET /* key offset in KV */ + Bytes.SIZEOF_SHORT /* key length */;
+    }
+    
+    static int getKeyLength(Result r) {
+        // Key length stored right before key as a short
+        return Bytes.toShort(getRawBytes(r), getKeyOffset(r) - Bytes.SIZEOF_SHORT);
+    }
+    
+    static byte[] getRawBytes(Result r) {
+        // Handle special case for when Result was instantiated via KeyValue array (no bytes in that case) versus returned from a scanner
+        ImmutableBytesWritable rPtr = r.getBytes();
+        if (rPtr != null)
+            return rPtr.get();
+        return r.raw()[0].getBuffer();
+    }
+
+    public static int compareKeys(Result r1, Result r2) {
+        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
+     */
+    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/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/util/SQLCloseable.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/SQLCloseable.java b/phoenix-core/src/main/java/org/apache/phoenix/util/SQLCloseable.java
new file mode 100644
index 0000000..11cf02f
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/SQLCloseable.java
@@ -0,0 +1,34 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.sql.SQLException;
+
+/**
+ * 
+ * Interface for a SQL resource that should be closed
+ * after it is no longer in use.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public interface SQLCloseable {
+    void close() throws SQLException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/util/SQLCloseables.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/SQLCloseables.java b/phoenix-core/src/main/java/org/apache/phoenix/util/SQLCloseables.java
new file mode 100644
index 0000000..c255c1d
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/SQLCloseables.java
@@ -0,0 +1,133 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.sql.SQLException;
+import java.util.*;
+
+import com.google.common.collect.Iterables;
+
+
+/**
+ * Utilities for operating on {@link SQLCloseable}s.
+ * 
+ * @author jtaylor
+ * @since 0.1
+ */
+public class SQLCloseables {
+    /** Not constructed */
+    private SQLCloseables() { }
+    
+    /**
+     * Allows you to close as many of the {@link SQLCloseable}s as possible.
+     * 
+     * If any of the close's fail with an IOException, those exception(s) will
+     * be thrown after attempting to close all of the inputs.
+     */
+    public static void closeAll(Iterable<? extends SQLCloseable> iterable) throws SQLException {
+        SQLException ex = closeAllQuietly(iterable);
+        if (ex != null) throw ex;
+    }
+ 
+    public static SQLException closeAllQuietly(Iterable<? extends SQLCloseable> iterable) {
+        if (iterable == null) return null;
+        
+        LinkedList<SQLException> exceptions = null;
+        for (SQLCloseable closeable : iterable) {
+            try {
+                closeable.close();
+            } catch (SQLException x) {
+                if (exceptions == null) exceptions = new LinkedList<SQLException>();
+                exceptions.add(x);
+            }
+        }
+        
+        SQLException ex = MultipleCausesSQLException.fromSQLExceptions(exceptions);
+        return ex;
+    }
+
+    /**
+     * A subclass of {@link SQLException} that allows you to chain multiple 
+     * causes together.
+     * 
+     * @author jtaylor
+     * @since 0.1
+     * @see SQLCloseables
+     */
+    static private class MultipleCausesSQLException extends SQLException {
+		private static final long serialVersionUID = 1L;
+
+		static SQLException fromSQLExceptions(Collection<? extends SQLException> exceptions) {
+            if (exceptions == null || exceptions.isEmpty()) return null;
+            if (exceptions.size() == 1) return Iterables.getOnlyElement(exceptions);
+            
+            return new MultipleCausesSQLException(exceptions);
+        }
+        
+        private final Collection<? extends SQLException> exceptions;
+        private boolean hasSetStackTrace;
+        
+        /**
+         * Use the {@link #fromIOExceptions(Collection) factory}.
+         */
+        private MultipleCausesSQLException(Collection<? extends SQLException> exceptions) {
+            this.exceptions = exceptions;
+        }
+
+        @Override
+        public String getMessage() {
+            StringBuilder sb = new StringBuilder(this.exceptions.size() * 50);
+            int exceptionNum = 0;
+            for (SQLException ex : this.exceptions) {
+                sb.append("Cause Number " + exceptionNum + ": " + ex.getMessage() + "\n");
+                exceptionNum++;
+            }
+            return sb.toString();
+        }
+        
+        @Override
+        public StackTraceElement[] getStackTrace() {
+            if (!this.hasSetStackTrace) {
+                ArrayList<StackTraceElement> frames = new ArrayList<StackTraceElement>(this.exceptions.size() * 20);
+                
+                int exceptionNum = 0;
+                for (SQLException exception : this.exceptions) {
+                    StackTraceElement header = new StackTraceElement(MultipleCausesSQLException.class.getName(), 
+                            "Exception Number " + exceptionNum, 
+                            "<no file>",
+                            0);
+                    
+                    frames.add(header);
+                    for (StackTraceElement ste : exception.getStackTrace()) {
+                        frames.add(ste);
+                    }
+                    exceptionNum++;
+                }
+                
+                setStackTrace(frames.toArray(new StackTraceElement[frames.size()]));
+                this.hasSetStackTrace = true;
+            }        
+            
+            return super.getStackTrace();
+        }
+
+    }
+    
+}


[21/51] [partial] Initial commit of master branch from github

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
new file mode 100644
index 0000000..8592f49
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
@@ -0,0 +1,1258 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.jdbc;
+
+import java.io.IOException;
+import java.io.Reader;
+import java.sql.ParameterMetaData;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
+import java.sql.SQLWarning;
+import java.sql.Statement;
+import java.text.Format;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.util.Pair;
+
+import com.google.common.collect.ListMultimap;
+import com.google.common.collect.Lists;
+import org.apache.phoenix.compile.ColumnProjector;
+import org.apache.phoenix.compile.CreateIndexCompiler;
+import org.apache.phoenix.compile.CreateSequenceCompiler;
+import org.apache.phoenix.compile.CreateTableCompiler;
+import org.apache.phoenix.compile.DeleteCompiler;
+import org.apache.phoenix.compile.DropSequenceCompiler;
+import org.apache.phoenix.compile.ExplainPlan;
+import org.apache.phoenix.compile.ExpressionProjector;
+import org.apache.phoenix.compile.MutationPlan;
+import org.apache.phoenix.compile.QueryCompiler;
+import org.apache.phoenix.compile.QueryPlan;
+import org.apache.phoenix.compile.RowProjector;
+import org.apache.phoenix.compile.StatementPlan;
+import org.apache.phoenix.compile.UpsertCompiler;
+import org.apache.phoenix.coprocessor.MetaDataProtocol;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+import org.apache.phoenix.execute.MutationState;
+import org.apache.phoenix.expression.RowKeyColumnExpression;
+import org.apache.phoenix.iterate.MaterializedResultIterator;
+import org.apache.phoenix.iterate.ResultIterator;
+import org.apache.phoenix.parse.AddColumnStatement;
+import org.apache.phoenix.parse.AliasedNode;
+import org.apache.phoenix.parse.AlterIndexStatement;
+import org.apache.phoenix.parse.BindableStatement;
+import org.apache.phoenix.parse.ColumnDef;
+import org.apache.phoenix.parse.ColumnName;
+import org.apache.phoenix.parse.CreateIndexStatement;
+import org.apache.phoenix.parse.CreateSequenceStatement;
+import org.apache.phoenix.parse.CreateTableStatement;
+import org.apache.phoenix.parse.DeleteStatement;
+import org.apache.phoenix.parse.DropColumnStatement;
+import org.apache.phoenix.parse.DropIndexStatement;
+import org.apache.phoenix.parse.DropSequenceStatement;
+import org.apache.phoenix.parse.DropTableStatement;
+import org.apache.phoenix.parse.ExplainStatement;
+import org.apache.phoenix.parse.HintNode;
+import org.apache.phoenix.parse.LimitNode;
+import org.apache.phoenix.parse.NamedNode;
+import org.apache.phoenix.parse.NamedTableNode;
+import org.apache.phoenix.parse.OrderByNode;
+import org.apache.phoenix.parse.ParseNode;
+import org.apache.phoenix.parse.ParseNodeFactory;
+import org.apache.phoenix.parse.PrimaryKeyConstraint;
+import org.apache.phoenix.parse.SQLParser;
+import org.apache.phoenix.parse.SelectStatement;
+import org.apache.phoenix.parse.TableName;
+import org.apache.phoenix.parse.TableNode;
+import org.apache.phoenix.parse.UpsertStatement;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.ExecuteQueryNotApplicableException;
+import org.apache.phoenix.schema.ExecuteUpdateNotApplicableException;
+import org.apache.phoenix.schema.MetaDataClient;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.PDatum;
+import org.apache.phoenix.schema.PIndexState;
+import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.schema.RowKeyValueAccessor;
+import org.apache.phoenix.schema.tuple.SingleKeyValueTuple;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.KeyValueUtil;
+import org.apache.phoenix.util.SQLCloseable;
+import org.apache.phoenix.util.SQLCloseables;
+import org.apache.phoenix.util.ServerUtil;
+
+
+/**
+ * 
+ * JDBC Statement implementation of Phoenix.
+ * Currently only the following methods are supported:
+ * - {@link #executeQuery(String)}
+ * - {@link #executeUpdate(String)}
+ * - {@link #execute(String)}
+ * - {@link #getResultSet()}
+ * - {@link #getUpdateCount()}
+ * - {@link #close()}
+ * The Statement only supports the following options:
+ * - ResultSet.FETCH_FORWARD
+ * - ResultSet.TYPE_FORWARD_ONLY
+ * - ResultSet.CLOSE_CURSORS_AT_COMMIT
+ * 
+ * @author jtaylor
+ * @since 0.1
+ */
+public class PhoenixStatement implements Statement, SQLCloseable, org.apache.phoenix.jdbc.Jdbc7Shim.Statement {
+    public enum UpdateOperation {
+        DELETED("deleted"),
+        UPSERTED("upserted");
+        
+        private final String toString;
+        UpdateOperation(String toString) {
+            this.toString = toString;
+        }
+        
+        @Override
+        public String toString() {
+            return toString;
+        }
+        };
+
+    protected final PhoenixConnection connection;
+    private static final int NO_UPDATE = -1;
+    private List<PhoenixResultSet> resultSets = new ArrayList<PhoenixResultSet>();
+    private QueryPlan lastQueryPlan;
+    private PhoenixResultSet lastResultSet;
+    private int lastUpdateCount = NO_UPDATE;
+    private UpdateOperation lastUpdateOperation;
+    private boolean isClosed = false;
+    private ResultSetMetaData resultSetMetaData;
+    private int maxRows;
+    
+    
+    public PhoenixStatement(PhoenixConnection connection) {
+        this.connection = connection;
+    }
+    
+    protected List<PhoenixResultSet> getResultSets() {
+        return resultSets;
+    }
+    
+    protected PhoenixResultSet newResultSet(ResultIterator iterator, RowProjector projector) throws SQLException {
+        return new PhoenixResultSet(iterator, projector, PhoenixStatement.this);
+    }
+    
+    protected static interface ExecutableStatement extends BindableStatement {
+        public boolean execute() throws SQLException;
+        public int executeUpdate() throws SQLException;
+        public PhoenixResultSet executeQuery() throws SQLException;
+        public ResultSetMetaData getResultSetMetaData() throws SQLException;
+        public StatementPlan optimizePlan() throws SQLException;
+        public StatementPlan compilePlan() throws SQLException;
+    }
+    
+    protected static interface MutatableStatement extends ExecutableStatement {
+        @Override
+        public MutationPlan optimizePlan() throws SQLException;
+    }
+    
+    private class ExecutableSelectStatement extends SelectStatement implements ExecutableStatement {
+        private ExecutableSelectStatement(List<? extends TableNode> from, HintNode hint, boolean isDistinct, List<AliasedNode> select, ParseNode where,
+                List<ParseNode> groupBy, ParseNode having, List<OrderByNode> orderBy, LimitNode limit, int bindCount, boolean isAggregate) {
+            super(from, hint, isDistinct, select, where, groupBy, having, orderBy, limit, bindCount, isAggregate);
+        }
+
+        @Override
+        public PhoenixResultSet executeQuery() throws SQLException {
+            QueryPlan plan = optimizePlan();
+            PhoenixResultSet rs = newResultSet(plan.iterator(), plan.getProjector());
+            resultSets.add(rs);
+            lastResultSet = rs;
+            lastUpdateCount = NO_UPDATE;
+            lastUpdateOperation = null;
+            return rs;
+        }
+
+        @Override
+        public boolean execute() throws SQLException {
+            executeQuery();
+            return true;
+        }
+
+        @Override
+        public int executeUpdate() throws SQLException {
+            throw new ExecuteUpdateNotApplicableException(this.toString());
+        }
+
+        @Override
+        public QueryPlan optimizePlan() throws SQLException {
+            return lastQueryPlan = connection.getQueryServices().getOptimizer().optimize(this, PhoenixStatement.this);
+        }
+        
+        @Override
+        public StatementPlan compilePlan() throws SQLException {
+            return new QueryCompiler(PhoenixStatement.this).compile(this);
+        }
+        
+        @Override
+        public ResultSetMetaData getResultSetMetaData() throws SQLException {
+            if (resultSetMetaData == null) {
+                // Just compile top level query without optimizing to get ResultSetMetaData
+                QueryPlan plan = new QueryCompiler(PhoenixStatement.this).compile(this);
+                resultSetMetaData = new PhoenixResultSetMetaData(connection, plan.getProjector());
+            }
+            return resultSetMetaData;
+        }
+    }
+    
+    private int executeMutation(MutationPlan plan) throws SQLException {
+        // Note that the upsert select statements will need to commit any open transaction here,
+        // since they'd update data directly from coprocessors, and should thus operate on
+        // the latest state
+        MutationState state = plan.execute();
+        connection.getMutationState().join(state);
+        if (connection.getAutoCommit()) {
+            connection.commit();
+        }
+        lastResultSet = null;
+        lastQueryPlan = null;
+        // Unfortunately, JDBC uses an int for update count, so we
+        // just max out at Integer.MAX_VALUE
+        long updateCount = state.getUpdateCount();
+        lastUpdateCount = (int)Math.min(Integer.MAX_VALUE, updateCount);
+        return lastUpdateCount;
+    }
+    
+    private class ExecutableUpsertStatement extends UpsertStatement implements MutatableStatement {
+        private ExecutableUpsertStatement(NamedTableNode table, HintNode hintNode, List<ColumnName> columns, List<ParseNode> values, SelectStatement select, int bindCount) {
+            super(table, hintNode, columns, values, select, bindCount);
+        }
+
+        @Override
+        public PhoenixResultSet executeQuery() throws SQLException {
+            throw new ExecuteQueryNotApplicableException("upsert", this.toString());
+        }
+
+        @Override
+        public boolean execute() throws SQLException {
+            executeUpdate();
+            return false;
+        }
+
+        @Override
+        public int executeUpdate() throws SQLException {
+            lastUpdateOperation = UpdateOperation.UPSERTED;
+            return executeMutation(optimizePlan());
+        }
+
+        @Override
+        public ResultSetMetaData getResultSetMetaData() throws SQLException {
+            return null;
+        }
+
+        @Override
+        public MutationPlan compilePlan() throws SQLException {
+            UpsertCompiler compiler = new UpsertCompiler(PhoenixStatement.this);
+            return compiler.compile(this);
+        }
+        
+        @Override
+        public MutationPlan optimizePlan() throws SQLException {
+            return compilePlan();
+        }
+    }
+    
+    private class ExecutableDeleteStatement extends DeleteStatement implements MutatableStatement {
+        private ExecutableDeleteStatement(NamedTableNode table, HintNode hint, ParseNode whereNode, List<OrderByNode> orderBy, LimitNode limit, int bindCount) {
+            super(table, hint, whereNode, orderBy, limit, bindCount);
+        }
+
+        @Override
+        public PhoenixResultSet executeQuery() throws SQLException {
+            throw new ExecuteQueryNotApplicableException("delete", this.toString());
+        }
+
+        @Override
+        public boolean execute() throws SQLException {
+            executeUpdate();
+            return false;
+        }
+
+        @Override
+        public int executeUpdate() throws SQLException {
+            lastUpdateOperation = UpdateOperation.DELETED;
+            return executeMutation(optimizePlan());
+        }
+
+        @Override
+        public ResultSetMetaData getResultSetMetaData() throws SQLException {
+            return null;
+        }
+
+        @Override
+        public MutationPlan compilePlan() throws SQLException {
+            DeleteCompiler compiler = new DeleteCompiler(PhoenixStatement.this);
+            return compiler.compile(this);
+        }
+        
+        @Override
+        public MutationPlan optimizePlan() throws SQLException {
+            return compilePlan();
+        }
+    }
+    
+    private class ExecutableCreateTableStatement extends CreateTableStatement implements ExecutableStatement {
+        ExecutableCreateTableStatement(TableName tableName, ListMultimap<String,Pair<String,Object>> props, List<ColumnDef> columnDefs,
+                PrimaryKeyConstraint pkConstraint, List<ParseNode> splitNodes, PTableType tableType, boolean ifNotExists,
+                TableName baseTableName, ParseNode tableTypeIdNode, int bindCount) {
+            super(tableName, props, columnDefs, pkConstraint, splitNodes, tableType, ifNotExists, baseTableName, tableTypeIdNode, bindCount);
+        }
+
+        @Override
+        public PhoenixResultSet executeQuery() throws SQLException {
+            throw new ExecuteQueryNotApplicableException("CREATE TABLE", this.toString());
+        }
+
+        @Override
+        public boolean execute() throws SQLException {
+            executeUpdate();
+            return false;
+        }
+
+        @Override
+        public int executeUpdate() throws SQLException {
+            MutationPlan plan = optimizePlan();
+            MutationState state = plan.execute();
+            lastQueryPlan = null;
+            lastResultSet = null;
+            lastUpdateCount = (int)Math.min(state.getUpdateCount(), Integer.MAX_VALUE);
+            lastUpdateOperation = UpdateOperation.UPSERTED;
+            return lastUpdateCount;
+        }
+
+        @Override
+        public ResultSetMetaData getResultSetMetaData() throws SQLException {
+            return null;
+        }
+
+        @Override
+        public MutationPlan compilePlan() throws SQLException {
+            CreateTableCompiler compiler = new CreateTableCompiler(PhoenixStatement.this);
+            return compiler.compile(this);
+        }
+        
+        @Override
+        public MutationPlan optimizePlan() throws SQLException {
+            return compilePlan();
+        }
+    }
+
+    private class ExecutableCreateIndexStatement extends CreateIndexStatement implements ExecutableStatement {
+
+        public ExecutableCreateIndexStatement(NamedNode indexName, NamedTableNode dataTable, PrimaryKeyConstraint pkConstraint, List<ColumnName> includeColumns, List<ParseNode> splits,
+                ListMultimap<String,Pair<String,Object>> props, boolean ifNotExists, int bindCount) {
+            super(indexName, dataTable, pkConstraint, includeColumns, splits, props, ifNotExists, bindCount);
+        }
+
+        @Override
+        public PhoenixResultSet executeQuery() throws SQLException {
+            throw new ExecuteQueryNotApplicableException("CREATE INDEX", this.toString());
+        }
+
+        @Override
+        public boolean execute() throws SQLException {
+            executeUpdate();
+            return false;
+        }
+
+        @Override
+        public int executeUpdate() throws SQLException {
+            MutationPlan plan = optimizePlan();
+            MutationState state = plan.execute();
+            lastQueryPlan = null;
+            lastResultSet = null;
+            lastUpdateCount = (int)Math.min(state.getUpdateCount(), Integer.MAX_VALUE);
+            lastUpdateOperation = UpdateOperation.UPSERTED;
+            return lastUpdateCount;
+        }
+
+        @Override
+        public ResultSetMetaData getResultSetMetaData() throws SQLException {
+            return null;
+        }
+
+        @Override
+        public MutationPlan compilePlan() throws SQLException {
+            CreateIndexCompiler compiler = new CreateIndexCompiler(PhoenixStatement.this);
+            return compiler.compile(this);
+        }
+        
+        @Override
+        public MutationPlan optimizePlan() throws SQLException {
+            return compilePlan();
+        }
+    }
+    
+    private class ExecutableCreateSequenceStatement extends	CreateSequenceStatement implements ExecutableStatement {
+
+		public ExecutableCreateSequenceStatement(TableName sequenceName, ParseNode startWith, ParseNode incrementBy, ParseNode cacheSize, boolean ifNotExists, int bindCount) {
+			super(sequenceName, startWith, incrementBy, cacheSize, ifNotExists, bindCount);
+		}
+
+		@Override
+		public PhoenixResultSet executeQuery() throws SQLException {
+			throw new ExecuteQueryNotApplicableException("CREATE SEQUENCE",	this.toString());
+		}
+
+		@Override
+		public boolean execute() throws SQLException {
+		    executeUpdate();
+		    return false;
+		}
+
+		@Override
+		public int executeUpdate() throws SQLException {
+            MutationPlan plan = optimizePlan();
+            MutationState state = plan.execute();
+            lastQueryPlan = null;
+            lastResultSet = null;
+            lastUpdateCount = (int)Math.min(state.getUpdateCount(), Integer.MAX_VALUE);
+            lastUpdateOperation = UpdateOperation.UPSERTED;
+            return lastUpdateCount;
+		}
+
+		@Override
+		public ResultSetMetaData getResultSetMetaData() throws SQLException {
+			return null;
+		}
+
+		@Override
+		public MutationPlan compilePlan() throws SQLException {
+		    CreateSequenceCompiler compiler = new CreateSequenceCompiler(PhoenixStatement.this);
+            return compiler.compile(this);
+		}
+
+        @Override
+        public MutationPlan optimizePlan() throws SQLException {
+            return compilePlan();
+        }
+	}
+
+    private class ExecutableDropSequenceStatement extends DropSequenceStatement implements ExecutableStatement {
+
+
+        public ExecutableDropSequenceStatement(TableName sequenceName, boolean ifExists, int bindCount) {
+            super(sequenceName, ifExists, bindCount);
+        }
+
+        @Override
+        public PhoenixResultSet executeQuery() throws SQLException {
+            throw new ExecuteQueryNotApplicableException("DROP SEQUENCE", this.toString());
+        }
+
+        @Override
+        public boolean execute() throws SQLException {
+            executeUpdate();
+            return false;
+        }
+
+        @Override
+        public int executeUpdate() throws SQLException {
+            MutationPlan plan = optimizePlan();
+            MutationState state = plan.execute();
+            lastQueryPlan = null;
+            lastResultSet = null;
+            lastUpdateCount = (int)Math.min(state.getUpdateCount(), Integer.MAX_VALUE);
+            lastUpdateOperation = UpdateOperation.UPSERTED;
+            return lastUpdateCount;
+        }
+
+        @Override
+        public ResultSetMetaData getResultSetMetaData() throws SQLException {
+            return null;
+        }
+
+        @Override
+        public MutationPlan compilePlan() throws SQLException {
+            DropSequenceCompiler compiler = new DropSequenceCompiler(PhoenixStatement.this);
+            return compiler.compile(this);
+        }
+
+        @Override
+        public MutationPlan optimizePlan() throws SQLException {
+            return compilePlan();
+        }
+    }
+
+    private class ExecutableDropTableStatement extends DropTableStatement implements ExecutableStatement {
+
+        ExecutableDropTableStatement(TableName tableName, PTableType tableType, boolean ifExists) {
+            super(tableName, tableType, ifExists);
+        }
+
+        @Override
+        public PhoenixResultSet executeQuery() throws SQLException {
+            throw new ExecuteQueryNotApplicableException("DROP TABLE", this.toString());
+        }
+
+        @Override
+        public boolean execute() throws SQLException {
+            executeUpdate();
+            return false;
+        }
+
+        @Override
+        public int executeUpdate() throws SQLException {
+            MetaDataClient client = new MetaDataClient(connection);
+            MutationState state = client.dropTable(this);
+            lastQueryPlan = null;
+            lastResultSet = null;
+            lastUpdateCount = (int)Math.min(state.getUpdateCount(), Integer.MAX_VALUE);
+            lastUpdateOperation = UpdateOperation.DELETED;
+            return lastUpdateCount;
+        }
+
+        @Override
+        public ResultSetMetaData getResultSetMetaData() throws SQLException {
+            return null;
+        }
+
+        @Override
+        public StatementPlan compilePlan() throws SQLException {
+            return new StatementPlan() {
+
+                @Override
+                public ParameterMetaData getParameterMetaData() {
+                    return PhoenixParameterMetaData.EMPTY_PARAMETER_META_DATA;
+                }
+
+                @Override
+                public ExplainPlan getExplainPlan() throws SQLException {
+                    return new ExplainPlan(Collections.singletonList("DROP TABLE"));
+                }
+            };
+        }
+        
+        @Override
+        public StatementPlan optimizePlan() throws SQLException {
+            return compilePlan();
+        }
+    }
+
+    private class ExecutableDropIndexStatement extends DropIndexStatement implements ExecutableStatement {
+
+        public ExecutableDropIndexStatement(NamedNode indexName, TableName tableName, boolean ifExists) {
+            super(indexName, tableName, ifExists);
+        }
+
+        @Override
+        public PhoenixResultSet executeQuery() throws SQLException {
+            throw new ExecuteQueryNotApplicableException("DROP INDEX", this.toString());
+        }
+
+        @Override
+        public boolean execute() throws SQLException {
+            executeUpdate();
+            return false;
+        }
+
+        @Override
+        public int executeUpdate() throws SQLException {
+            MetaDataClient client = new MetaDataClient(connection);
+            MutationState state = client.dropIndex(this);
+            lastQueryPlan = null;
+            lastResultSet = null;
+            lastUpdateCount = (int)Math.min(state.getUpdateCount(), Integer.MAX_VALUE);
+            lastUpdateOperation = UpdateOperation.DELETED;
+            return lastUpdateCount;
+        }
+
+        @Override
+        public ResultSetMetaData getResultSetMetaData() throws SQLException {
+            return null;
+        }
+
+        @Override
+        public StatementPlan compilePlan() throws SQLException {
+            return new StatementPlan() {
+                
+                @Override
+                public ParameterMetaData getParameterMetaData() {
+                    return PhoenixParameterMetaData.EMPTY_PARAMETER_META_DATA;
+                }
+                
+                @Override
+                public ExplainPlan getExplainPlan() throws SQLException {
+                    return new ExplainPlan(Collections.singletonList("DROP INDEX"));
+                }
+            };
+        }
+        
+        @Override
+        public StatementPlan optimizePlan() throws SQLException {
+            return compilePlan();
+        }
+    }
+
+    private class ExecutableAlterIndexStatement extends AlterIndexStatement implements ExecutableStatement {
+
+        public ExecutableAlterIndexStatement(NamedTableNode indexTableNode, String dataTableName, boolean ifExists, PIndexState state) {
+            super(indexTableNode, dataTableName, ifExists, state);
+        }
+
+        @Override
+        public PhoenixResultSet executeQuery() throws SQLException {
+            throw new ExecuteQueryNotApplicableException("ALTER INDEX", this.toString());
+        }
+
+        @Override
+        public boolean execute() throws SQLException {
+            executeUpdate();
+            return false;
+        }
+
+        @Override
+        public int executeUpdate() throws SQLException {
+            MetaDataClient client = new MetaDataClient(connection);
+            MutationState state = client.alterIndex(this);
+            lastQueryPlan = null;
+            lastResultSet = null;
+            lastUpdateCount = (int)Math.min(state.getUpdateCount(), Integer.MAX_VALUE);
+            lastUpdateOperation = UpdateOperation.UPSERTED;
+            return lastUpdateCount;
+        }
+
+        @Override
+        public ResultSetMetaData getResultSetMetaData() throws SQLException {
+            return null;
+        }
+
+        @Override
+        public StatementPlan compilePlan() throws SQLException {
+            return new StatementPlan() {
+                
+                @Override
+                public ParameterMetaData getParameterMetaData() {
+                    return PhoenixParameterMetaData.EMPTY_PARAMETER_META_DATA;
+                }
+                
+                @Override
+                public ExplainPlan getExplainPlan() throws SQLException {
+                    return new ExplainPlan(Collections.singletonList("ALTER INDEX"));
+                }
+            };
+        }
+        
+        @Override
+        public StatementPlan optimizePlan() throws SQLException {
+            return compilePlan();
+        }
+    }
+
+    private class ExecutableAddColumnStatement extends AddColumnStatement implements ExecutableStatement {
+
+        ExecutableAddColumnStatement(NamedTableNode table, PTableType tableType, List<ColumnDef> columnDefs, boolean ifNotExists, Map<String, Object> props) {
+            super(table, tableType, columnDefs, ifNotExists, props);
+        }
+
+        @Override
+        public PhoenixResultSet executeQuery() throws SQLException {
+            throw new ExecuteQueryNotApplicableException("ALTER TABLE", this.toString());
+        }
+
+        @Override
+        public boolean execute() throws SQLException {
+            executeUpdate();
+            return false;
+        }
+
+        @Override
+        public int executeUpdate() throws SQLException {
+            MetaDataClient client = new MetaDataClient(connection);
+            MutationState state = client.addColumn(this);
+            lastQueryPlan = null;
+            lastResultSet = null;
+            lastUpdateCount = (int)Math.min(state.getUpdateCount(), Integer.MAX_VALUE);
+            lastUpdateOperation = UpdateOperation.UPSERTED;
+            return lastUpdateCount;
+        }
+
+        @Override
+        public ResultSetMetaData getResultSetMetaData() throws SQLException {
+            return null;
+        }
+
+        @Override
+        public StatementPlan compilePlan() throws SQLException {
+            return new StatementPlan() {
+
+                @Override
+                public ParameterMetaData getParameterMetaData() {
+                    return PhoenixParameterMetaData.EMPTY_PARAMETER_META_DATA;
+                }
+
+                @Override
+                public ExplainPlan getExplainPlan() throws SQLException {
+                    return new ExplainPlan(Collections.singletonList("ALTER " + getTableType() + " ADD COLUMN"));
+                }
+            };
+        }
+        
+        @Override
+        public StatementPlan optimizePlan() throws SQLException {
+            return compilePlan();
+        }
+    }
+
+    private class ExecutableDropColumnStatement extends DropColumnStatement implements ExecutableStatement {
+
+        ExecutableDropColumnStatement(NamedTableNode table, PTableType tableType, List<ColumnName> columnRefs, boolean ifExists) {
+            super(table, tableType, columnRefs, ifExists);
+        }
+
+        @Override
+        public PhoenixResultSet executeQuery() throws SQLException {
+            throw new ExecuteQueryNotApplicableException("ALTER TABLE", this.toString());
+        }
+
+        @Override
+        public boolean execute() throws SQLException {
+            executeUpdate();
+            return false;
+        }
+
+        @Override
+        public int executeUpdate() throws SQLException {
+            MetaDataClient client = new MetaDataClient(connection);
+            MutationState state = client.dropColumn(this);
+            lastQueryPlan = null;
+            lastResultSet = null;
+            lastUpdateCount = (int)Math.min(state.getUpdateCount(), Integer.MAX_VALUE);
+            lastUpdateOperation = UpdateOperation.UPSERTED;
+            return lastUpdateCount;
+        }
+
+        @Override
+        public ResultSetMetaData getResultSetMetaData() throws SQLException {
+            return null;
+        }
+
+        @Override
+        public StatementPlan compilePlan() throws SQLException {
+            return new StatementPlan() {
+
+                @Override
+                public ParameterMetaData getParameterMetaData() {
+                    return new PhoenixParameterMetaData(0);
+                }
+
+                @Override
+                public ExplainPlan getExplainPlan() throws SQLException {
+                    return new ExplainPlan(Collections.singletonList("ALTER " + getTableType() + " DROP COLUMN"));
+                }
+            };
+        }
+        
+        @Override
+        public StatementPlan optimizePlan() throws SQLException {
+            return compilePlan();
+        }
+    }
+
+    private static final byte[] EXPLAIN_PLAN_FAMILY = QueryConstants.SINGLE_COLUMN_FAMILY;
+    private static final byte[] EXPLAIN_PLAN_COLUMN = PDataType.VARCHAR.toBytes("Plan");
+    private static final String EXPLAIN_PLAN_ALIAS = "PLAN";
+    private static final String EXPLAIN_PLAN_TABLE_NAME = "PLAN_TABLE";
+    private static final PDatum EXPLAIN_PLAN_DATUM = new PDatum() {
+        @Override
+        public boolean isNullable() {
+            return false;
+        }
+        @Override
+        public PDataType getDataType() {
+            return PDataType.VARCHAR;
+        }
+        @Override
+        public Integer getByteSize() {
+            return null;
+        }
+        @Override
+        public Integer getMaxLength() {
+            return null;
+        }
+        @Override
+        public Integer getScale() {
+            return null;
+        }
+		@Override
+		public ColumnModifier getColumnModifier() {
+			return null;
+		}
+    };
+    private static final RowProjector EXPLAIN_PLAN_ROW_PROJECTOR = new RowProjector(Arrays.<ColumnProjector>asList(
+            new ExpressionProjector(EXPLAIN_PLAN_ALIAS, EXPLAIN_PLAN_TABLE_NAME, 
+                    new RowKeyColumnExpression(EXPLAIN_PLAN_DATUM,
+                            new RowKeyValueAccessor(Collections.<PDatum>singletonList(EXPLAIN_PLAN_DATUM), 0)), false)
+            ), 0, true);
+    private class ExecutableExplainStatement extends ExplainStatement implements ExecutableStatement {
+
+        public ExecutableExplainStatement(BindableStatement statement) {
+            super(statement);
+        }
+
+        @Override
+        public ExecutableStatement getStatement() {
+            return (ExecutableStatement) super.getStatement();
+        }
+        
+        @Override
+        public int getBindCount() {
+            return getStatement().getBindCount();
+        }
+
+        @Override
+        public PhoenixResultSet executeQuery() throws SQLException {
+            StatementPlan plan = getStatement().optimizePlan();
+            List<String> planSteps = plan.getExplainPlan().getPlanSteps();
+            List<Tuple> tuples = Lists.newArrayListWithExpectedSize(planSteps.size());
+            for (String planStep : planSteps) {
+                Tuple tuple = new SingleKeyValueTuple(KeyValueUtil.newKeyValue(PDataType.VARCHAR.toBytes(planStep), EXPLAIN_PLAN_FAMILY, EXPLAIN_PLAN_COLUMN, MetaDataProtocol.MIN_TABLE_TIMESTAMP, ByteUtil.EMPTY_BYTE_ARRAY));
+                tuples.add(tuple);
+            }
+            PhoenixResultSet rs = new PhoenixResultSet(new MaterializedResultIterator(tuples),EXPLAIN_PLAN_ROW_PROJECTOR, new PhoenixStatement(connection));
+            lastResultSet = rs;
+            lastQueryPlan = null;
+            lastUpdateCount = NO_UPDATE;
+            return rs;
+        }
+
+        @Override
+        public boolean execute() throws SQLException {
+            executeQuery();
+            return true;
+        }
+
+        @Override
+        public int executeUpdate() throws SQLException {
+            throw new ExecuteUpdateNotApplicableException("ALTER TABLE", this.toString());
+        }
+
+        @Override
+        public ResultSetMetaData getResultSetMetaData() throws SQLException {
+            return new PhoenixResultSetMetaData(connection, EXPLAIN_PLAN_ROW_PROJECTOR);
+        }
+
+        @Override
+        public StatementPlan compilePlan() throws SQLException {
+            return StatementPlan.EMPTY_PLAN;
+        }
+        
+        @Override
+        public StatementPlan optimizePlan() throws SQLException {
+            return compilePlan();
+        }
+    }
+
+    protected class ExecutableNodeFactory extends ParseNodeFactory {
+        @Override
+        public ExecutableSelectStatement select(List<? extends TableNode> from, HintNode hint, boolean isDistinct, List<AliasedNode> select,
+                                                ParseNode where, List<ParseNode> groupBy, ParseNode having,
+                                                List<OrderByNode> orderBy, LimitNode limit, int bindCount, boolean isAggregate) {
+            return new ExecutableSelectStatement(from, hint, isDistinct, select, where, groupBy == null ? Collections.<ParseNode>emptyList() : groupBy, having, orderBy == null ? Collections.<OrderByNode>emptyList() : orderBy, limit, bindCount, isAggregate);
+        }
+        
+        @Override
+        public ExecutableUpsertStatement upsert(NamedTableNode table, HintNode hintNode, List<ColumnName> columns, List<ParseNode> values, SelectStatement select, int bindCount) {
+            return new ExecutableUpsertStatement(table, hintNode, columns, values, select, bindCount);
+        }
+        
+        @Override
+        public ExecutableDeleteStatement delete(NamedTableNode table, HintNode hint, ParseNode whereNode, List<OrderByNode> orderBy, LimitNode limit, int bindCount) {
+            return new ExecutableDeleteStatement(table, hint, whereNode, orderBy, limit, bindCount);
+        }
+        
+        @Override
+        public CreateTableStatement createTable(TableName tableName, ListMultimap<String,Pair<String,Object>> props, List<ColumnDef> columns, PrimaryKeyConstraint pkConstraint,
+                List<ParseNode> splits, PTableType tableType, boolean ifNotExists, TableName baseTableName, ParseNode tableTypeIdNode, int bindCount) {
+            return new ExecutableCreateTableStatement(tableName, props, columns, pkConstraint, splits, tableType, ifNotExists, baseTableName, tableTypeIdNode, bindCount);
+        }
+        
+        @Override
+        public CreateSequenceStatement createSequence(TableName tableName, ParseNode startsWith, ParseNode incrementBy, ParseNode cacheSize, boolean ifNotExists, int bindCount){
+        	return new ExecutableCreateSequenceStatement(tableName, startsWith, incrementBy, cacheSize, ifNotExists, bindCount);
+        }
+        
+        @Override
+        public DropSequenceStatement dropSequence(TableName tableName, boolean ifExists, int bindCount){
+            return new ExecutableDropSequenceStatement(tableName, ifExists, bindCount);
+        }
+        
+        @Override
+        public CreateIndexStatement createIndex(NamedNode indexName, NamedTableNode dataTable, PrimaryKeyConstraint pkConstraint, List<ColumnName> includeColumns, List<ParseNode> splits, ListMultimap<String,Pair<String,Object>> props, boolean ifNotExists, int bindCount) {
+            return new ExecutableCreateIndexStatement(indexName, dataTable, pkConstraint, includeColumns, splits, props, ifNotExists, bindCount);
+        }
+        
+        @Override
+        public AddColumnStatement addColumn(NamedTableNode table,  PTableType tableType, List<ColumnDef> columnDefs, boolean ifNotExists, Map<String,Object> props) {
+            return new ExecutableAddColumnStatement(table, tableType, columnDefs, ifNotExists, props);
+        }
+        
+        @Override
+        public DropColumnStatement dropColumn(NamedTableNode table,  PTableType tableType, List<ColumnName> columnNodes, boolean ifExists) {
+            return new ExecutableDropColumnStatement(table, tableType, columnNodes, ifExists);
+        }
+        
+        @Override
+        public DropTableStatement dropTable(TableName tableName, PTableType tableType, boolean ifExists) {
+            return new ExecutableDropTableStatement(tableName, tableType, ifExists);
+        }
+        
+        @Override
+        public DropIndexStatement dropIndex(NamedNode indexName, TableName tableName, boolean ifExists) {
+            return new ExecutableDropIndexStatement(indexName, tableName, ifExists);
+        }
+        
+        @Override
+        public AlterIndexStatement alterIndex(NamedTableNode indexTableNode, String dataTableName, boolean ifExists, PIndexState state) {
+            return new ExecutableAlterIndexStatement(indexTableNode, dataTableName, ifExists, state);
+        }
+        
+        @Override
+        public ExplainStatement explain(BindableStatement statement) {
+            return new ExecutableExplainStatement(statement);
+        }
+    }
+    
+    static class PhoenixStatementParser extends SQLParser {
+        PhoenixStatementParser(String query, ParseNodeFactory nodeFactory) throws IOException {
+            super(query, nodeFactory);
+        }
+
+        PhoenixStatementParser(Reader reader) throws IOException {
+            super(reader);
+        }
+        
+        @Override
+        public ExecutableStatement nextStatement(ParseNodeFactory nodeFactory) throws SQLException {
+            return (ExecutableStatement) super.nextStatement(nodeFactory);
+        }
+
+        @Override
+        public ExecutableStatement parseStatement() throws SQLException {
+            return (ExecutableStatement) super.parseStatement();
+        }
+    }
+    
+    public Format getFormatter(PDataType type) {
+        return connection.getFormatter(type);
+    }
+    
+    @Override
+    public void addBatch(String sql) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void cancel() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void clearBatch() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void clearWarnings() throws SQLException {
+    }
+
+    @Override
+    public void close() throws SQLException {
+        try {
+            List<PhoenixResultSet> resultSets = this.resultSets;
+            // Create new list so that remove of the PhoenixResultSet
+            // during closeAll doesn't needless do a linear search
+            // on this list.
+            this.resultSets = Lists.newArrayList();
+            SQLCloseables.closeAll(resultSets);
+        } finally {
+            try {
+                connection.removeStatement(this);
+            } finally {
+                isClosed = true;
+            }
+        }
+    }
+
+    public List<Object> getParameters() {
+        return Collections.<Object>emptyList();
+    }
+    
+    protected ExecutableStatement parseStatement(String sql) throws SQLException {
+        PhoenixStatementParser parser = null;
+        try {
+            parser = new PhoenixStatementParser(sql, new ExecutableNodeFactory());
+        } catch (IOException e) {
+            throw ServerUtil.parseServerException(e);
+        }
+        ExecutableStatement statement = parser.parseStatement();
+        return statement;
+    }
+    
+    @Override
+    public boolean execute(String sql) throws SQLException {
+        return parseStatement(sql).execute();
+    }
+
+    public QueryPlan optimizeQuery(String sql) throws SQLException {
+        return (QueryPlan)parseStatement(sql).optimizePlan();
+    }
+
+    public QueryPlan compileQuery(String sql) throws SQLException {
+        return (QueryPlan)parseStatement(sql).compilePlan();
+    }
+
+    @Override
+    public ResultSet executeQuery(String sql) throws SQLException {
+        return parseStatement(sql).executeQuery();
+    }
+
+    @Override
+    public int executeUpdate(String sql) throws SQLException {
+        return parseStatement(sql).executeUpdate();
+    }
+
+    @Override
+    public boolean execute(String sql, int autoGeneratedKeys) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public boolean execute(String sql, int[] columnIndexes) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public boolean execute(String sql, String[] columnNames) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public int[] executeBatch() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public int executeUpdate(String sql, int autoGeneratedKeys) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public int executeUpdate(String sql, int[] columnIndexes) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public int executeUpdate(String sql, String[] columnNames) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public PhoenixConnection getConnection() {
+        return connection;
+    }
+
+    @Override
+    public int getFetchDirection() throws SQLException {
+        return ResultSet.FETCH_FORWARD;
+    }
+
+    @Override
+    public int getFetchSize() throws SQLException {
+        return connection.getQueryServices().getProps().getInt(QueryServices.SCAN_CACHE_SIZE_ATTRIB, QueryServicesOptions.DEFAULT_SCAN_CACHE_SIZE);
+    }
+
+    @Override
+    public ResultSet getGeneratedKeys() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public int getMaxFieldSize() throws SQLException {
+        return 0; // TODO: 4000?
+    }
+
+    @Override
+    public int getMaxRows() throws SQLException {
+        return maxRows;
+    }
+
+    @Override
+    public boolean getMoreResults() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean getMoreResults(int current) throws SQLException {
+        return false;
+    }
+
+    @Override
+    public int getQueryTimeout() throws SQLException {
+        return connection.getQueryServices().getProps().getInt(QueryServices.KEEP_ALIVE_MS_ATTRIB, 0) / 1000;
+    }
+
+    // For testing
+    public QueryPlan getQueryPlan() {
+        return lastQueryPlan;
+    }
+    
+    @Override
+    public ResultSet getResultSet() throws SQLException {
+        ResultSet rs = lastResultSet;
+        lastResultSet = null;
+        return rs;
+    }
+
+    @Override
+    public int getResultSetConcurrency() throws SQLException {
+        return ResultSet.CONCUR_READ_ONLY;
+    }
+
+    @Override
+    public int getResultSetHoldability() throws SQLException {
+        // TODO: not sure this matters
+        return ResultSet.CLOSE_CURSORS_AT_COMMIT;
+    }
+
+    @Override
+    public int getResultSetType() throws SQLException {
+        return ResultSet.TYPE_FORWARD_ONLY;
+    }
+
+    public UpdateOperation getUpdateOperation() {
+        return lastUpdateOperation;
+    }
+    
+    @Override
+    public int getUpdateCount() throws SQLException {
+        int updateCount = lastUpdateCount;
+        // Only first call can get the update count, otherwise
+        // some SQL clients get into an infinite loop when an
+        // update occurs.
+        lastUpdateCount = NO_UPDATE;
+        return updateCount;
+    }
+
+    @Override
+    public SQLWarning getWarnings() throws SQLException {
+        return null;
+    }
+
+    @Override
+    public boolean isClosed() throws SQLException {
+        return isClosed;
+    }
+
+    @Override
+    public boolean isPoolable() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public void setCursorName(String name) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void setEscapeProcessing(boolean enable) throws SQLException {
+        // TODO: any escaping we need to do?
+    }
+
+    @Override
+    public void setFetchDirection(int direction) throws SQLException {
+        if (direction != ResultSet.FETCH_FORWARD) {
+            throw new SQLFeatureNotSupportedException();
+        }
+    }
+
+    @Override
+    public void setFetchSize(int rows) throws SQLException {
+        // TODO: map to Scan.setBatch() ?
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void setMaxFieldSize(int max) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void setMaxRows(int max) throws SQLException {
+        this.maxRows = max;
+    }
+
+    @Override
+    public void setPoolable(boolean poolable) throws SQLException {
+        if (poolable) {
+            throw new SQLFeatureNotSupportedException();
+        }
+    }
+
+    @Override
+    public void setQueryTimeout(int seconds) throws SQLException {
+        // The Phoenix setting for this is shared across all connections currently
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public boolean isWrapperFor(Class<?> iface) throws SQLException {
+        return iface.isInstance(this);
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public <T> T unwrap(Class<T> iface) throws SQLException {
+        if (!iface.isInstance(this)) {
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode.CLASS_NOT_UNWRAPPABLE)
+                .setMessage(this.getClass().getName() + " not unwrappable from " + iface.getName())
+                .build().buildException();
+        }
+        return (T)this;
+    }
+
+    @Override
+    public void closeOnCompletion() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public boolean isCloseOnCompletion() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatementFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatementFactory.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatementFactory.java
new file mode 100644
index 0000000..5081049
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatementFactory.java
@@ -0,0 +1,24 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.jdbc;
+
+public interface PhoenixStatementFactory {
+    public PhoenixStatement newStatement(PhoenixConnection connection);
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/job/AbstractRoundRobinQueue.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/job/AbstractRoundRobinQueue.java b/phoenix-core/src/main/java/org/apache/phoenix/job/AbstractRoundRobinQueue.java
new file mode 100644
index 0000000..e77e4ca
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/job/AbstractRoundRobinQueue.java
@@ -0,0 +1,314 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.job;
+
+import java.util.*;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.TimeUnit;
+
+/**
+ *
+ * An bounded blocking queue implementation that keeps a virtual queue of elements on per-producer
+ * basis and iterates through each producer queue in round robin fashion.
+ *
+ */
+public abstract class AbstractRoundRobinQueue<E> extends AbstractQueue<E>
+        implements BlockingQueue<E>{
+
+    /**
+     * Construct an AbstractBlockingRoundRobinQueue that limits the size of the queued elements
+     * to at most maxSize. Attempts to insert new elements after that point will cause the
+     * caller to block.
+     * @param maxSize
+     */
+    public AbstractRoundRobinQueue(int maxSize) {
+        this(maxSize, false);
+    }
+    /**
+     * @param newProducerToFront If true, new producers go to the front of the round-robin list, if false, they go to the end.
+     */
+    public AbstractRoundRobinQueue(int maxSize, boolean newProducerToFront) {
+        this.producerMap = new HashMap<Object,ProducerList<E>>();
+        this.producerLists = new LinkedList<ProducerList<E>>();
+        this.lock = new Object();
+        this.newProducerToFront = newProducerToFront;
+        this.maxSize = maxSize;
+    }
+
+    @Override
+    public Iterator<E> iterator() {
+        synchronized(lock) {
+            ArrayList<E> allElements = new ArrayList<E>(this.size);
+            ListIterator<ProducerList<E>> iter = this.producerLists.listIterator(this.currentProducer);
+            while(iter.hasNext()) {
+                ProducerList<E> tList = iter.next();
+                allElements.addAll(tList.list);
+            }
+            return allElements.iterator();
+        }
+    }
+
+    @Override
+    public boolean offer(E o, long timeout, TimeUnit unit) throws InterruptedException {
+        boolean taken = false;
+        long endAt = System.currentTimeMillis() + unit.toMillis(timeout);
+        synchronized(lock) {
+            long waitTime = endAt - System.currentTimeMillis();
+            while (!(taken = offer(o)) && waitTime > 0) {
+                this.lock.wait(waitTime);
+                waitTime = endAt - System.currentTimeMillis();
+            }
+        }
+        return taken;
+    }
+
+    @Override
+    public boolean offer(E o) {
+        if (o == null)
+            throw new NullPointerException();
+
+        final Object producerKey = extractProducer(o);
+
+        ProducerList<E> producerList = null;
+        synchronized(lock) {
+            if (this.size == this.maxSize) {
+                return false;
+            }
+            producerList = this.producerMap.get(producerKey);
+            if (producerList == null) {
+                producerList = new ProducerList<E>(producerKey);
+                this.producerMap.put(producerKey, producerList);
+                this.producerLists.add(this.currentProducer, producerList);
+                if (!this.newProducerToFront) {
+                    incrementCurrentProducerPointer();
+                }
+            }
+            producerList.list.add(o);
+            this.size++;
+            lock.notifyAll();
+        }
+        return true;
+    }
+    
+    /**
+     * Implementations must extracts the producer object which is used as the key to identify a unique producer.
+     */
+    protected abstract Object extractProducer(E o);
+
+    @Override
+    public void put(E o) {
+        offer(o);
+    }
+
+    @Override
+    public E take() throws InterruptedException {
+        synchronized(lock) {
+            while (this.size == 0) {
+                this.lock.wait();
+            }
+            E element = poll();
+            assert element != null;
+            return element;
+        }
+    }
+
+    @Override
+    public E poll(long timeout, TimeUnit unit) throws InterruptedException {
+        long endAt = System.currentTimeMillis() + unit.toMillis(timeout);
+        synchronized(lock) {
+            long waitTime = endAt - System.currentTimeMillis();
+            while (this.size == 0 && waitTime > 0) {
+                this.lock.wait(waitTime);
+                waitTime = endAt - System.currentTimeMillis();
+            }
+            return poll();
+        }
+    }
+
+    @Override
+    public E poll() {
+        synchronized(lock) {
+            ListIterator<ProducerList<E>> iter = this.producerLists.listIterator(this.currentProducer);
+            while (iter.hasNext()) {
+                ProducerList<E> tList = iter.next();
+                if (tList.list.isEmpty()) {
+                    iter.remove();
+                    this.producerMap.remove(tList.producer);
+                    adjustCurrentProducerPointer();
+                } else {
+                    E element = tList.list.removeFirst();
+                    this.size--;
+                    assert element != null;
+                    // This is the round robin part. When we take an element from the current thread's queue
+                    // we move on to the next thread.
+                    if (tList.list.isEmpty()) {
+                        iter.remove();
+                        this.producerMap.remove(tList.producer);
+                        adjustCurrentProducerPointer();
+                    } else {
+                        incrementCurrentProducerPointer();
+                    }
+                    lock.notifyAll();
+                    return element;
+                }
+            }
+            assert this.size == 0;
+        }
+        return null;
+    }
+
+    /**
+     * Polls using the given producer key.
+     */
+    protected E pollProducer(Object producer) {
+        synchronized(lock) {
+            ProducerList<E> tList = this.producerMap.get(producer);
+            if (tList != null && !tList.list.isEmpty()) {
+                E element = tList.list.removeFirst();
+                this.size--;
+                if (tList.list.isEmpty()) {
+                    this.producerLists.remove(tList);
+                    this.producerMap.remove(tList.producer);
+                    // we need to adjust the current thread pointer in case it pointed to this thread list, which is now removed
+                    adjustCurrentProducerPointer();
+                }
+                lock.notifyAll();
+                assert element != null;
+                // Since this is only processing the current thread's work, we'll leave the
+                // round-robin part alone and just return the work
+                return element;
+            }
+        }
+        return null;
+    }
+
+    @Override
+    public E peek() {
+        synchronized(lock) {
+            ListIterator<ProducerList<E>> iter = this.producerLists.listIterator(this.currentProducer);
+            while (iter.hasNext()) {
+                ProducerList<E> tList = iter.next();
+                if (tList.list.isEmpty()) {
+                    iter.remove();
+                    this.producerMap.remove(tList.producer);
+                    adjustCurrentProducerPointer();
+                } else {
+                    E element = tList.list.getFirst();
+                    assert element != null;
+                    return element;
+                }
+            }
+            assert this.size == 0;
+        }
+        return null;
+    }
+
+    @Override
+    public int drainTo(Collection<? super E> c) {
+        if (c == null)
+            throw new NullPointerException();
+        if (c == this)
+            throw new IllegalArgumentException();
+
+        synchronized(this.lock) {
+            int originalSize = this.size;
+            int drained = drainTo(c, this.size);
+            assert drained == originalSize;
+            assert this.size == 0;
+            assert this.producerLists.isEmpty();
+            assert this.producerMap.isEmpty();
+            return drained;
+        }
+    }
+
+    @Override
+    public int drainTo(Collection<? super E> c, int maxElements) {
+        if (c == null)
+            throw new NullPointerException();
+        if (c == this)
+            throw new IllegalArgumentException();
+
+        synchronized(this.lock) {
+            int i = 0;
+            while(i < maxElements) {
+                E element = poll();
+                if (element != null) {
+                    c.add(element);
+                    i++;
+                } else {
+                    break;
+                }
+            }
+            return i;
+        }
+    }
+
+    @Override
+    public int remainingCapacity() {
+        return Integer.MAX_VALUE;
+    }
+
+    @Override
+    public int size() {
+        synchronized(this.lock) {
+            return this.size;
+        }
+    }
+    
+    private void incrementCurrentProducerPointer() {
+        synchronized(lock) {
+            if (this.producerLists.size() == 0) {
+                this.currentProducer = 0;
+            } else {
+                this.currentProducer = (this.currentProducer+1)%this.producerLists.size();
+            }
+        }
+    }
+    
+    /**
+     * Adjusts the current pointer to a decrease in size.
+     */
+    private void adjustCurrentProducerPointer() {
+        synchronized(lock) {
+            if (this.producerLists.size() == 0) {
+                this.currentProducer = 0;
+            } else {
+                this.currentProducer = (this.currentProducer)%this.producerLists.size();
+            }
+        }
+    }
+
+    private static class ProducerList<E> {
+        public ProducerList(Object producer) {
+            this.producer = producer;
+            this.list = new LinkedList<E>();
+        }
+        private final Object producer;
+        private final LinkedList<E> list;
+    }
+
+    private final Map<Object,ProducerList<E>> producerMap;
+    private final LinkedList<ProducerList<E>> producerLists;
+    private final Object lock;
+    private final boolean newProducerToFront;
+    private int currentProducer;
+    private int size;
+    private int maxSize;
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/job/JobManager.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/job/JobManager.java b/phoenix-core/src/main/java/org/apache/phoenix/job/JobManager.java
new file mode 100644
index 0000000..7d17a6d
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/job/JobManager.java
@@ -0,0 +1,115 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.job;
+
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.AtomicLong;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+/**
+ * 
+ * Thread pool executor that executes scans in parallel
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+@SuppressWarnings("rawtypes")
+public class JobManager<T> extends AbstractRoundRobinQueue<T> {
+	
+    private static final AtomicLong PHOENIX_POOL_INDEX = new AtomicLong(1);
+	
+    public JobManager(int maxSize) {
+        super(maxSize, true); // true -> new producers move to front of queue; this reduces latency.
+    }
+
+	@Override
+    protected Object extractProducer(T o) {
+        return ((JobFutureTask)o).getJobId();
+    }        
+
+    public static interface JobRunnable<T> extends Runnable {
+        public Object getJobId();
+    }
+
+    public static ThreadPoolExecutor createThreadPoolExec(int keepAliveMs, int size, int queueSize) {
+        BlockingQueue<Runnable> queue;
+        if (queueSize == 0) {
+            queue = new SynchronousQueue<Runnable>(); // Specialized for 0 length.
+        } else {
+            queue = new JobManager<Runnable>(queueSize);
+        }
+        ThreadFactory threadFactory = new ThreadFactoryBuilder().setNameFormat(
+				"phoenix-" + PHOENIX_POOL_INDEX.getAndIncrement()
+						+ "-thread-%s").build();
+        // For thread pool, set core threads = max threads -- we don't ever want to exceed core threads, but want to go up to core threads *before* using the queue.
+        ThreadPoolExecutor exec = new ThreadPoolExecutor(size, size, keepAliveMs, TimeUnit.MILLISECONDS, queue, threadFactory) {
+            @Override
+            protected <T> RunnableFuture<T> newTaskFor(Callable<T> call) {
+                // Override this so we can create a JobFutureTask so we can extract out the parentJobId (otherwise, in the default FutureTask, it is private). 
+                return new JobFutureTask<T>(call);
+            }
+    
+            @Override
+            protected <T> RunnableFuture<T> newTaskFor(Runnable runnable, T value) {
+                return new JobFutureTask<T>((JobRunnable)runnable, value);
+            }
+            
+        };
+        
+        exec.allowCoreThreadTimeOut(true); // ... and allow core threads to time out.  This just keeps things clean when idle, and is nice for ftests modes, etc., where we'd especially like these not to linger.
+        return exec;
+    }
+
+    /**
+     * Subclasses FutureTask for the sole purpose of providing {@link #getCallable()}, which is used to extract the producer in the {@link JobBasedRoundRobinQueue}
+     */
+    static class JobFutureTask<T> extends FutureTask<T> {
+        private final Object jobId;
+        
+        public JobFutureTask(JobRunnable r, T t) {
+            super(r, t);
+            this.jobId = r.getJobId();
+        }
+        
+        public JobFutureTask(Callable<T> c) {
+            super(c);
+            // FIXME: this fails when executor used by hbase
+            if (c instanceof JobCallable) {
+                this.jobId = ((JobCallable<T>) c).getJobId();
+            } else {
+                this.jobId = this;
+            }
+        }
+        
+        public Object getJobId() {
+            return jobId;
+        }
+    }
+
+
+    /**
+     * Delegating callable implementation that preserves the parentJobId and sets up thread tracker stuff before delegating to the actual command. 
+     */
+    public static interface JobCallable<T> extends Callable<T> {
+        public Object getJobId();
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/join/HashCacheClient.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/join/HashCacheClient.java b/phoenix-core/src/main/java/org/apache/phoenix/join/HashCacheClient.java
new file mode 100644
index 0000000..c374618
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/join/HashCacheClient.java
@@ -0,0 +1,133 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.join;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.WritableUtils;
+import org.xerial.snappy.Snappy;
+
+import org.apache.phoenix.cache.ServerCacheClient;
+import org.apache.phoenix.cache.ServerCacheClient.ServerCache;
+import org.apache.phoenix.compile.ScanRanges;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.ExpressionType;
+import org.apache.phoenix.iterate.ResultIterator;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.ServerUtil;
+import org.apache.phoenix.util.TrustedByteArrayOutputStream;
+import org.apache.phoenix.util.TupleUtil;
+
+/**
+ * 
+ * Client for adding cache of one side of a join to region servers
+ * 
+ * @author jtaylor
+ * @since 0.1
+ */
+public class HashCacheClient  {
+    private final ServerCacheClient serverCache;
+    /**
+     * Construct client used to create a serialized cached snapshot of a table and send it to each region server
+     * for caching during hash join processing.
+     * @param connection the client connection
+     */
+    public HashCacheClient(PhoenixConnection connection) {
+        serverCache = new ServerCacheClient(connection);
+    }
+
+    /**
+     * Send the results of scanning through the scanner to all
+     * region servers for regions of the table that will use the cache
+     * that intersect with the minMaxKeyRange.
+     * @param scanner scanner for the table or intermediate results being cached
+     * @return client-side {@link ServerCache} representing the added hash cache
+     * @throws SQLException 
+     * @throws MaxServerCacheSizeExceededException if size of hash cache exceeds max allowed
+     * size
+     */
+    public ServerCache addHashCache(ScanRanges keyRanges, ResultIterator iterator, long estimatedSize, List<Expression> onExpressions, TableRef cacheUsingTableRef) throws SQLException {
+        /**
+         * Serialize and compress hashCacheTable
+         */
+        ImmutableBytesWritable ptr = new ImmutableBytesWritable();
+        serialize(ptr, iterator, estimatedSize, onExpressions);
+        return serverCache.addServerCache(keyRanges, ptr, new HashCacheFactory(), cacheUsingTableRef);
+    }
+    
+    private void serialize(ImmutableBytesWritable ptr, ResultIterator iterator, long estimatedSize, List<Expression> onExpressions) throws SQLException {
+        long maxSize = serverCache.getConnection().getQueryServices().getProps().getLong(QueryServices.MAX_SERVER_CACHE_SIZE_ATTRIB, QueryServicesOptions.DEFAULT_MAX_SERVER_CACHE_SIZE);
+        estimatedSize = Math.min(estimatedSize, maxSize);
+        if (estimatedSize > Integer.MAX_VALUE) {
+            throw new IllegalStateException("Estimated size(" + estimatedSize + ") must not be greater than Integer.MAX_VALUE(" + Integer.MAX_VALUE + ")");
+        }
+        try {
+            TrustedByteArrayOutputStream baOut = new TrustedByteArrayOutputStream((int)estimatedSize);
+            DataOutputStream out = new DataOutputStream(baOut);
+            // Write onExpressions first, for hash key evaluation along with deserialization
+            out.writeInt(onExpressions.size());
+            for (Expression expression : onExpressions) {
+                WritableUtils.writeVInt(out, ExpressionType.valueOf(expression).ordinal());
+                expression.write(out);                
+            }
+            int exprSize = baOut.size() + Bytes.SIZEOF_INT;
+            out.writeInt(exprSize);
+            int nRows = 0;
+            out.writeInt(nRows); // In the end will be replaced with total number of rows            
+            for (Tuple result = iterator.next(); result != null; result = iterator.next()) {
+                TupleUtil.write(result, out);
+                if (baOut.size() > maxSize) {
+                    throw new MaxServerCacheSizeExceededException("Size of hash cache (" + baOut.size() + " bytes) exceeds the maximum allowed size (" + maxSize + " bytes)");
+                }
+                nRows++;
+            }
+            TrustedByteArrayOutputStream sizeOut = new TrustedByteArrayOutputStream(Bytes.SIZEOF_INT);
+            DataOutputStream dataOut = new DataOutputStream(sizeOut);
+            try {
+                dataOut.writeInt(nRows);
+                dataOut.flush();
+                byte[] cache = baOut.getBuffer();
+                // Replace number of rows written above with the correct value.
+                System.arraycopy(sizeOut.getBuffer(), 0, cache, exprSize, sizeOut.size());
+                // Reallocate to actual size plus compressed buffer size (which is allocated below)
+                int maxCompressedSize = Snappy.maxCompressedLength(baOut.size());
+                byte[] compressed = new byte[maxCompressedSize]; // size for worst case
+                int compressedSize = Snappy.compress(baOut.getBuffer(), 0, baOut.size(), compressed, 0);
+                // Last realloc to size of compressed buffer.
+                ptr.set(compressed,0,compressedSize);
+            } finally {
+                dataOut.close();
+            }
+        } catch (IOException e) {
+            throw ServerUtil.parseServerException(e);
+        } finally {
+            iterator.close();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..6c3f69b
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/join/HashCacheFactory.java
@@ -0,0 +1,127 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.join;
+
+import java.io.*;
+import java.sql.SQLException;
+import java.util.*;
+
+import net.jcip.annotations.Immutable;
+
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.WritableUtils;
+import org.xerial.snappy.Snappy;
+
+import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.cache.HashCache;
+import org.apache.phoenix.coprocessor.ServerCachingProtocol.ServerCacheFactory;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.ExpressionType;
+import org.apache.phoenix.memory.MemoryManager.MemoryChunk;
+import org.apache.phoenix.schema.tuple.ResultTuple;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.*;
+
+public class HashCacheFactory implements ServerCacheFactory {
+
+    public HashCacheFactory() {
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+    }
+
+    @Override
+    public Closeable newCache(ImmutableBytesWritable cachePtr, MemoryChunk chunk) throws SQLException {
+        try {
+            int size = Snappy.uncompressedLength(cachePtr.get());
+            byte[] uncompressed = new byte[size];
+            Snappy.uncompress(cachePtr.get(), 0, cachePtr.getLength(), uncompressed, 0);
+            return new HashCacheImpl(uncompressed, chunk);
+        } catch (IOException e) {
+            throw ServerUtil.parseServerException(e);
+        }
+    }
+
+    @Immutable
+    private class HashCacheImpl implements HashCache {
+        private final Map<ImmutableBytesPtr,List<Tuple>> hashCache;
+        private final MemoryChunk memoryChunk;
+        
+        private HashCacheImpl(byte[] hashCacheBytes, MemoryChunk memoryChunk) {
+            try {
+                this.memoryChunk = memoryChunk;
+                byte[] hashCacheByteArray = hashCacheBytes;
+                int offset = 0;
+                ByteArrayInputStream input = new ByteArrayInputStream(hashCacheByteArray, offset, hashCacheBytes.length);
+                DataInputStream dataInput = new DataInputStream(input);
+                int nExprs = dataInput.readInt();
+                List<Expression> onExpressions = new ArrayList<Expression>(nExprs);
+                for (int i = 0; i < nExprs; i++) {
+                    int expressionOrdinal = WritableUtils.readVInt(dataInput);
+                    Expression expression = ExpressionType.values()[expressionOrdinal].newInstance();
+                    expression.readFields(dataInput);
+                    onExpressions.add(expression);                        
+                }
+                int exprSize = dataInput.readInt();
+                offset += exprSize;
+                int nRows = dataInput.readInt();
+                int estimatedSize = SizedUtil.sizeOfMap(nRows, SizedUtil.IMMUTABLE_BYTES_WRITABLE_SIZE, SizedUtil.RESULT_SIZE) + hashCacheBytes.length;
+                this.memoryChunk.resize(estimatedSize);
+                HashMap<ImmutableBytesPtr,List<Tuple>> hashCacheMap = new HashMap<ImmutableBytesPtr,List<Tuple>>(nRows * 5 / 4);
+                offset += Bytes.SIZEOF_INT;
+                // Build Map with evaluated hash key as key and row as value
+                for (int i = 0; i < nRows; i++) {
+                    int resultSize = (int)Bytes.readVLong(hashCacheByteArray, offset);
+                    offset += WritableUtils.decodeVIntSize(hashCacheByteArray[offset]);
+                    ImmutableBytesWritable value = new ImmutableBytesWritable(hashCacheByteArray,offset,resultSize);
+                    Tuple result = new ResultTuple(new Result(value));
+                    ImmutableBytesPtr key = TupleUtil.getConcatenatedValue(result, onExpressions);
+                    List<Tuple> tuples = hashCacheMap.get(key);
+                    if (tuples == null) {
+                        tuples = new ArrayList<Tuple>(1);
+                        hashCacheMap.put(key, tuples);
+                    }
+                    tuples.add(result);
+                    offset += resultSize;
+                }
+                this.hashCache = Collections.unmodifiableMap(hashCacheMap);
+            } catch (IOException e) { // Not possible with ByteArrayInputStream
+                throw new RuntimeException(e);
+            }
+        }
+
+        @Override
+        public void close() {
+            memoryChunk.close();
+        }
+        
+        @Override
+        public List<Tuple> get(ImmutableBytesPtr hashKey) {
+            return hashCache.get(hashKey);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/join/HashJoinInfo.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/join/HashJoinInfo.java b/phoenix-core/src/main/java/org/apache/phoenix/join/HashJoinInfo.java
new file mode 100644
index 0000000..efeb717
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/join/HashJoinInfo.java
@@ -0,0 +1,210 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.join;
+
+import java.io.*;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.io.WritableUtils;
+
+import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.ExpressionType;
+import org.apache.phoenix.parse.JoinTableNode.JoinType;
+import org.apache.phoenix.schema.KeyValueSchema;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.KeyValueSchema.KeyValueSchemaBuilder;
+import org.apache.phoenix.util.SchemaUtil;
+
+public class HashJoinInfo {
+    private static final String HASH_JOIN = "HashJoin";
+    
+    private KeyValueSchema joinedSchema;
+    private ImmutableBytesPtr[] joinIds;
+    private List<Expression>[] joinExpressions;
+    private JoinType[] joinTypes;
+    private boolean[] earlyEvaluation;
+    private KeyValueSchema[] schemas;
+    private int[] fieldPositions;
+    private Expression postJoinFilterExpression;
+    
+    public HashJoinInfo(PTable joinedTable, ImmutableBytesPtr[] joinIds, List<Expression>[] joinExpressions, JoinType[] joinTypes, boolean[] earlyEvaluation, PTable[] tables, int[] fieldPositions, Expression postJoinFilterExpression) {
+    	this(buildSchema(joinedTable), joinIds, joinExpressions, joinTypes, earlyEvaluation, buildSchemas(tables), fieldPositions, postJoinFilterExpression);
+    }
+    
+    private static KeyValueSchema[] buildSchemas(PTable[] tables) {
+    	KeyValueSchema[] schemas = new KeyValueSchema[tables.length];
+    	for (int i = 0; i < tables.length; i++) {
+    		schemas[i] = buildSchema(tables[i]);
+    	}
+    	return schemas;
+    }
+    
+    private static KeyValueSchema buildSchema(PTable table) {
+    	KeyValueSchemaBuilder builder = new KeyValueSchemaBuilder(0);
+    	if (table != null) {
+    	    for (PColumn column : table.getColumns()) {
+    	        if (!SchemaUtil.isPKColumn(column)) {
+    	            builder.addField(column);
+    	        }
+    	    }
+    	}
+        return builder.build();
+    }
+    
+    private HashJoinInfo(KeyValueSchema joinedSchema, ImmutableBytesPtr[] joinIds, List<Expression>[] joinExpressions, JoinType[] joinTypes, boolean[] earlyEvaluation, KeyValueSchema[] schemas, int[] fieldPositions, Expression postJoinFilterExpression) {
+    	this.joinedSchema = joinedSchema;
+    	this.joinIds = joinIds;
+        this.joinExpressions = joinExpressions;
+        this.joinTypes = joinTypes;
+        this.earlyEvaluation = earlyEvaluation;
+        this.schemas = schemas;
+        this.fieldPositions = fieldPositions;
+        this.postJoinFilterExpression = postJoinFilterExpression;
+    }
+    
+    public KeyValueSchema getJoinedSchema() {
+    	return joinedSchema;
+    }
+    
+    public ImmutableBytesPtr[] getJoinIds() {
+        return joinIds;
+    }
+    
+    public List<Expression>[] getJoinExpressions() {
+        return joinExpressions;
+    }
+    
+    public JoinType[] getJoinTypes() {
+        return joinTypes;
+    }
+    
+    public boolean[] earlyEvaluation() {
+    	return earlyEvaluation;
+    }
+    
+    public KeyValueSchema[] getSchemas() {
+    	return schemas;
+    }
+    
+    public int[] getFieldPositions() {
+    	return fieldPositions;
+    }
+    
+    public Expression getPostJoinFilterExpression() {
+        return postJoinFilterExpression;
+    }
+    
+    public static void serializeHashJoinIntoScan(Scan scan, HashJoinInfo joinInfo) {
+        ByteArrayOutputStream stream = new ByteArrayOutputStream();
+        try {
+            DataOutputStream output = new DataOutputStream(stream);
+            joinInfo.joinedSchema.write(output);
+            int count = joinInfo.joinIds.length;
+            WritableUtils.writeVInt(output, count);
+            for (int i = 0; i < count; i++) {
+                joinInfo.joinIds[i].write(output);
+                WritableUtils.writeVInt(output, joinInfo.joinExpressions[i].size());
+                for (Expression expr : joinInfo.joinExpressions[i]) {
+                    WritableUtils.writeVInt(output, ExpressionType.valueOf(expr).ordinal());
+                    expr.write(output);
+                }
+                WritableUtils.writeVInt(output, joinInfo.joinTypes[i].ordinal());
+                output.writeBoolean(joinInfo.earlyEvaluation[i]);
+                joinInfo.schemas[i].write(output);
+                WritableUtils.writeVInt(output, joinInfo.fieldPositions[i]);
+            }
+            if (joinInfo.postJoinFilterExpression != null) {
+                WritableUtils.writeVInt(output, ExpressionType.valueOf(joinInfo.postJoinFilterExpression).ordinal());
+                joinInfo.postJoinFilterExpression.write(output);
+            } else {
+                WritableUtils.writeVInt(output, -1);
+            }
+            scan.setAttribute(HASH_JOIN, stream.toByteArray());
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        } finally {
+            try {
+                stream.close();
+            } catch (IOException e) {
+                throw new RuntimeException(e);
+            }
+        }
+        
+    }
+    
+    @SuppressWarnings("unchecked")
+    public static HashJoinInfo deserializeHashJoinFromScan(Scan scan) {
+        byte[] join = scan.getAttribute(HASH_JOIN);
+        if (join == null) {
+            return null;
+        }
+        ByteArrayInputStream stream = new ByteArrayInputStream(join);
+        try {
+            DataInputStream input = new DataInputStream(stream);
+            KeyValueSchema joinedSchema = new KeyValueSchema();
+            joinedSchema.readFields(input);
+            int count = WritableUtils.readVInt(input);
+            ImmutableBytesPtr[] joinIds = new ImmutableBytesPtr[count];
+            List<Expression>[] joinExpressions = new List[count];
+            JoinType[] joinTypes = new JoinType[count];
+            boolean[] earlyEvaluation = new boolean[count];
+            KeyValueSchema[] schemas = new KeyValueSchema[count];
+            int[] fieldPositions = new int[count];
+            for (int i = 0; i < count; i++) {
+                joinIds[i] = new ImmutableBytesPtr();
+                joinIds[i].readFields(input);
+                int nExprs = WritableUtils.readVInt(input);
+                joinExpressions[i] = new ArrayList<Expression>(nExprs);
+                for (int j = 0; j < nExprs; j++) {
+                    int expressionOrdinal = WritableUtils.readVInt(input);
+                    Expression expression = ExpressionType.values()[expressionOrdinal].newInstance();
+                    expression.readFields(input);
+                    joinExpressions[i].add(expression);                    
+                }
+                int type = WritableUtils.readVInt(input);
+                joinTypes[i] = JoinType.values()[type];
+                earlyEvaluation[i] = input.readBoolean();
+                schemas[i] = new KeyValueSchema();
+                schemas[i].readFields(input);
+                fieldPositions[i] = WritableUtils.readVInt(input);
+            }
+            Expression postJoinFilterExpression = null;
+            int expressionOrdinal = WritableUtils.readVInt(input);
+            if (expressionOrdinal != -1) {
+                postJoinFilterExpression = ExpressionType.values()[expressionOrdinal].newInstance();
+                postJoinFilterExpression.readFields(input);
+            }
+            return new HashJoinInfo(joinedSchema, joinIds, joinExpressions, joinTypes, earlyEvaluation, schemas, fieldPositions, postJoinFilterExpression);
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        } finally {
+            try {
+                stream.close();
+            } catch (IOException e) {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+
+}


[46/51] [partial] Initial commit of master branch from github

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/scanner/FilteredKeyValueScanner.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/scanner/FilteredKeyValueScanner.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/scanner/FilteredKeyValueScanner.java
new file mode 100644
index 0000000..0f7fed3
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/scanner/FilteredKeyValueScanner.java
@@ -0,0 +1,144 @@
+package org.apache.hadoop.hbase.index.scanner;
+
+import java.io.IOException;
+import java.util.SortedSet;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.Filter.ReturnCode;
+import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
+
+import org.apache.hadoop.hbase.index.covered.KeyValueStore;
+
+/**
+ * Combine a simplified version of the logic in the ScanQueryMatcher and the KeyValueScanner. We can get away with this
+ * here because we are only concerned with a single MemStore for the index; we don't need to worry about multiple column
+ * families or minimizing seeking through file - we just want to iterate the kvs quickly, in-memory.
+ */
+public class FilteredKeyValueScanner implements KeyValueScanner {
+
+    private KeyValueScanner delegate;
+    private Filter filter;
+
+    public FilteredKeyValueScanner(Filter filter, KeyValueStore store) {
+        this(filter, store.getScanner());
+    }
+
+    private FilteredKeyValueScanner(Filter filter, KeyValueScanner delegate) {
+        this.delegate = delegate;
+        this.filter = filter;
+    }
+
+    @Override
+    public KeyValue peek() {
+        return delegate.peek();
+    }
+
+    /**
+     * Same a {@link KeyValueScanner#next()} except that we filter out the next {@link KeyValue} until we find one that
+     * passes the filter.
+     * 
+     * @return the next {@link KeyValue} or <tt>null</tt> if no next {@link KeyValue} is present and passes all the
+     *         filters.
+     */
+    @Override
+    public KeyValue next() throws IOException {
+        seekToNextUnfilteredKeyValue();
+        return delegate.next();
+    }
+
+    @Override
+    public boolean seek(KeyValue key) throws IOException {
+        if (filter.filterAllRemaining()) { return false; }
+        // see if we can seek to the next key
+        if (!delegate.seek(key)) { return false; }
+
+        return seekToNextUnfilteredKeyValue();
+    }
+
+    private boolean seekToNextUnfilteredKeyValue() throws IOException {
+        while (true) {
+            KeyValue peeked = delegate.peek();
+            // no more key values, so we are done
+            if (peeked == null) { return false; }
+
+            // filter the peeked value to see if it should be served
+            ReturnCode code = filter.filterKeyValue(peeked);
+            switch (code) {
+            // included, so we are done
+            case INCLUDE:
+            case INCLUDE_AND_NEXT_COL:
+                return true;
+                // not included, so we need to go to the next row
+            case SKIP:
+            case NEXT_COL:
+            case NEXT_ROW:
+                delegate.next();
+                break;
+            // use a seek hint to find out where we should go
+            case SEEK_NEXT_USING_HINT:
+                delegate.seek(filter.getNextKeyHint(peeked));
+            }
+        }
+    }
+
+    @Override
+    public boolean reseek(KeyValue key) throws IOException {
+        this.delegate.reseek(key);
+        return this.seekToNextUnfilteredKeyValue();
+    }
+
+    @Override
+    public boolean requestSeek(KeyValue kv, boolean forward, boolean useBloom) throws IOException {
+        return this.reseek(kv);
+    }
+
+    @Override
+    public boolean isFileScanner() {
+        return false;
+    }
+
+    @Override
+    public long getSequenceID() {
+        return this.delegate.getSequenceID();
+    }
+
+    @Override
+    public boolean shouldUseScanner(Scan scan, SortedSet<byte[]> columns, long oldestUnexpiredTS) {
+        throw new UnsupportedOperationException(this.getClass().getName()
+                + " doesn't support checking to see if it should use a scanner!");
+    }
+
+    @Override
+    public boolean realSeekDone() {
+        return this.delegate.realSeekDone();
+    }
+
+    @Override
+    public void enforceSeek() throws IOException {
+        this.delegate.enforceSeek();
+    }
+
+    @Override
+    public void close() {
+        this.delegate.close();
+    }
+
+    /*
+    @Override
+    public boolean backwardSeek(KeyValue arg0) throws IOException {
+        return this.delegate.backwardSeek(arg0);
+    }
+
+    @Override
+    public boolean seekToLastRow() throws IOException {
+        return this.delegate.seekToLastRow();
+    }
+
+    @Override
+    public boolean seekToPreviousRow(KeyValue arg0) throws IOException {
+        return this.delegate.seekToPreviousRow(arg0);
+    }
+    */
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/scanner/Scanner.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/scanner/Scanner.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/scanner/Scanner.java
new file mode 100644
index 0000000..9df96cd
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/scanner/Scanner.java
@@ -0,0 +1,37 @@
+package org.apache.hadoop.hbase.index.scanner;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.KeyValue;
+
+/**
+ * Scan the primary table. This is similar to HBase's scanner, but ensures that you will never see
+ * deleted columns/rows
+ */
+public interface Scanner extends Closeable {
+
+  /**
+   * @return the next keyvalue in the scanner or <tt>null</tt> if there is no next {@link KeyValue}
+   * @throws IOException if there is an underlying error reading the data
+   */
+  public KeyValue next() throws IOException;
+
+  /**
+   * Seek to immediately before the given {@link KeyValue}. If that exact {@link KeyValue} is
+   * present in <tt>this</tt>, it will be returned by the next call to {@link #next()}. Otherwise,
+   * returns the next {@link KeyValue} after the seeked {@link KeyValue}.
+   * @param next {@link KeyValue} to seek to. Doesn't need to already be present in <tt>this</tt>
+   * @return <tt>true</tt> if there are values left in <tt>this</tt>, <tt>false</tt> otherwise
+   * @throws IOException if there is an error reading the underlying data.
+   */
+  public boolean seek(KeyValue next) throws IOException;
+
+  /**
+   * Read the {@link KeyValue} at the top of <tt>this</tt> without 'popping' it off the top of the
+   * scanner.
+   * @return the next {@link KeyValue} or <tt>null</tt> if there are no more values in <tt>this</tt>
+   * @throws IOException if there is an error reading the underlying data.
+   */
+  public KeyValue peek() throws IOException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/scanner/ScannerBuilder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/scanner/ScannerBuilder.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/scanner/ScannerBuilder.java
new file mode 100644
index 0000000..bbecb9a
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/scanner/ScannerBuilder.java
@@ -0,0 +1,147 @@
+package org.apache.hadoop.hbase.index.scanner;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.filter.BinaryComparator;
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+import org.apache.hadoop.hbase.filter.FamilyFilter;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.filter.QualifierFilter;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import com.google.common.collect.Lists;
+import org.apache.hadoop.hbase.index.covered.KeyValueStore;
+import org.apache.hadoop.hbase.index.covered.filter.ApplyAndFilterDeletesFilter;
+import org.apache.hadoop.hbase.index.covered.filter.ColumnTrackingNextLargestTimestampFilter;
+import org.apache.hadoop.hbase.index.covered.update.ColumnReference;
+import org.apache.hadoop.hbase.index.covered.update.ColumnTracker;
+import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+
+/**
+ *
+ */
+public class ScannerBuilder {
+
+  private KeyValueStore memstore;
+  private Mutation update;
+
+
+  public ScannerBuilder(KeyValueStore memstore, Mutation update) {
+    this.memstore = memstore;
+    this.update = update;
+  }
+
+  public Scanner buildIndexedColumnScanner(Collection<? extends ColumnReference> indexedColumns, ColumnTracker tracker, long ts) {
+
+    Filter columnFilters = getColumnFilters(indexedColumns);
+    FilterList filters = new FilterList(Lists.newArrayList(columnFilters));
+
+    // skip to the right TS. This needs to come before the deletes since the deletes will hide any
+    // state that comes before the actual kvs, so we need to capture those TS as they change the row
+    // state.
+    filters.addFilter(new ColumnTrackingNextLargestTimestampFilter(ts, tracker));
+
+    // filter out kvs based on deletes
+    filters.addFilter(new ApplyAndFilterDeletesFilter(getAllFamilies(indexedColumns)));
+
+    // combine the family filters and the rest of the filters as a
+    return getFilteredScanner(filters);
+  }
+
+  /**
+   * @param columns columns to filter
+   * @return filter that will skip any {@link KeyValue} that doesn't match one of the passed columns
+   *         and the
+   */
+  private Filter
+      getColumnFilters(Collection<? extends ColumnReference> columns) {
+    // each column needs to be added as an OR, so we need to separate them out
+    FilterList columnFilters = new FilterList(FilterList.Operator.MUST_PASS_ONE);
+
+    // create a filter that matches each column reference
+    for (ColumnReference ref : columns) {
+      Filter columnFilter =
+          new FamilyFilter(CompareOp.EQUAL, new BinaryComparator(ref.getFamily()));
+      // combine with a match for the qualifier, if the qualifier is a specific qualifier
+      if (!Bytes.equals(ColumnReference.ALL_QUALIFIERS, ref.getQualifier())) {
+        columnFilter =
+            new FilterList(columnFilter, new QualifierFilter(CompareOp.EQUAL, new BinaryComparator(
+                ref.getQualifier())));
+      }
+      columnFilters.addFilter(columnFilter);
+    }
+    return columnFilters;
+  }
+
+  private Set<ImmutableBytesPtr>
+      getAllFamilies(Collection<? extends ColumnReference> columns) {
+    Set<ImmutableBytesPtr> families = new HashSet<ImmutableBytesPtr>();
+    for (ColumnReference ref : columns) {
+      families.add(new ImmutableBytesPtr(ref.getFamily()));
+    }
+    return families;
+  }
+
+  private Scanner getFilteredScanner(Filter filters) {
+    // create a scanner and wrap it as an iterator, meaning you can only go forward
+    final FilteredKeyValueScanner kvScanner = new FilteredKeyValueScanner(filters, memstore);
+    // seek the scanner to initialize it
+    KeyValue start = KeyValue.createFirstOnRow(update.getRow());
+    try {
+      if (!kvScanner.seek(start)) {
+        return new EmptyScanner();
+      }
+    } catch (IOException e) {
+      // This should never happen - everything should explode if so.
+      throw new RuntimeException(
+          "Failed to seek to first key from update on the memstore scanner!", e);
+    }
+
+    // we have some info in the scanner, so wrap it in an iterator and return.
+    return new Scanner() {
+
+      @Override
+      public KeyValue next() {
+        try {
+          return kvScanner.next();
+        } catch (IOException e) {
+          throw new RuntimeException("Error reading kvs from local memstore!");
+        }
+      }
+
+      @Override
+      public boolean seek(KeyValue next) throws IOException {
+        // check to see if the next kv is after the current key, in which case we can use reseek,
+        // which will be more efficient
+        KeyValue peek = kvScanner.peek();
+        // there is another value and its before the requested one - we can do a reseek!
+        if (peek != null) {
+          int compare = KeyValue.COMPARATOR.compare(peek, next);
+          if (compare < 0) {
+            return kvScanner.reseek(next);
+          } else if (compare == 0) {
+            // we are already at the given key!
+            return true;
+          }
+        }
+        return kvScanner.seek(next);
+      }
+
+      @Override
+      public KeyValue peek() throws IOException {
+        return kvScanner.peek();
+      }
+
+      @Override
+      public void close() {
+        kvScanner.close();
+      }
+    };
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/table/CachingHTableFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/table/CachingHTableFactory.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/table/CachingHTableFactory.java
new file mode 100644
index 0000000..d95e67a
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/table/CachingHTableFactory.java
@@ -0,0 +1,112 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index.table;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.commons.collections.map.LRUMap;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+
+/**
+ * A simple cache that just uses usual GC mechanisms to cleanup unused {@link HTableInterface}s.
+ * When requesting an {@link HTableInterface} via {@link #getTable}, you may get the same table as
+ * last time, or it may be a new table.
+ * <p>
+ * You <b>should not call {@link HTableInterface#close()} </b> that is handled when the table goes
+ * out of scope. Along the same lines, you must ensure to not keep a reference to the table for
+ * longer than necessary - this leak will ensure that the table never gets closed.
+ */
+public class CachingHTableFactory implements HTableFactory {
+
+  /**
+   * LRUMap that closes the {@link HTableInterface} when the table is evicted
+   */
+  @SuppressWarnings("serial")
+  public class HTableInterfaceLRUMap extends LRUMap {
+
+    public HTableInterfaceLRUMap(int cacheSize) {
+      super(cacheSize);
+    }
+
+    @Override
+    protected boolean removeLRU(LinkEntry entry) {
+      HTableInterface table = (HTableInterface) entry.getValue();
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Closing connection to table: " + Bytes.toString(table.getTableName())
+            + " because it was evicted from the cache.");
+      }
+      try {
+        table.close();
+      } catch (IOException e) {
+        LOG.info("Failed to correctly close HTable: " + Bytes.toString(table.getTableName())
+            + " ignoring since being removed from queue.");
+      }
+      return true;
+    }
+  }
+
+  public static int getCacheSize(Configuration conf) {
+    return conf.getInt(CACHE_SIZE_KEY, DEFAULT_CACHE_SIZE);
+  }
+
+  private static final Log LOG = LogFactory.getLog(CachingHTableFactory.class);
+  private static final String CACHE_SIZE_KEY = "index.tablefactory.cache.size";
+  private static final int DEFAULT_CACHE_SIZE = 10;
+
+  private HTableFactory delegate;
+
+  @SuppressWarnings("rawtypes")
+  Map openTables;
+
+  public CachingHTableFactory(HTableFactory tableFactory, Configuration conf) {
+    this(tableFactory, getCacheSize(conf));
+  }
+
+  public CachingHTableFactory(HTableFactory factory, int cacheSize) {
+    this.delegate = factory;
+    openTables = new HTableInterfaceLRUMap(cacheSize);
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public HTableInterface getTable(ImmutableBytesPtr tablename) throws IOException {
+    ImmutableBytesPtr tableBytes = new ImmutableBytesPtr(tablename);
+    synchronized (openTables) {
+      HTableInterface table = (HTableInterface) openTables.get(tableBytes);
+      if (table == null) {
+        table = delegate.getTable(tablename);
+        openTables.put(tableBytes, table);
+      }
+      return table;
+    }
+  }
+
+  @Override
+  public void shutdown() {
+    this.delegate.shutdown();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/table/CoprocessorHTableFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/table/CoprocessorHTableFactory.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/table/CoprocessorHTableFactory.java
new file mode 100644
index 0000000..5ded879
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/table/CoprocessorHTableFactory.java
@@ -0,0 +1,50 @@
+package org.apache.hadoop.hbase.index.table;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CoprocessorEnvironment;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+import org.apache.hadoop.hbase.index.util.IndexManagementUtil;
+
+public class CoprocessorHTableFactory implements HTableFactory {
+
+  /** Number of milliseconds per-interval to retry zookeeper */
+  private static final String ZOOKEEPER_RECOVERY_RETRY_INTERVALMILL = "zookeeper.recovery.retry.intervalmill";
+  /** Number of retries for zookeeper */
+  private static final String ZOOKEEPER_RECOVERY_RETRY_KEY = "zookeeper.recovery.retry";
+  private static final Log LOG = LogFactory.getLog(CoprocessorHTableFactory.class);
+  private CoprocessorEnvironment e;
+
+  public CoprocessorHTableFactory(CoprocessorEnvironment e) {
+    this.e = e;
+  }
+
+  @Override
+  public HTableInterface getTable(ImmutableBytesPtr tablename) throws IOException {
+    Configuration conf = e.getConfiguration();
+    // make sure writers fail fast
+    IndexManagementUtil.setIfNotSet(conf, HConstants.HBASE_CLIENT_RETRIES_NUMBER, 3);
+    IndexManagementUtil.setIfNotSet(conf, HConstants.HBASE_CLIENT_PAUSE, 1000);
+    IndexManagementUtil.setIfNotSet(conf, ZOOKEEPER_RECOVERY_RETRY_KEY, 3);
+    IndexManagementUtil.setIfNotSet(conf, ZOOKEEPER_RECOVERY_RETRY_INTERVALMILL, 100);
+    IndexManagementUtil.setIfNotSet(conf, HConstants.ZK_SESSION_TIMEOUT, 30000);
+    IndexManagementUtil.setIfNotSet(conf, HConstants.HBASE_RPC_TIMEOUT_KEY, 5000);
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Creating new HTable: " + Bytes.toString(tablename.copyBytesIfNecessary()));
+    }
+    return this.e.getTable(tablename.copyBytesIfNecessary());
+  }
+
+  @Override
+  public void shutdown() {
+    // noop
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/table/HTableFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/table/HTableFactory.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/table/HTableFactory.java
new file mode 100644
index 0000000..a27a238
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/table/HTableFactory.java
@@ -0,0 +1,14 @@
+package org.apache.hadoop.hbase.index.table;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.client.HTableInterface;
+
+import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+
+public interface HTableFactory {
+
+  public HTableInterface getTable(ImmutableBytesPtr tablename) throws IOException;
+
+  public void shutdown();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/table/HTableInterfaceReference.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/table/HTableInterfaceReference.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/table/HTableInterfaceReference.java
new file mode 100644
index 0000000..134006f
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/table/HTableInterfaceReference.java
@@ -0,0 +1,46 @@
+package org.apache.hadoop.hbase.index.table;
+
+import org.apache.hadoop.hbase.util.Bytes;
+
+import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+
+/**
+ * Reference to an HTableInterface. Currently, its pretty simple in that it is just a wrapper around
+ * the table name.
+ */
+public class HTableInterfaceReference {
+
+  private ImmutableBytesPtr tableName;
+
+
+  public HTableInterfaceReference(ImmutableBytesPtr tableName) {
+    this.tableName = tableName;
+  }
+
+  public ImmutableBytesPtr get() {
+    return this.tableName;
+  }
+
+  public String getTableName() {
+    return Bytes.toString(this.tableName.get(),this.tableName.getOffset(), this.tableName.getLength());
+  }
+
+  @Override
+  public int hashCode() {
+      return tableName.hashCode();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+      if (this == obj) return true;
+      if (obj == null) return false;
+      if (getClass() != obj.getClass()) return false;
+      HTableInterfaceReference other = (HTableInterfaceReference)obj;
+      return tableName.equals(other.tableName);
+  }
+
+  @Override
+  public String toString() {
+    return Bytes.toString(this.tableName.get());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/util/ImmutableBytesPtr.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/util/ImmutableBytesPtr.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/util/ImmutableBytesPtr.java
new file mode 100644
index 0000000..4ae6afb
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/util/ImmutableBytesPtr.java
@@ -0,0 +1,113 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index.util;
+
+import java.io.DataInput;
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+
+public class ImmutableBytesPtr extends ImmutableBytesWritable {
+    private int hashCode;
+    
+    public ImmutableBytesPtr() {
+    }
+
+    public ImmutableBytesPtr(byte[] bytes) {
+        super(bytes);
+        hashCode = super.hashCode();
+    }
+
+    public ImmutableBytesPtr(ImmutableBytesWritable ibw) {
+        super(ibw.get(), ibw.getOffset(), ibw.getLength());
+        hashCode = super.hashCode();
+    }
+
+    public ImmutableBytesPtr(ImmutableBytesPtr ibp) {
+        super(ibp.get(), ibp.getOffset(), ibp.getLength());
+        hashCode = ibp.hashCode;
+    }
+
+    public ImmutableBytesPtr(byte[] bytes, int offset, int length) {
+        super(bytes, offset, length);
+        hashCode = super.hashCode();
+    }
+
+    @Override
+    public int hashCode() {
+        return hashCode;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) return true;
+        if (obj == null) return false;
+        if (getClass() != obj.getClass()) return false;
+        ImmutableBytesPtr that = (ImmutableBytesPtr)obj;
+        if (this.hashCode != that.hashCode) return false;
+        if (Bytes.compareTo(this.get(), this.getOffset(), this.getLength(), that.get(), that.getOffset(), that.getLength()) != 0) return false;
+        return true;
+    }
+
+    public void set(ImmutableBytesWritable ptr) {
+        set(ptr.get(),ptr.getOffset(),ptr.getLength());
+      }
+
+    /**
+     * @param b Use passed bytes as backing array for this instance.
+     */
+    @Override
+    public void set(final byte [] b) {
+      super.set(b);
+      hashCode = super.hashCode();
+    }
+
+    /**
+     * @param b Use passed bytes as backing array for this instance.
+     * @param offset
+     * @param length
+     */
+    @Override
+    public void set(final byte [] b, final int offset, final int length) {
+        super.set(b,offset,length);
+        hashCode = super.hashCode();
+    }
+
+    @Override
+    public void readFields(final DataInput in) throws IOException {
+        super.readFields(in);
+        hashCode = super.hashCode();
+    }
+    
+    /**
+     * @return the backing byte array, copying only if necessary
+     */
+    public byte[] copyBytesIfNecessary() {
+    return copyBytesIfNecessary(this);
+    }
+
+  public static byte[] copyBytesIfNecessary(ImmutableBytesWritable ptr) {
+    if (ptr.getOffset() == 0 && ptr.getLength() == ptr.get().length) {
+      return ptr.get();
+    }
+    return ptr.copyBytes();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/util/IndexManagementUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/util/IndexManagementUtil.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/util/IndexManagementUtil.java
new file mode 100644
index 0000000..59e07cc
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/util/IndexManagementUtil.java
@@ -0,0 +1,246 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index.util;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Scan;
+
+import com.google.common.collect.Maps;
+import org.apache.hadoop.hbase.index.ValueGetter;
+import org.apache.hadoop.hbase.index.builder.IndexBuildingFailureException;
+import org.apache.hadoop.hbase.index.covered.data.LazyValueGetter;
+import org.apache.hadoop.hbase.index.covered.update.ColumnReference;
+import org.apache.hadoop.hbase.index.scanner.Scanner;
+
+/**
+ * Utility class to help manage indexes
+ */
+public class IndexManagementUtil {
+
+    private IndexManagementUtil() {
+        // private ctor for util classes
+    }
+
+    // Don't rely on statically defined classes constants from classes that may not exist
+    // in earlier HBase versions
+    public static final String INDEX_WAL_EDIT_CODEC_CLASS_NAME = "org.apache.hadoop.hbase.regionserver.wal.IndexedWALEditCodec";
+    public static final String HLOG_READER_IMPL_KEY = "hbase.regionserver.hlog.reader.impl";
+    public static final String WAL_EDIT_CODEC_CLASS_KEY = "hbase.regionserver.wal.codec";
+
+    private static final String INDEX_HLOG_READER_CLASS_NAME = "org.apache.hadoop.hbase.regionserver.wal.IndexedHLogReader";
+    private static final Log LOG = LogFactory.getLog(IndexManagementUtil.class);
+
+    public static boolean isWALEditCodecSet(Configuration conf) {
+        // check to see if the WALEditCodec is installed
+        try {
+            // Use reflection to load the IndexedWALEditCodec, since it may not load with an older version
+            // of HBase
+            Class.forName(INDEX_WAL_EDIT_CODEC_CLASS_NAME);
+        } catch (Throwable t) {
+            return false;
+        }
+        if (INDEX_WAL_EDIT_CODEC_CLASS_NAME.equals(conf.get(WAL_EDIT_CODEC_CLASS_KEY, null))) {
+            // its installed, and it can handle compression and non-compression cases
+            return true;
+        }
+        return false;
+    }
+
+    public static void ensureMutableIndexingCorrectlyConfigured(Configuration conf) throws IllegalStateException {
+
+        // check to see if the WALEditCodec is installed
+        if (isWALEditCodecSet(conf)) { return; }
+
+        // otherwise, we have to install the indexedhlogreader, but it cannot have compression
+        String codecClass = INDEX_WAL_EDIT_CODEC_CLASS_NAME;
+        String indexLogReaderName = INDEX_HLOG_READER_CLASS_NAME;
+        try {
+            // Use reflection to load the IndexedHLogReader, since it may not load with an older version
+            // of HBase
+            Class.forName(indexLogReaderName);
+        } catch (ClassNotFoundException e) {
+            throw new IllegalStateException(codecClass + " is not installed, but "
+                    + indexLogReaderName + " hasn't been installed in hbase-site.xml under " + HLOG_READER_IMPL_KEY);
+        }
+        if (indexLogReaderName.equals(conf.get(HLOG_READER_IMPL_KEY, indexLogReaderName))) {
+            if (conf.getBoolean(HConstants.ENABLE_WAL_COMPRESSION, false)) { throw new IllegalStateException(
+                    "WAL Compression is only supported with " + codecClass
+                            + ". You can install in hbase-site.xml, under " + WAL_EDIT_CODEC_CLASS_KEY); }
+        } else {
+            throw new IllegalStateException(codecClass + " is not installed, but "
+                    + indexLogReaderName + " hasn't been installed in hbase-site.xml under " + HLOG_READER_IMPL_KEY);
+        }
+
+    }
+
+    public static ValueGetter createGetterFromKeyValues(Collection<KeyValue> pendingUpdates) {
+        final Map<ReferencingColumn, ImmutableBytesPtr> valueMap = Maps.newHashMapWithExpectedSize(pendingUpdates
+                .size());
+        for (KeyValue kv : pendingUpdates) {
+            // create new pointers to each part of the kv
+            ImmutableBytesPtr family = new ImmutableBytesPtr(kv.getBuffer(), kv.getFamilyOffset(), kv.getFamilyLength());
+            ImmutableBytesPtr qual = new ImmutableBytesPtr(kv.getBuffer(), kv.getQualifierOffset(),
+                    kv.getQualifierLength());
+            ImmutableBytesPtr value = new ImmutableBytesPtr(kv.getBuffer(), kv.getValueOffset(), kv.getValueLength());
+            valueMap.put(new ReferencingColumn(family, qual), value);
+        }
+        return new ValueGetter() {
+            @Override
+            public ImmutableBytesPtr getLatestValue(ColumnReference ref) throws IOException {
+                return valueMap.get(ReferencingColumn.wrap(ref));
+            }
+        };
+    }
+
+    private static class ReferencingColumn {
+        ImmutableBytesPtr family;
+        ImmutableBytesPtr qual;
+
+        static ReferencingColumn wrap(ColumnReference ref) {
+            ImmutableBytesPtr family = new ImmutableBytesPtr(ref.getFamily());
+            ImmutableBytesPtr qual = new ImmutableBytesPtr(ref.getQualifier());
+            return new ReferencingColumn(family, qual);
+        }
+
+        public ReferencingColumn(ImmutableBytesPtr family, ImmutableBytesPtr qual) {
+            this.family = family;
+            this.qual = qual;
+        }
+
+        @Override
+        public int hashCode() {
+            final int prime = 31;
+            int result = 1;
+            result = prime * result + ((family == null) ? 0 : family.hashCode());
+            result = prime * result + ((qual == null) ? 0 : qual.hashCode());
+            return result;
+        }
+
+        @Override
+        public boolean equals(Object obj) {
+            if (this == obj) return true;
+            if (obj == null) return false;
+            if (getClass() != obj.getClass()) return false;
+            ReferencingColumn other = (ReferencingColumn)obj;
+            if (family == null) {
+                if (other.family != null) return false;
+            } else if (!family.equals(other.family)) return false;
+            if (qual == null) {
+                if (other.qual != null) return false;
+            } else if (!qual.equals(other.qual)) return false;
+            return true;
+        }
+    }
+
+    public static ValueGetter createGetterFromScanner(Scanner scanner, byte[] currentRow) {
+        return new LazyValueGetter(scanner, currentRow);
+    }
+
+    /**
+     * check to see if the kvs in the update match any of the passed columns. Generally, this is useful to for an index
+     * codec to determine if a given update should even be indexed. This assumes that for any index, there are going to
+     * small number of columns, versus the number of kvs in any one batch.
+     */
+    public static boolean updateMatchesColumns(Collection<KeyValue> update, List<ColumnReference> columns) {
+        // check to see if the kvs in the new update even match any of the columns requested
+        // assuming that for any index, there are going to small number of columns, versus the number of
+        // kvs in any one batch.
+        boolean matches = false;
+        outer: for (KeyValue kv : update) {
+            for (ColumnReference ref : columns) {
+                if (ref.matchesFamily(kv.getFamily()) && ref.matchesQualifier(kv.getQualifier())) {
+                    matches = true;
+                    // if a single column matches a single kv, we need to build a whole scanner
+                    break outer;
+                }
+            }
+        }
+        return matches;
+    }
+
+    /**
+     * Check to see if the kvs in the update match any of the passed columns. Generally, this is useful to for an index
+     * codec to determine if a given update should even be indexed. This assumes that for any index, there are going to
+     * small number of kvs, versus the number of columns in any one batch.
+     * <p>
+     * This employs the same logic as {@link #updateMatchesColumns(Collection, List)}, but is flips the iteration logic
+     * to search columns before kvs.
+     */
+    public static boolean columnMatchesUpdate(List<ColumnReference> columns, Collection<KeyValue> update) {
+        boolean matches = false;
+        outer: for (ColumnReference ref : columns) {
+            for (KeyValue kv : update) {
+                if (ref.matchesFamily(kv.getFamily()) && ref.matchesQualifier(kv.getQualifier())) {
+                    matches = true;
+                    // if a single column matches a single kv, we need to build a whole scanner
+                    break outer;
+                }
+            }
+        }
+        return matches;
+    }
+
+    public static Scan newLocalStateScan(List<? extends Iterable<? extends ColumnReference>> refsArray) {
+        Scan s = new Scan();
+        s.setRaw(true);
+        // add the necessary columns to the scan
+        for (Iterable<? extends ColumnReference> refs : refsArray) {
+            for (ColumnReference ref : refs) {
+                s.addFamily(ref.getFamily());
+            }
+        }
+        s.setMaxVersions();
+        return s;
+    }
+
+    /**
+     * Propagate the given failure as a generic {@link IOException}, if it isn't already
+     * 
+     * @param e
+     *            reason indexing failed. If ,tt>null</tt>, throws a {@link NullPointerException}, which should unload
+     *            the coprocessor.
+     */
+    public static void rethrowIndexingException(Throwable e) throws IOException {
+        try {
+            throw e;
+        } catch (IOException e1) {
+            LOG.info("Rethrowing " + e);
+            throw e1;
+        } catch (Throwable e1) {
+            LOG.info("Rethrowing " + e1 + " as a " + IndexBuildingFailureException.class.getSimpleName());
+            throw new IndexBuildingFailureException("Failed to build index for unexpected reason!", e1);
+        }
+    }
+
+    public static void setIfNotSet(Configuration conf, String key, int value) {
+        if (conf.get(key) == null) {
+            conf.setInt(key, value);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/wal/IndexedKeyValue.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/wal/IndexedKeyValue.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/wal/IndexedKeyValue.java
new file mode 100644
index 0000000..a7f4e82
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/wal/IndexedKeyValue.java
@@ -0,0 +1,155 @@
+package org.apache.hadoop.hbase.index.wal;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.regionserver.wal.HLog;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+
+public class IndexedKeyValue extends KeyValue {
+    private static int calcHashCode(ImmutableBytesPtr indexTableName, Mutation mutation) {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + indexTableName.hashCode();
+        result = prime * result + Arrays.hashCode(mutation.getRow());
+        return result;
+    }
+
+    private ImmutableBytesPtr indexTableName;
+    private Mutation mutation;
+    // optimization check to ensure that batches don't get replayed to the index more than once
+    private boolean batchFinished = false;
+    private int hashCode;
+
+    public IndexedKeyValue() {}
+
+    public IndexedKeyValue(byte[] bs, Mutation mutation) {
+        this.indexTableName = new ImmutableBytesPtr(bs);
+        this.mutation = mutation;
+        this.hashCode = calcHashCode(indexTableName, mutation);
+    }
+
+    public byte[] getIndexTable() {
+        return this.indexTableName.get();
+    }
+
+    public Mutation getMutation() {
+        return mutation;
+    }
+
+    /**
+     * This is a KeyValue that shouldn't actually be replayed, so we always mark it as an {@link HLog#METAFAMILY} so it
+     * isn't replayed via the normal replay mechanism
+     */
+    @Override
+    public boolean matchingFamily(final byte[] family) {
+        return Bytes.equals(family, HLog.METAFAMILY);
+    }
+
+    @Override
+    public String toString() {
+        return "IndexWrite:\n\ttable: " + indexTableName + "\n\tmutation:" + mutation;
+    }
+
+    /**
+     * This is a very heavy-weight operation and should only be done when absolutely necessary - it does a full
+     * serialization of the underyling mutation to compare the underlying data.
+     */
+    @Override
+    public boolean equals(Object obj) {
+        if(obj == null) return false;
+        if (this == obj) return true;
+        if (getClass() != obj.getClass()) return false;
+        IndexedKeyValue other = (IndexedKeyValue)obj;
+        if (hashCode() != other.hashCode()) return false;
+        if (!other.indexTableName.equals(this.indexTableName)) return false;
+        byte[] current = this.getMutationBytes();
+        byte[] otherMutation = other.getMutationBytes();
+        return Bytes.equals(current, otherMutation);
+    }
+
+    private byte[] getMutationBytes() {
+        ByteArrayOutputStream bos = null;
+        try {
+            bos = new ByteArrayOutputStream();
+            this.mutation.write(new DataOutputStream(bos));
+            bos.flush();
+            return bos.toByteArray();
+        } catch (IOException e) {
+            throw new IllegalArgumentException("Failed to get bytes for mutation!", e);
+        } finally {
+            if (bos != null) {
+                try {
+                    bos.close();
+                } catch (IOException e) {
+                    throw new IllegalArgumentException("Failed to get bytes for mutation!", e);
+                }
+            }
+        }
+    }
+
+    @Override
+    public int hashCode() {
+        return hashCode;
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+        KeyValueCodec.write(out, this);
+    }
+
+    /**
+     * Internal write the underlying data for the entry - this does not do any special prefixing. Writing should be done
+     * via {@link KeyValueCodec#write(DataOutput, KeyValue)} to ensure consistent reading/writing of
+     * {@link IndexedKeyValue}s.
+     * 
+     * @param out
+     *            to write data to. Does not close or flush the passed object.
+     * @throws IOException
+     *             if there is a problem writing the underlying data
+     */
+    void writeData(DataOutput out) throws IOException {
+        Bytes.writeByteArray(out, this.indexTableName.get());
+        out.writeUTF(this.mutation.getClass().getName());
+        this.mutation.write(out);
+    }
+
+    /**
+     * This method shouldn't be used - you should use {@link KeyValueCodec#readKeyValue(DataInput)} instead. Its the
+     * complement to {@link #writeData(DataOutput)}.
+     */
+    @SuppressWarnings("javadoc")
+    @Override
+    public void readFields(DataInput in) throws IOException {
+        this.indexTableName = new ImmutableBytesPtr(Bytes.readByteArray(in));
+        Class<? extends Mutation> clazz;
+        try {
+            clazz = Class.forName(in.readUTF()).asSubclass(Mutation.class);
+            this.mutation = clazz.newInstance();
+            this.mutation.readFields(in);
+            this.hashCode = calcHashCode(indexTableName, mutation);
+        } catch (ClassNotFoundException e) {
+            throw new IOException(e);
+        } catch (InstantiationException e) {
+            throw new IOException(e);
+        } catch (IllegalAccessException e) {
+            throw new IOException(e);
+        }
+    }
+
+    public boolean getBatchFinished() {
+        return this.batchFinished;
+    }
+
+    public void markBatchFinished() {
+        this.batchFinished = true;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/wal/KeyValueCodec.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/wal/KeyValueCodec.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/wal/KeyValueCodec.java
new file mode 100644
index 0000000..0abdf8d
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/wal/KeyValueCodec.java
@@ -0,0 +1,79 @@
+package org.apache.hadoop.hbase.index.wal;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+
+/**
+ * Codec to encode/decode {@link KeyValue}s and {@link IndexedKeyValue}s within a {@link WALEdit}
+ */
+public class KeyValueCodec {
+
+  /**
+   * KeyValue length marker specifying that its actually an {@link IndexedKeyValue} rather than a
+   * regular {@link KeyValue}.
+   */
+  public static final int INDEX_TYPE_LENGTH_MARKER = -1;
+
+  /**
+   * Read a {@link List} of {@link KeyValue} from the input stream - may contain regular
+   * {@link KeyValue}s or {@link IndexedKeyValue}s.
+   * @param in to read from
+   * @return the next {@link KeyValue}s
+   * @throws IOException if the next {@link KeyValue} cannot be read
+   */
+  public static List<KeyValue> readKeyValues(DataInput in) throws IOException {
+    int size = in.readInt();
+    if (size == 0) {
+      return Collections.<KeyValue>emptyList();
+    }
+    List<KeyValue> kvs = new ArrayList<KeyValue>(size);
+    for (int i = 0; i < size; i++) {
+      kvs.add(readKeyValue(in));
+    }
+    return kvs;
+  }
+
+  /**
+   * Read a single {@link KeyValue} from the input stream - may either be a regular {@link KeyValue}
+   * or an {@link IndexedKeyValue}.
+   * @param in to read from
+   * @return the next {@link KeyValue}, if one is available
+   * @throws IOException if the next {@link KeyValue} cannot be read
+   */
+  public static KeyValue readKeyValue(DataInput in) throws IOException {
+    int length = in.readInt();
+    KeyValue kv;
+    // its a special IndexedKeyValue
+    if (length == INDEX_TYPE_LENGTH_MARKER) {
+      kv = new IndexedKeyValue();
+      kv.readFields(in);
+    } else {
+      kv = new KeyValue();
+      kv.readFields(length, in);
+    }
+    return kv;
+  }
+
+  /**
+   * Write a {@link KeyValue} or an {@link IndexedKeyValue} to the output stream. These can be read
+   * back via {@link #readKeyValue(DataInput)} or {@link #readKeyValues(DataInput)}.
+   * @param out to write to
+   * @param kv {@link KeyValue} to which to write
+   * @throws IOException if there is an error writing
+   */
+  public static void write(DataOutput out, KeyValue kv) throws IOException {
+    if (kv instanceof IndexedKeyValue) {
+      out.writeInt(INDEX_TYPE_LENGTH_MARKER);
+      ((IndexedKeyValue) kv).writeData(out);
+    } else {
+      kv.write(out);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/IndexCommitter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/IndexCommitter.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/IndexCommitter.java
new file mode 100644
index 0000000..aed4b89
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/IndexCommitter.java
@@ -0,0 +1,39 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index.write;
+
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+
+import com.google.common.collect.Multimap;
+import org.apache.hadoop.hbase.index.exception.IndexWriteException;
+import org.apache.hadoop.hbase.index.table.HTableInterfaceReference;
+
+/**
+ * Write the index updates to the index tables
+ */
+public interface IndexCommitter extends Stoppable {
+
+  void setup(IndexWriter parent, RegionCoprocessorEnvironment env, String name);
+
+  public void write(Multimap<HTableInterfaceReference, Mutation> toWrite)
+      throws IndexWriteException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/IndexFailurePolicy.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/IndexFailurePolicy.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/IndexFailurePolicy.java
new file mode 100644
index 0000000..683efd5
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/IndexFailurePolicy.java
@@ -0,0 +1,46 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index.write;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+
+import com.google.common.collect.Multimap;
+import org.apache.hadoop.hbase.index.table.HTableInterfaceReference;
+
+/**
+ * Handle failures to write to the index tables.
+ */
+public interface IndexFailurePolicy extends Stoppable {
+
+  public void setup(Stoppable parent, RegionCoprocessorEnvironment env);
+
+  /**
+   * Handle the failure of the attempted index updates
+   * @param attempted map of index table -> mutations to apply
+   * @param cause reason why there was a failure
+ * @throws IOException 
+   */
+  public void
+      handleFailure(Multimap<HTableInterfaceReference, Mutation> attempted, Exception cause) throws IOException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/IndexWriter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/IndexWriter.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/IndexWriter.java
new file mode 100644
index 0000000..8c30642
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/IndexWriter.java
@@ -0,0 +1,226 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index.write;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.util.Pair;
+
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+import org.apache.hadoop.hbase.index.exception.IndexWriteException;
+import org.apache.hadoop.hbase.index.table.HTableInterfaceReference;
+import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+
+/**
+ * Do the actual work of writing to the index tables. Ensures that if we do fail to write to the
+ * index table that we cleanly kill the region/server to ensure that the region's WAL gets replayed.
+ * <p>
+ * We attempt to do the index updates in parallel using a backing threadpool. All threads are daemon
+ * threads, so it will not block the region from shutting down.
+ */
+public class IndexWriter implements Stoppable {
+
+  private static final Log LOG = LogFactory.getLog(IndexWriter.class);
+  private static final String INDEX_COMMITTER_CONF_KEY = "index.writer.commiter.class";
+  public static final String INDEX_FAILURE_POLICY_CONF_KEY = "index.writer.failurepolicy.class";
+  private AtomicBoolean stopped = new AtomicBoolean(false);
+  private IndexCommitter writer;
+  private IndexFailurePolicy failurePolicy;
+
+  /**
+   * @throws IOException if the {@link IndexWriter} or {@link IndexFailurePolicy} cannot be
+   *           instantiated
+   */
+  public IndexWriter(RegionCoprocessorEnvironment env, String name) throws IOException {
+    this(getCommitter(env), getFailurePolicy(env), env, name);
+  }
+
+  public static IndexCommitter getCommitter(RegionCoprocessorEnvironment env) throws IOException {
+    Configuration conf = env.getConfiguration();
+    try {
+      IndexCommitter committer =
+          conf.getClass(INDEX_COMMITTER_CONF_KEY, ParallelWriterIndexCommitter.class,
+            IndexCommitter.class).newInstance();
+      return committer;
+    } catch (InstantiationException e) {
+      throw new IOException(e);
+    } catch (IllegalAccessException e) {
+      throw new IOException(e);
+    }
+  }
+
+  public static IndexFailurePolicy getFailurePolicy(RegionCoprocessorEnvironment env)
+      throws IOException {
+    Configuration conf = env.getConfiguration();
+    try {
+      IndexFailurePolicy committer =
+          conf.getClass(INDEX_FAILURE_POLICY_CONF_KEY, KillServerOnFailurePolicy.class,
+            IndexFailurePolicy.class).newInstance();
+      return committer;
+    } catch (InstantiationException e) {
+      throw new IOException(e);
+    } catch (IllegalAccessException e) {
+      throw new IOException(e);
+    }
+  }
+
+  /**
+   * Directly specify the {@link IndexCommitter} and {@link IndexFailurePolicy}. Both are expected
+   * to be fully setup before calling.
+   * @param committer
+   * @param policy
+   * @param env
+   */
+  public IndexWriter(IndexCommitter committer, IndexFailurePolicy policy,
+      RegionCoprocessorEnvironment env, String name) {
+    this(committer, policy);
+    this.writer.setup(this, env, name);
+    this.failurePolicy.setup(this, env);
+  }
+
+  /**
+   * Create an {@link IndexWriter} with an already setup {@link IndexCommitter} and
+   * {@link IndexFailurePolicy}.
+   * @param committer to write updates
+   * @param policy to handle failures
+   */
+  IndexWriter(IndexCommitter committer, IndexFailurePolicy policy) {
+    this.writer = committer;
+    this.failurePolicy = policy;
+  }
+  
+  /**
+   * Write the mutations to their respective table.
+   * <p>
+   * This method is blocking and could potentially cause the writer to block for a long time as we
+   * write the index updates. When we return depends on the specified {@link IndexCommitter}.
+   * <p>
+   * If update fails, we pass along the failure to the installed {@link IndexFailurePolicy}, which
+   * then decides how to handle the failure. By default, we use a {@link KillServerOnFailurePolicy},
+   * which ensures that the server crashes when an index write fails, ensuring that we get WAL
+   * replay of the index edits.
+   * @param indexUpdates Updates to write
+ * @throws IOException 
+   */
+  public void writeAndKillYourselfOnFailure(Collection<Pair<Mutation, byte[]>> indexUpdates) throws IOException  {
+    // convert the strings to htableinterfaces to which we can talk and group by TABLE
+    Multimap<HTableInterfaceReference, Mutation> toWrite = resolveTableReferences(indexUpdates);
+    writeAndKillYourselfOnFailure(toWrite);
+  }
+
+  /**
+   * see {@link #writeAndKillYourselfOnFailure(Collection)}.
+   * @param toWrite
+ * @throws IOException 
+   */
+  public void writeAndKillYourselfOnFailure(Multimap<HTableInterfaceReference, Mutation> toWrite) throws IOException {
+    try {
+      write(toWrite);
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Done writing all index updates!\n\t" + toWrite);
+      }
+    } catch (Exception e) {
+      this.failurePolicy.handleFailure(toWrite, e);
+    }
+  }
+
+  /**
+   * Write the mutations to their respective table.
+   * <p>
+   * This method is blocking and could potentially cause the writer to block for a long time as we
+   * write the index updates. We only return when either:
+   * <ol>
+   * <li>All index writes have returned, OR</li>
+   * <li>Any single index write has failed</li>
+   * </ol>
+   * We attempt to quickly determine if any write has failed and not write to the remaining indexes
+   * to ensure a timely recovery of the failed index writes.
+   * @param toWrite Updates to write
+   * @throws IndexWriteException if we cannot successfully write to the index. Whether or not we
+   *           stop early depends on the {@link IndexCommitter}.
+   */
+  public void write(Collection<Pair<Mutation, byte[]>> toWrite) throws IndexWriteException {
+    write(resolveTableReferences(toWrite));
+  }
+
+  /**
+   * see {@link #write(Collection)}
+   * @param toWrite
+   * @throws IndexWriteException
+   */
+  public void write(Multimap<HTableInterfaceReference, Mutation> toWrite)
+      throws IndexWriteException {
+    this.writer.write(toWrite);
+  }
+
+
+  /**
+   * Convert the passed index updates to {@link HTableInterfaceReference}s.
+   * @param indexUpdates from the index builder
+   * @return pairs that can then be written by an {@link IndexWriter}.
+   */
+  public static Multimap<HTableInterfaceReference, Mutation> resolveTableReferences(
+      Collection<Pair<Mutation, byte[]>> indexUpdates) {
+    Multimap<HTableInterfaceReference, Mutation> updates = ArrayListMultimap
+        .<HTableInterfaceReference, Mutation> create();
+    // simple map to make lookups easy while we build the map of tables to create
+    Map<ImmutableBytesPtr, HTableInterfaceReference> tables =
+        new HashMap<ImmutableBytesPtr, HTableInterfaceReference>(updates.size());
+    for (Pair<Mutation, byte[]> entry : indexUpdates) {
+      byte[] tableName = entry.getSecond();
+      ImmutableBytesPtr ptr = new ImmutableBytesPtr(tableName);
+      HTableInterfaceReference table = tables.get(ptr);
+      if (table == null) {
+        table = new HTableInterfaceReference(ptr);
+        tables.put(ptr, table);
+      }
+      updates.put(table, entry.getFirst());
+    }
+
+    return updates;
+  }
+
+  @Override
+  public void stop(String why) {
+    if (!this.stopped.compareAndSet(false, true)) {
+      // already stopped
+      return;
+    }
+    LOG.debug("Stopping because " + why);
+    this.writer.stop(why);
+    this.failurePolicy.stop(why);
+  }
+
+  @Override
+  public boolean isStopped() {
+    return this.stopped.get();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/IndexWriterUtils.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/IndexWriterUtils.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/IndexWriterUtils.java
new file mode 100644
index 0000000..bee8953
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/IndexWriterUtils.java
@@ -0,0 +1,74 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index.write;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CoprocessorEnvironment;
+
+import org.apache.hadoop.hbase.index.table.CoprocessorHTableFactory;
+import org.apache.hadoop.hbase.index.table.HTableFactory;
+import org.apache.hadoop.hbase.index.util.IndexManagementUtil;
+
+public class IndexWriterUtils {
+
+  private static final Log LOG = LogFactory.getLog(IndexWriterUtils.class);
+
+  /**
+   * Maximum number of threads to allow per-table when writing. Each writer thread (from
+   * {@link IndexWriterUtils#NUM_CONCURRENT_INDEX_WRITER_THREADS_CONF_KEY}) has a single HTable.
+   * However, each table is backed by a threadpool to manage the updates to that table. this
+   * specifies the number of threads to allow in each of those tables. Generally, you shouldn't need
+   * to change this, unless you have a small number of indexes to which most of the writes go.
+   * Defaults to: {@value #DEFAULT_NUM_PER_TABLE_THREADS}.
+   * <p>
+   * For tables to which there are not a lot of writes, the thread pool automatically will decrease
+   * the number of threads to one (though it can burst up to the specified max for any given table),
+   * so increasing this to meet the max case is reasonable.
+   * <p>
+   * Setting this value too small can cause <b>catastrophic cluster failure</b>. The way HTable's
+   * underlying pool works is such that is does direct hand-off of tasks to threads. This works fine
+   * because HTables are assumed to work in a single-threaded context, so we never get more threads
+   * than regionservers. In a multi-threaded context, we can easily grow to more than that number of
+   * threads. Currently, HBase doesn't support a custom thread-pool to back the HTable via the
+   * coprocesor hooks, so we can't modify this behavior.
+   */
+  private static final String INDEX_WRITER_PER_TABLE_THREADS_CONF_KEY =
+      "index.writer.threads.pertable.max";
+  private static final int DEFAULT_NUM_PER_TABLE_THREADS = Integer.MAX_VALUE;
+
+  /** Configuration key that HBase uses to set the max number of threads for an HTable */
+  public static final String HTABLE_THREAD_KEY = "hbase.htable.threads.max";
+  private IndexWriterUtils() {
+    // private ctor for utilites
+  }
+
+  public static HTableFactory getDefaultDelegateHTableFactory(CoprocessorEnvironment env) {
+    // create a simple delegate factory, setup the way we need
+    Configuration conf = env.getConfiguration();
+    // set the number of threads allowed per table.
+    int htableThreads =
+        conf.getInt(IndexWriterUtils.INDEX_WRITER_PER_TABLE_THREADS_CONF_KEY, IndexWriterUtils.DEFAULT_NUM_PER_TABLE_THREADS);
+    LOG.trace("Creating HTableFactory with " + htableThreads + " threads for each HTable.");
+    IndexManagementUtil.setIfNotSet(conf, HTABLE_THREAD_KEY, htableThreads);
+    return new CoprocessorHTableFactory(env);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/KillServerOnFailurePolicy.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/KillServerOnFailurePolicy.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/KillServerOnFailurePolicy.java
new file mode 100644
index 0000000..76bde1a
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/KillServerOnFailurePolicy.java
@@ -0,0 +1,82 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index.write;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+
+import com.google.common.collect.Multimap;
+import org.apache.hadoop.hbase.index.table.HTableInterfaceReference;
+
+/**
+ * Naive failure policy - kills the server on which it resides
+ */
+public class KillServerOnFailurePolicy implements IndexFailurePolicy {
+
+  private static final Log LOG = LogFactory.getLog(KillServerOnFailurePolicy.class);
+  private Abortable abortable;
+  private Stoppable stoppable;
+
+  @Override
+  public void setup(Stoppable parent, RegionCoprocessorEnvironment env) {
+    setup(parent, env.getRegionServerServices());
+  }
+
+  public void setup(Stoppable parent, Abortable abort) {
+    this.stoppable = parent;
+    this.abortable = abort;
+  }
+
+  @Override
+  public void stop(String why) {
+    // noop
+  }
+
+  @Override
+  public boolean isStopped() {
+    return this.stoppable.isStopped();
+  }
+
+  @Override
+  public void
+      handleFailure(Multimap<HTableInterfaceReference, Mutation> attempted, Exception cause) throws IOException {
+    // cleanup resources
+    this.stop("Killing ourselves because of an error:" + cause);
+    // notify the regionserver of the failure
+    String msg =
+        "Could not update the index table, killing server region because couldn't write to an index table";
+    LOG.error(msg, cause);
+    try {
+      this.abortable.abort(msg, cause);
+    } catch (Exception e) {
+      LOG.fatal("Couldn't abort this server to preserve index writes, "
+          + "attempting to hard kill the server");
+      System.exit(1);
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/ParallelWriterIndexCommitter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/ParallelWriterIndexCommitter.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/ParallelWriterIndexCommitter.java
new file mode 100644
index 0000000..86abde4
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/ParallelWriterIndexCommitter.java
@@ -0,0 +1,212 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index.write;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+
+import com.google.common.collect.Multimap;
+import org.apache.hadoop.hbase.index.exception.SingleIndexWriteFailureException;
+import org.apache.hadoop.hbase.index.parallel.EarlyExitFailure;
+import org.apache.hadoop.hbase.index.parallel.QuickFailingTaskRunner;
+import org.apache.hadoop.hbase.index.parallel.Task;
+import org.apache.hadoop.hbase.index.parallel.TaskBatch;
+import org.apache.hadoop.hbase.index.parallel.ThreadPoolBuilder;
+import org.apache.hadoop.hbase.index.parallel.ThreadPoolManager;
+import org.apache.hadoop.hbase.index.table.CachingHTableFactory;
+import org.apache.hadoop.hbase.index.table.HTableFactory;
+import org.apache.hadoop.hbase.index.table.HTableInterfaceReference;
+
+/**
+ * Write index updates to the index tables in parallel. We attempt to early exit from the writes if
+ * any of the index updates fails. Completion is determined by the following criteria: *
+ * <ol>
+ * <li>All index writes have returned, OR</li>
+ * <li>Any single index write has failed</li>
+ * </ol>
+ * We attempt to quickly determine if any write has failed and not write to the remaining indexes to
+ * ensure a timely recovery of the failed index writes.
+ */
+public class ParallelWriterIndexCommitter implements IndexCommitter {
+
+  public static final String NUM_CONCURRENT_INDEX_WRITER_THREADS_CONF_KEY = "index.writer.threads.max";
+  private static final int DEFAULT_CONCURRENT_INDEX_WRITER_THREADS = 10;
+  private static final String INDEX_WRITER_KEEP_ALIVE_TIME_CONF_KEY =
+      "index.writer.threads.keepalivetime";
+  private static final Log LOG = LogFactory.getLog(ParallelWriterIndexCommitter.class);
+
+  private HTableFactory factory;
+  private Stoppable stopped;
+  private QuickFailingTaskRunner pool;
+
+  @Override
+  public void setup(IndexWriter parent, RegionCoprocessorEnvironment env, String name) {
+    Configuration conf = env.getConfiguration();
+    setup(IndexWriterUtils.getDefaultDelegateHTableFactory(env),
+      ThreadPoolManager.getExecutor(
+        new ThreadPoolBuilder(name, conf).
+          setMaxThread(NUM_CONCURRENT_INDEX_WRITER_THREADS_CONF_KEY,
+            DEFAULT_CONCURRENT_INDEX_WRITER_THREADS).
+          setCoreTimeout(INDEX_WRITER_KEEP_ALIVE_TIME_CONF_KEY), env),
+      env.getRegionServerServices(), parent, CachingHTableFactory.getCacheSize(conf));
+  }
+
+  /**
+   * Setup <tt>this</tt>.
+   * <p>
+   * Exposed for TESTING
+   */
+  void setup(HTableFactory factory, ExecutorService pool, Abortable abortable, Stoppable stop,
+      int cacheSize) {
+    this.factory = new CachingHTableFactory(factory, cacheSize);
+    this.pool = new QuickFailingTaskRunner(pool);
+    this.stopped = stop;
+  }
+
+  @Override
+  public void write(Multimap<HTableInterfaceReference, Mutation> toWrite)
+      throws SingleIndexWriteFailureException {
+    /*
+     * This bit here is a little odd, so let's explain what's going on. Basically, we want to do the
+     * writes in parallel to each index table, so each table gets its own task and is submitted to
+     * the pool. Where it gets tricky is that we want to block the calling thread until one of two
+     * things happens: (1) all index tables get successfully updated, or (2) any one of the index
+     * table writes fail; in either case, we should return as quickly as possible. We get a little
+     * more complicated in that if we do get a single failure, but any of the index writes hasn't
+     * been started yet (its been queued up, but not submitted to a thread) we want to that task to
+     * fail immediately as we know that write is a waste and will need to be replayed anyways.
+     */
+
+    Set<Entry<HTableInterfaceReference, Collection<Mutation>>> entries = toWrite.asMap().entrySet();
+    TaskBatch<Void> tasks = new TaskBatch<Void>(entries.size());
+    for (Entry<HTableInterfaceReference, Collection<Mutation>> entry : entries) {
+      // get the mutations for each table. We leak the implementation here a little bit to save
+      // doing a complete copy over of all the index update for each table.
+      final List<Mutation> mutations = (List<Mutation>) entry.getValue();
+      final HTableInterfaceReference tableReference = entry.getKey();
+      /*
+       * Write a batch of index updates to an index table. This operation stops (is cancelable) via
+       * two mechanisms: (1) setting aborted or stopped on the IndexWriter or, (2) interrupting the
+       * running thread. The former will only work if we are not in the midst of writing the current
+       * batch to the table, though we do check these status variables before starting and before
+       * writing the batch. The latter usage, interrupting the thread, will work in the previous
+       * situations as was at some points while writing the batch, depending on the underlying
+       * writer implementation (HTableInterface#batch is blocking, but doesn't elaborate when is
+       * supports an interrupt).
+       */
+      tasks.add(new Task<Void>() {
+
+        /**
+         * Do the actual write to the primary table. We don't need to worry about closing the table
+         * because that is handled the {@link CachingHTableFactory}.
+         */
+        @Override
+        public Void call() throws Exception {
+          // this may have been queued, so another task infront of us may have failed, so we should
+          // early exit, if that's the case
+          throwFailureIfDone();
+
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Writing index update:" + mutations + " to table: " + tableReference);
+          }
+          try {
+            HTableInterface table = factory.getTable(tableReference.get());
+            throwFailureIfDone();
+            table.batch(mutations);
+          } catch (SingleIndexWriteFailureException e) {
+            throw e;
+          } catch (IOException e) {
+            throw new SingleIndexWriteFailureException(tableReference.toString(), mutations, e);
+          } catch (InterruptedException e) {
+            // reset the interrupt status on the thread
+            Thread.currentThread().interrupt();
+            throw new SingleIndexWriteFailureException(tableReference.toString(), mutations, e);
+          }
+          return null;
+        }
+
+        private void throwFailureIfDone() throws SingleIndexWriteFailureException {
+          if (this.isBatchFailed() || Thread.currentThread().isInterrupted()) {
+            throw new SingleIndexWriteFailureException(
+                "Pool closed, not attempting to write to the index!", null);
+          }
+
+        }
+      });
+    }
+
+    // actually submit the tasks to the pool and wait for them to finish/fail
+    try {
+      pool.submitUninterruptible(tasks);
+    } catch (EarlyExitFailure e) {
+      propagateFailure(e);
+    } catch (ExecutionException e) {
+      LOG.error("Found a failed index update!");
+      propagateFailure(e.getCause());
+    }
+
+  }
+
+  private void propagateFailure(Throwable throwable) throws SingleIndexWriteFailureException {
+    try {
+      throw throwable;
+    } catch (SingleIndexWriteFailureException e1) {
+      throw e1;
+    } catch (Throwable e1) {
+      throw new SingleIndexWriteFailureException(
+          "Got an abort notification while writing to the index!", e1);
+    }
+
+  }
+
+  /**
+   * {@inheritDoc}
+   * <p>
+   * This method should only be called <b>once</b>. Stopped state ({@link #isStopped()}) is managed
+   * by the external {@link Stoppable}. This call does not delegate the stop down to the
+   * {@link Stoppable} passed in the constructor.
+   * @param why the reason for stopping
+   */
+  @Override
+  public void stop(String why) {
+    LOG.info("Shutting down " + this.getClass().getSimpleName() + " because " + why);
+    this.pool.stop(why);
+    this.factory.shutdown();
+  }
+
+  @Override
+  public boolean isStopped() {
+    return this.stopped.isStopped();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/recovery/PerRegionIndexWriteCache.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/recovery/PerRegionIndexWriteCache.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/recovery/PerRegionIndexWriteCache.java
new file mode 100644
index 0000000..3d77a83
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/recovery/PerRegionIndexWriteCache.java
@@ -0,0 +1,65 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index.write.recovery;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+import org.apache.hadoop.hbase.index.table.HTableInterfaceReference;
+
+
+public class PerRegionIndexWriteCache {
+
+  private Map<HRegion, Multimap<HTableInterfaceReference, Mutation>> cache =
+      new HashMap<HRegion, Multimap<HTableInterfaceReference, Mutation>>();
+
+
+  /**
+   * Get the edits for the current region. Removes the edits from the cache. To add them back, call
+   * {@link #addEdits(HRegion, HTableInterfaceReference, Collection)}.
+   * @param region
+   * @return Get the edits for the given region. Returns <tt>null</tt> if there are no pending edits
+   *         for the region
+   */
+  public Multimap<HTableInterfaceReference, Mutation> getEdits(HRegion region) {
+    return cache.remove(region);
+  }
+
+  /**
+   * @param region
+   * @param table
+   * @param collection
+   */
+  public void addEdits(HRegion region, HTableInterfaceReference table,
+      Collection<Mutation> collection) {
+    Multimap<HTableInterfaceReference, Mutation> edits = cache.get(region);
+    if (edits == null) {
+      edits = ArrayListMultimap.<HTableInterfaceReference, Mutation> create();
+      cache.put(region, edits);
+    }
+    edits.putAll(table, collection);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/recovery/StoreFailuresInCachePolicy.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/recovery/StoreFailuresInCachePolicy.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/recovery/StoreFailuresInCachePolicy.java
new file mode 100644
index 0000000..a17395e
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/recovery/StoreFailuresInCachePolicy.java
@@ -0,0 +1,84 @@
+/**
+ * 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.hadoop.hbase.index.write.recovery;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+
+import com.google.common.collect.Multimap;
+import org.apache.hadoop.hbase.index.exception.MultiIndexWriteFailureException;
+import org.apache.hadoop.hbase.index.table.HTableInterfaceReference;
+import org.apache.hadoop.hbase.index.write.IndexFailurePolicy;
+import org.apache.hadoop.hbase.index.write.KillServerOnFailurePolicy;
+
+/**
+ * Tracks any failed writes in The {@link PerRegionIndexWriteCache}, given a
+ * {@link MultiIndexWriteFailureException} (which is thrown from the
+ * {@link TrackingParallelWriterIndexCommitter}. Any other exception failure causes the a server
+ * abort via the usual {@link KillServerOnFailurePolicy}.
+ */
+public class StoreFailuresInCachePolicy implements IndexFailurePolicy {
+
+  private KillServerOnFailurePolicy delegate;
+  private PerRegionIndexWriteCache cache;
+  private HRegion region;
+
+  /**
+   * @param failedIndexEdits cache to update when we find a failure
+   */
+  public StoreFailuresInCachePolicy(PerRegionIndexWriteCache failedIndexEdits) {
+    this.cache = failedIndexEdits;
+  }
+
+  @Override
+  public void setup(Stoppable parent, RegionCoprocessorEnvironment env) {
+    this.region = env.getRegion();
+    this.delegate = new KillServerOnFailurePolicy();
+    this.delegate.setup(parent, env);
+
+  }
+
+  @Override
+  public void handleFailure(Multimap<HTableInterfaceReference, Mutation> attempted, Exception cause) throws IOException {
+    // if its not an exception we can handle, let the delegate take care of it
+    if (!(cause instanceof MultiIndexWriteFailureException)) {
+      delegate.handleFailure(attempted, cause);
+    }
+    List<HTableInterfaceReference> failedTables =
+        ((MultiIndexWriteFailureException) cause).getFailedTables();
+    for (HTableInterfaceReference table : failedTables) {
+      cache.addEdits(this.region, table, attempted.get(table));
+    }
+  }
+
+
+  @Override
+  public void stop(String why) {
+    this.delegate.stop(why);
+  }
+
+  @Override
+  public boolean isStopped() {
+    return this.delegate.isStopped();
+  }
+}
\ No newline at end of file


[38/51] [partial] Initial commit of master branch from github

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/compile/TrackOrderPreservingExpressionCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/TrackOrderPreservingExpressionCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/TrackOrderPreservingExpressionCompiler.java
new file mode 100644
index 0000000..a0c4674
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/TrackOrderPreservingExpressionCompiler.java
@@ -0,0 +1,211 @@
+package org.apache.phoenix.compile;
+
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+
+import com.google.common.base.Objects;
+import com.google.common.collect.Lists;
+import org.apache.phoenix.compile.GroupByCompiler.GroupBy;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.LiteralExpression;
+import org.apache.phoenix.expression.function.FunctionExpression;
+import org.apache.phoenix.expression.function.FunctionExpression.OrderPreserving;
+import org.apache.phoenix.parse.CaseParseNode;
+import org.apache.phoenix.parse.ColumnParseNode;
+import org.apache.phoenix.parse.DivideParseNode;
+import org.apache.phoenix.parse.MultiplyParseNode;
+import org.apache.phoenix.parse.SubtractParseNode;
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.ColumnRef;
+import org.apache.phoenix.util.SchemaUtil;
+
+/**
+ * Visitor that builds the expressions of a GROUP BY and ORDER BY clause. While traversing
+ * the parse node tree, the visitor also determines if the natural key order of the scan
+ * will match the order of the expressions. For GROUP BY, if order is preserved we can use
+ * an optimization during server-side aggregation to do the aggregation on-the-fly versus
+ * keeping track of each distinct group. We can only do this optimization if all the rows
+ * for each group will be contiguous. For ORDER BY, we can drop the ORDER BY statement if
+ * the order is preserved.
+ * 
+ */
+public class TrackOrderPreservingExpressionCompiler extends ExpressionCompiler {
+    public enum Ordering {ORDERED, UNORDERED};
+    
+    private final List<Entry> entries;
+    private final Ordering ordering;
+    private final int positionOffset;
+    private OrderPreserving orderPreserving = OrderPreserving.YES;
+    private ColumnRef columnRef;
+    private boolean isOrderPreserving = true;
+    private Boolean isReverse;
+    
+    TrackOrderPreservingExpressionCompiler(StatementContext context, GroupBy groupBy, int expectedEntrySize, Ordering ordering) {
+        super(context, groupBy);
+        positionOffset =  context.getResolver().getTables().get(0).getTable().getBucketNum() == null ? 0 : 1;
+        entries = Lists.newArrayListWithExpectedSize(expectedEntrySize);
+        this.ordering = ordering;
+    }
+    
+    public Boolean isReverse() {
+        return isReverse;
+    }
+
+    public boolean isOrderPreserving() {
+        if (!isOrderPreserving) {
+            return false;
+        }
+        if (ordering == Ordering.UNORDERED) {
+            // Sort by position
+            Collections.sort(entries, new Comparator<Entry>() {
+                @Override
+                public int compare(Entry o1, Entry o2) {
+                    return o1.getPkPosition()-o2.getPkPosition();
+                }
+            });
+        }
+        // Determine if there are any gaps in the PK columns (in which case we don't need
+        // to sort in the coprocessor because the keys will already naturally be in sorted
+        // order.
+        int prevPos = positionOffset - 1;
+        OrderPreserving prevOrderPreserving = OrderPreserving.YES;
+        for (int i = 0; i < entries.size() && isOrderPreserving; i++) {
+            Entry entry = entries.get(i);
+            int pos = entry.getPkPosition();
+            isOrderPreserving &= (entry.getOrderPreserving() != OrderPreserving.NO) && (pos == prevPos || ((pos - 1 == prevPos) && (prevOrderPreserving == OrderPreserving.YES)));
+            prevPos = pos;
+            prevOrderPreserving = entries.get(i).getOrderPreserving();
+        }
+        return isOrderPreserving;
+    }
+    
+    @Override
+    protected Expression addExpression(Expression expression) {
+        // TODO: have FunctionExpression visitor instead and remove this cast
+        if (expression instanceof FunctionExpression) {
+            // Keep the minimum value between this function and the current value,
+            // so that we never increase OrderPreserving from NO or YES_IF_LAST.
+            orderPreserving = OrderPreserving.values()[Math.min(orderPreserving.ordinal(), ((FunctionExpression)expression).preservesOrder().ordinal())];
+        }
+        return super.addExpression(expression);
+    }
+
+    @Override
+    public boolean visitEnter(CaseParseNode node) throws SQLException {
+        orderPreserving = OrderPreserving.NO;
+        return super.visitEnter(node);
+    }
+    
+    @Override
+    public boolean visitEnter(DivideParseNode node) throws SQLException {
+        // A divide expression may not preserve row order.
+        // For example: GROUP BY 1/x
+        orderPreserving = OrderPreserving.NO;
+        return super.visitEnter(node);
+    }
+
+    @Override
+    public boolean visitEnter(SubtractParseNode node) throws SQLException {
+        // A subtract expression may not preserve row order.
+        // For example: GROUP BY 10 - x
+        orderPreserving = OrderPreserving.NO;
+        return super.visitEnter(node);
+    }
+
+    @Override
+    public boolean visitEnter(MultiplyParseNode node) throws SQLException {
+        // A multiply expression may not preserve row order.
+        // For example: GROUP BY -1 * x
+        orderPreserving = OrderPreserving.NO;
+        return super.visitEnter(node);
+    }
+
+    @Override
+    public void reset() {
+        super.reset();
+        columnRef = null;
+        orderPreserving = OrderPreserving.YES;
+    }
+    
+    @Override
+    protected ColumnRef resolveColumn(ColumnParseNode node) throws SQLException {
+        ColumnRef ref = super.resolveColumn(node);
+        // If we encounter any non PK column, then we can't aggregate on-the-fly
+        // because the distinct groups have no correlation to the KV column value
+        if (!SchemaUtil.isPKColumn(ref.getColumn())) {
+            orderPreserving = OrderPreserving.NO;
+        }
+        
+        if (columnRef == null) {
+            columnRef = ref;
+        } else if (!columnRef.equals(ref)) {
+            // If we encounter more than one column reference in an expression,
+            // we can't assume the result of the expression will be key ordered.
+            // For example GROUP BY a * b
+            orderPreserving = OrderPreserving.NO;
+        }
+        return ref;
+    }
+
+    public boolean addEntry(Expression expression) {
+        if (expression instanceof LiteralExpression) {
+            return false;
+        }
+        isOrderPreserving &= (orderPreserving != OrderPreserving.NO);
+        entries.add(new Entry(expression, columnRef, orderPreserving));
+        return true;
+    }
+    
+    public boolean addEntry(Expression expression, ColumnModifier modifier) {
+        // If the expression is sorted in a different order than the specified sort order
+        // then the expressions are not order preserving.
+        if (!Objects.equal(expression.getColumnModifier(), modifier)) {
+            if (isReverse == null) {
+                isReverse = true;
+            } else if (!isReverse){
+                orderPreserving = OrderPreserving.NO;
+            }
+        } else {
+            if (isReverse == null) {
+                isReverse = false;
+            } else if (isReverse){
+                orderPreserving = OrderPreserving.NO;
+            }
+        }
+        return addEntry(expression);
+    }
+    
+    public List<Entry> getEntries() {
+        return entries;
+    }
+
+    public static class Entry {
+        private final Expression expression;
+        private final ColumnRef columnRef;
+        private final OrderPreserving orderPreserving;
+        
+        private Entry(Expression expression, ColumnRef columnRef, OrderPreserving orderPreserving) {
+            this.expression = expression;
+            this.columnRef = columnRef;
+            this.orderPreserving = orderPreserving;
+        }
+
+        public Expression getExpression() {
+            return expression;
+        }
+
+        public int getPkPosition() {
+            return columnRef.getPKSlotPosition();
+        }
+
+        public int getColumnPosition() {
+            return columnRef.getColumnPosition();
+        }
+
+        public OrderPreserving getOrderPreserving() {
+            return orderPreserving;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..7f12117
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
@@ -0,0 +1,837 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.compile;
+
+import static com.google.common.collect.Lists.newArrayListWithCapacity;
+
+import java.sql.ParameterMetaData;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.BitSet;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.cache.ServerCacheClient.ServerCache;
+import org.apache.phoenix.compile.GroupByCompiler.GroupBy;
+import org.apache.phoenix.compile.OrderByCompiler.OrderBy;
+import org.apache.phoenix.coprocessor.UngroupedAggregateRegionObserver;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+import org.apache.phoenix.execute.AggregatePlan;
+import org.apache.phoenix.execute.MutationState;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.LiteralExpression;
+import org.apache.phoenix.index.IndexMetaDataCacheClient;
+import org.apache.phoenix.index.PhoenixIndexCodec;
+import org.apache.phoenix.iterate.ParallelIterators.ParallelIteratorFactory;
+import org.apache.phoenix.iterate.ResultIterator;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixResultSet;
+import org.apache.phoenix.jdbc.PhoenixStatement;
+import org.apache.phoenix.optimize.QueryOptimizer;
+import org.apache.phoenix.parse.AliasedNode;
+import org.apache.phoenix.parse.BindParseNode;
+import org.apache.phoenix.parse.ColumnName;
+import org.apache.phoenix.parse.ColumnParseNode;
+import org.apache.phoenix.parse.ComparisonParseNode;
+import org.apache.phoenix.parse.HintNode;
+import org.apache.phoenix.parse.HintNode.Hint;
+import org.apache.phoenix.parse.IsNullParseNode;
+import org.apache.phoenix.parse.LiteralParseNode;
+import org.apache.phoenix.parse.ParseNode;
+import org.apache.phoenix.parse.SQLParser;
+import org.apache.phoenix.parse.SelectStatement;
+import org.apache.phoenix.parse.SequenceValueParseNode;
+import org.apache.phoenix.parse.UpsertStatement;
+import org.apache.phoenix.query.ConnectionQueryServices;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.ColumnRef;
+import org.apache.phoenix.schema.ConstraintViolationException;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PColumnImpl;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTable.ViewType;
+import org.apache.phoenix.schema.PTableImpl;
+import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.schema.ReadOnlyTableException;
+import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.schema.TypeMismatchException;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.SchemaUtil;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+public class UpsertCompiler {
+    private static void setValues(byte[][] values, int[] pkSlotIndex, int[] columnIndexes, PTable table, Map<ImmutableBytesPtr,Map<PColumn,byte[]>> mutation) {
+        Map<PColumn,byte[]> columnValues = Maps.newHashMapWithExpectedSize(columnIndexes.length);
+        byte[][] pkValues = new byte[table.getPKColumns().size()][];
+        // If the table uses salting, the first byte is the salting byte, set to an empty array
+        // here and we will fill in the byte later in PRowImpl.
+        if (table.getBucketNum() != null) {
+            pkValues[0] = new byte[] {0};
+        }
+        for (int i = 0; i < values.length; i++) {
+            byte[] value = values[i];
+            PColumn column = table.getColumns().get(columnIndexes[i]);
+            if (SchemaUtil.isPKColumn(column)) {
+                pkValues[pkSlotIndex[i]] = value;
+            } else {
+                columnValues.put(column, value);
+            }
+        }
+        ImmutableBytesPtr ptr = new ImmutableBytesPtr();
+        table.newKey(ptr, pkValues);
+        mutation.put(ptr, columnValues);
+    }
+
+    private static MutationState upsertSelect(PhoenixStatement statement, 
+            TableRef tableRef, RowProjector projector, ResultIterator iterator, int[] columnIndexes,
+            int[] pkSlotIndexes) throws SQLException {
+        try {
+            PhoenixConnection connection = statement.getConnection();
+            ConnectionQueryServices services = connection.getQueryServices();
+            int maxSize = services.getProps().getInt(QueryServices.MAX_MUTATION_SIZE_ATTRIB,QueryServicesOptions.DEFAULT_MAX_MUTATION_SIZE);
+            int batchSize = Math.min(connection.getMutateBatchSize(), maxSize);
+            boolean isAutoCommit = connection.getAutoCommit();
+            byte[][] values = new byte[columnIndexes.length][];
+            int rowCount = 0;
+            Map<ImmutableBytesPtr,Map<PColumn,byte[]>> mutation = Maps.newHashMapWithExpectedSize(batchSize);
+            PTable table = tableRef.getTable();
+            ResultSet rs = new PhoenixResultSet(iterator, projector, statement);
+            while (rs.next()) {
+                for (int i = 0; i < values.length; i++) {
+                    PColumn column = table.getColumns().get(columnIndexes[i]);
+                    byte[] byteValue = rs.getBytes(i+1);
+                    Object value = rs.getObject(i+1);
+                    int rsPrecision = rs.getMetaData().getPrecision(i+1);
+                    Integer precision = rsPrecision == 0 ? null : rsPrecision;
+                    int rsScale = rs.getMetaData().getScale(i+1);
+                    Integer scale = rsScale == 0 ? null : rsScale;
+                    // If ColumnModifier from expression in SELECT doesn't match the
+                    // column being projected into then invert the bits.
+                    if (column.getColumnModifier() == ColumnModifier.SORT_DESC) {
+                        byte[] tempByteValue = Arrays.copyOf(byteValue, byteValue.length);
+                        byteValue = ColumnModifier.SORT_DESC.apply(byteValue, 0, tempByteValue, 0, byteValue.length);
+                    }
+                    // We are guaranteed that the two column will have compatible types,
+                    // as we checked that before.
+                    if (!column.getDataType().isSizeCompatible(column.getDataType(),
+                            value, byteValue,
+                            precision, column.getMaxLength(), 
+                            scale, column.getScale())) {
+                        throw new SQLExceptionInfo.Builder(SQLExceptionCode.DATA_INCOMPATIBLE_WITH_TYPE)
+                            .setColumnName(column.getName().getString()).build().buildException();
+                    }
+                    values[i] = column.getDataType().coerceBytes(byteValue, value, column.getDataType(),
+                            precision, scale, column.getMaxLength(), column.getScale());
+                }
+                setValues(values, pkSlotIndexes, columnIndexes, table, mutation);
+                rowCount++;
+                // Commit a batch if auto commit is true and we're at our batch size
+                if (isAutoCommit && rowCount % batchSize == 0) {
+                    MutationState state = new MutationState(tableRef, mutation, 0, maxSize, connection);
+                    connection.getMutationState().join(state);
+                    connection.commit();
+                    mutation.clear();
+                }
+            }
+            // If auto commit is true, this last batch will be committed upon return
+            return new MutationState(tableRef, mutation, rowCount / batchSize * batchSize, maxSize, connection);
+        } finally {
+            iterator.close();
+        }
+    }
+
+    private static class UpsertingParallelIteratorFactory extends MutatingParallelIteratorFactory {
+        private RowProjector projector;
+        private int[] columnIndexes;
+        private int[] pkSlotIndexes;
+
+        private UpsertingParallelIteratorFactory (PhoenixConnection connection, TableRef tableRef) {
+            super(connection, tableRef);
+        }
+
+        @Override
+        protected MutationState mutate(PhoenixConnection connection, ResultIterator iterator) throws SQLException {
+            PhoenixStatement statement = new PhoenixStatement(connection);
+            return upsertSelect(statement, tableRef, projector, iterator, columnIndexes, pkSlotIndexes);
+        }
+        
+        public void setRowProjector(RowProjector projector) {
+            this.projector = projector;
+        }
+        public void setColumnIndexes(int[] columnIndexes) {
+            this.columnIndexes = columnIndexes;
+        }
+        public void setPkSlotIndexes(int[] pkSlotIndexes) {
+            this.pkSlotIndexes = pkSlotIndexes;
+        }
+    }
+    
+    private final PhoenixStatement statement;
+    
+    public UpsertCompiler(PhoenixStatement statement) {
+        this.statement = statement;
+    }
+    
+    public MutationPlan compile(UpsertStatement upsert) throws SQLException {
+        final PhoenixConnection connection = statement.getConnection();
+        ConnectionQueryServices services = connection.getQueryServices();
+        final int maxSize = services.getProps().getInt(QueryServices.MAX_MUTATION_SIZE_ATTRIB,QueryServicesOptions.DEFAULT_MAX_MUTATION_SIZE);
+        final ColumnResolver resolver = FromCompiler.getResolver(upsert, connection);
+        final TableRef tableRef = resolver.getTables().get(0);
+        PTable table = tableRef.getTable();
+        if (table.getType() == PTableType.VIEW) {
+            if (table.getViewType().isReadOnly()) {
+                throw new ReadOnlyTableException(table.getSchemaName().getString(),table.getTableName().getString());
+            }
+        }
+        boolean isSalted = table.getBucketNum() != null;
+        boolean isTenantSpecific = table.isMultiTenant() && connection.getTenantId() != null;
+        String tenantId = isTenantSpecific ? connection.getTenantId().getString() : null;
+        int posOffset = isSalted ? 1 : 0;
+        // Setup array of column indexes parallel to values that are going to be set
+        List<ColumnName> columnNodes = upsert.getColumns();
+        List<PColumn> allColumns = table.getColumns();
+        Map<ColumnRef, byte[]> addViewColumns = Collections.emptyMap();
+        Map<PColumn, byte[]> overlapViewColumns = Collections.emptyMap();
+
+        int[] columnIndexesToBe;
+        int nColumnsToSet = 0;
+        int[] pkSlotIndexesToBe;
+        List<PColumn> targetColumns;
+        if (table.getViewType() == ViewType.UPDATABLE) {
+            StatementContext context = new StatementContext(statement, resolver, this.statement.getParameters(), new Scan());
+            ViewValuesMapBuilder builder = new ViewValuesMapBuilder(context);
+            ParseNode viewNode = new SQLParser(table.getViewStatement()).parseQuery().getWhere();
+            viewNode.accept(builder);
+            addViewColumns = builder.getViewColumns();
+        }
+        // Allow full row upsert if no columns or only dynamic ones are specified and values count match
+        if (columnNodes.isEmpty() || columnNodes.size() == upsert.getTable().getDynamicColumns().size()) {
+            nColumnsToSet = allColumns.size() - posOffset;
+            columnIndexesToBe = new int[nColumnsToSet];
+            pkSlotIndexesToBe = new int[columnIndexesToBe.length];
+            targetColumns = Lists.newArrayListWithExpectedSize(columnIndexesToBe.length);
+            targetColumns.addAll(Collections.<PColumn>nCopies(columnIndexesToBe.length, null));
+            for (int i = posOffset, j = posOffset; i < allColumns.size(); i++) {
+                PColumn column = allColumns.get(i);
+                columnIndexesToBe[i-posOffset] = i;
+                targetColumns.set(i-posOffset, column);
+                if (SchemaUtil.isPKColumn(column)) {
+                    pkSlotIndexesToBe[i-posOffset] = j++;
+                }
+            }
+            if (!addViewColumns.isEmpty()) {
+                // All view columns overlap in this case
+                overlapViewColumns = Maps.newHashMapWithExpectedSize(addViewColumns.size());
+                for (Map.Entry<ColumnRef, byte[]> entry : addViewColumns.entrySet()) {
+                    ColumnRef ref = entry.getKey();
+                    PColumn column = ref.getColumn();
+                    overlapViewColumns.put(column, entry.getValue());
+                }
+                addViewColumns.clear();
+            }
+        } else {
+            // Size for worse case
+            int numColsInUpsert = columnNodes.size();
+            nColumnsToSet = numColsInUpsert + addViewColumns.size() + (isTenantSpecific ? 1 : 0);
+            columnIndexesToBe = new int[nColumnsToSet];
+            pkSlotIndexesToBe = new int[columnIndexesToBe.length];
+            targetColumns = Lists.newArrayListWithExpectedSize(columnIndexesToBe.length);
+            targetColumns.addAll(Collections.<PColumn>nCopies(columnIndexesToBe.length, null));
+            Arrays.fill(columnIndexesToBe, -1); // TODO: necessary? So we'll get an AIOB exception if it's not replaced
+            Arrays.fill(pkSlotIndexesToBe, -1); // TODO: necessary? So we'll get an AIOB exception if it's not replaced
+            BitSet pkColumnsSet = new BitSet(table.getPKColumns().size());
+            int i = 0;
+            for (i = 0; i < numColsInUpsert; i++) {
+                ColumnName colName = columnNodes.get(i);
+                ColumnRef ref = resolver.resolveColumn(null, colName.getFamilyName(), colName.getColumnName());
+                PColumn column = ref.getColumn();
+                byte[] viewValue = addViewColumns.remove(ref);
+                if (viewValue != null) {
+                    if (overlapViewColumns.isEmpty()) {
+                        overlapViewColumns = Maps.newHashMapWithExpectedSize(addViewColumns.size());
+                    }
+                    nColumnsToSet--;
+                    overlapViewColumns.put(column, viewValue);
+                }
+                columnIndexesToBe[i] = ref.getColumnPosition();
+                targetColumns.set(i, column);
+                if (SchemaUtil.isPKColumn(column)) {
+                    pkColumnsSet.set(pkSlotIndexesToBe[i] = ref.getPKSlotPosition());
+                }
+            }
+            for (Map.Entry<ColumnRef, byte[]> entry : addViewColumns.entrySet()) {
+                ColumnRef ref = entry.getKey();
+                PColumn column = ref.getColumn();
+                columnIndexesToBe[i] = ref.getColumnPosition();
+                targetColumns.set(i, column);
+                if (SchemaUtil.isPKColumn(column)) {
+                    pkColumnsSet.set(pkSlotIndexesToBe[i] = ref.getPKSlotPosition());
+                }
+                i++;
+            }
+            // Add tenant column directly, as we don't want to resolve it as this will fail
+            if (isTenantSpecific) {
+                PColumn tenantColumn = table.getPKColumns().get(posOffset);
+                columnIndexesToBe[i] = tenantColumn.getPosition();
+                pkColumnsSet.set(pkSlotIndexesToBe[i] = posOffset);
+                targetColumns.set(i, tenantColumn);
+                i++;
+            }
+            i = posOffset;
+            for ( ; i < table.getPKColumns().size(); i++) {
+                PColumn pkCol = table.getPKColumns().get(i);
+                if (!pkColumnsSet.get(i)) {
+                    if (!pkCol.isNullable()) {
+                        throw new ConstraintViolationException(table.getName().getString() + "." + pkCol.getName().getString() + " may not be null");
+                    }
+                }
+            }
+        }
+        
+        List<ParseNode> valueNodes = upsert.getValues();
+        QueryPlan plan = null;
+        RowProjector rowProjectorToBe = null;
+        int nValuesToSet;
+        boolean sameTable = false;
+        boolean runOnServer = false;
+        UpsertingParallelIteratorFactory upsertParallelIteratorFactoryToBe = null;
+        final boolean isAutoCommit = connection.getAutoCommit();
+        if (valueNodes == null) {
+            SelectStatement select = upsert.getSelect();
+            assert(select != null);
+            select = addTenantAndViewConstants(table, select, tenantId, addViewColumns);
+            TableRef selectTableRef = FromCompiler.getResolver(select, connection).getTables().get(0);
+            sameTable = tableRef.equals(selectTableRef);
+            /* We can run the upsert in a coprocessor if:
+             * 1) the into table matches from table
+             * 2) the select query isn't doing aggregation
+             * 3) autoCommit is on
+             * 4) the table is not immutable, as the client is the one that figures out the additional
+             *    puts for index tables.
+             * 5) no limit clause
+             * Otherwise, run the query to pull the data from the server
+             * and populate the MutationState (upto a limit).
+            */            
+            runOnServer = sameTable && isAutoCommit && !table.isImmutableRows() && !select.isAggregate() && !select.isDistinct() && select.getLimit() == null && table.getBucketNum() == null;
+            ParallelIteratorFactory parallelIteratorFactory;
+            // TODO: once MutationState is thread safe, then when auto commit is off, we can still run in parallel
+            if (select.isAggregate() || select.isDistinct() || select.getLimit() != null) {
+                parallelIteratorFactory = null;
+            } else {
+                // We can pipeline the upsert select instead of spooling everything to disk first,
+                // if we don't have any post processing that's required.
+                parallelIteratorFactory = upsertParallelIteratorFactoryToBe = new UpsertingParallelIteratorFactory(connection, tableRef);
+            }
+            // If we may be able to run on the server, add a hint that favors using the data table
+            // if all else is equal.
+            // TODO: it'd be nice if we could figure out in advance if the PK is potentially changing,
+            // as this would disallow running on the server. We currently use the row projector we
+            // get back to figure this out.
+            HintNode hint = upsert.getHint();
+            if (!upsert.getHint().hasHint(Hint.USE_INDEX_OVER_DATA_TABLE)) {
+                hint = HintNode.create(hint, Hint.USE_DATA_OVER_INDEX_TABLE);
+            }
+            select = SelectStatement.create(select, hint);
+            // Pass scan through if same table in upsert and select so that projection is computed correctly
+            // Use optimizer to choose the best plan 
+            plan = new QueryOptimizer(services).optimize(select, statement, targetColumns, parallelIteratorFactory);
+            runOnServer &= plan.getTableRef().equals(tableRef);
+            rowProjectorToBe = plan.getProjector();
+            nValuesToSet = rowProjectorToBe.getColumnCount();
+            // Cannot auto commit if doing aggregation or topN or salted
+            // Salted causes problems because the row may end up living on a different region
+        } else {
+            nValuesToSet = valueNodes.size() + addViewColumns.size() + (isTenantSpecific ? 1 : 0);
+        }
+        final RowProjector projector = rowProjectorToBe;
+        final UpsertingParallelIteratorFactory upsertParallelIteratorFactory = upsertParallelIteratorFactoryToBe;
+        final QueryPlan queryPlan = plan;
+        // Resize down to allow a subset of columns to be specifiable
+        if (columnNodes.isEmpty() && columnIndexesToBe.length >= nValuesToSet) {
+            nColumnsToSet = nValuesToSet;
+            columnIndexesToBe = Arrays.copyOf(columnIndexesToBe, nValuesToSet);
+            pkSlotIndexesToBe = Arrays.copyOf(pkSlotIndexesToBe, nValuesToSet);
+        }
+        
+        if (nValuesToSet != nColumnsToSet) {
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode.UPSERT_COLUMN_NUMBERS_MISMATCH)
+                .setMessage("Numbers of columns: " + nColumnsToSet + ". Number of values: " + nValuesToSet)
+                .build().buildException();
+        }
+        
+        final int[] columnIndexes = columnIndexesToBe;
+        final int[] pkSlotIndexes = pkSlotIndexesToBe;
+        
+        // TODO: break this up into multiple functions
+        ////////////////////////////////////////////////////////////////////
+        // UPSERT SELECT
+        /////////////////////////////////////////////////////////////////////
+        if (valueNodes == null) {
+            // Before we re-order, check that for updatable view columns
+            // the projected expression either matches the column name or
+            // is a constant with the same required value.
+            throwIfNotUpdatable(tableRef, overlapViewColumns, targetColumns, projector, sameTable);
+            
+            ////////////////////////////////////////////////////////////////////
+            // UPSERT SELECT run server-side (maybe)
+            /////////////////////////////////////////////////////////////////////
+            if (runOnServer) {
+                // At most this array will grow bigger by the number of PK columns
+                int[] allColumnsIndexes = Arrays.copyOf(columnIndexes, columnIndexes.length + nValuesToSet);
+                int[] reverseColumnIndexes = new int[table.getColumns().size()];
+                List<Expression> projectedExpressions = Lists.newArrayListWithExpectedSize(reverseColumnIndexes.length);
+                Arrays.fill(reverseColumnIndexes, -1);
+                for (int i =0; i < nValuesToSet; i++) {
+                    projectedExpressions.add(projector.getColumnProjector(i).getExpression());
+                    reverseColumnIndexes[columnIndexes[i]] = i;
+                }
+                /*
+                 * Order projected columns and projected expressions with PK columns
+                 * leading order by slot position
+                 */
+                int offset = table.getBucketNum() == null ? 0 : 1;
+                for (int i = 0; i < table.getPKColumns().size() - offset; i++) {
+                    PColumn column = table.getPKColumns().get(i + offset);
+                    int pos = reverseColumnIndexes[column.getPosition()];
+                    if (pos == -1) {
+                        // Last PK column may be fixed width and nullable
+                        // We don't want to insert a null expression b/c
+                        // it's not valid to set a fixed width type to null.
+                        if (column.getDataType().isFixedWidth()) {
+                            continue;
+                        }
+                        // Add literal null for missing PK columns
+                        pos = projectedExpressions.size();
+                        Expression literalNull = LiteralExpression.newConstant(null, column.getDataType(), true);
+                        projectedExpressions.add(literalNull);
+                        allColumnsIndexes[pos] = column.getPosition();
+                    } 
+                    // Swap select expression at pos with i
+                    Collections.swap(projectedExpressions, i, pos);
+                    // Swap column indexes and reverse column indexes too
+                    int tempPos = allColumnsIndexes[i];
+                    allColumnsIndexes[i] = allColumnsIndexes[pos];
+                    allColumnsIndexes[pos] = tempPos;
+                    reverseColumnIndexes[tempPos] = reverseColumnIndexes[i];
+                    reverseColumnIndexes[i] = i;
+                }
+                // If any pk slots are changing, be conservative and don't run this server side.
+                // If the row ends up living in a different region, we'll get an error otherwise.
+                for (int i = 0; i < table.getPKColumns().size(); i++) {
+                    PColumn column = table.getPKColumns().get(i);
+                    Expression source = projectedExpressions.get(i);
+                    if (source == null || !source.equals(new ColumnRef(tableRef, column.getPosition()).newColumnExpression())) {
+                        // TODO: we could check the region boundaries to see if the pk will still be in it.
+                        runOnServer = false; // bail on running server side, since PK may be changing
+                        break;
+                    }
+                }
+                
+                ////////////////////////////////////////////////////////////////////
+                // UPSERT SELECT run server-side
+                /////////////////////////////////////////////////////////////////////
+                if (runOnServer) {
+                    // Iterate through columns being projected
+                    List<PColumn> projectedColumns = Lists.newArrayListWithExpectedSize(projectedExpressions.size());
+                    for (int i = 0; i < projectedExpressions.size(); i++) {
+                        // Must make new column if position has changed
+                        PColumn column = allColumns.get(allColumnsIndexes[i]);
+                        projectedColumns.add(column.getPosition() == i ? column : new PColumnImpl(column, i));
+                    }
+                    // Build table from projectedColumns
+                    PTable projectedTable = PTableImpl.makePTable(table, projectedColumns);
+                    
+                    SelectStatement select = SelectStatement.create(SelectStatement.COUNT_ONE, upsert.getHint());
+                    final RowProjector aggProjector = ProjectionCompiler.compile(queryPlan.getContext(), select, GroupBy.EMPTY_GROUP_BY);
+                    /*
+                     * Transfer over PTable representing subset of columns selected, but all PK columns.
+                     * Move columns setting PK first in pkSlot order, adding LiteralExpression of null for any missing ones.
+                     * Transfer over List<Expression> for projection.
+                     * In region scan, evaluate expressions in order, collecting first n columns for PK and collection non PK in mutation Map
+                     * Create the PRow and get the mutations, adding them to the batch
+                     */
+                    final StatementContext context = queryPlan.getContext();
+                    final Scan scan = context.getScan();
+                    scan.setAttribute(UngroupedAggregateRegionObserver.UPSERT_SELECT_TABLE, UngroupedAggregateRegionObserver.serialize(projectedTable));
+                    scan.setAttribute(UngroupedAggregateRegionObserver.UPSERT_SELECT_EXPRS, UngroupedAggregateRegionObserver.serialize(projectedExpressions));
+                    // Ignore order by - it has no impact
+                    final QueryPlan aggPlan = new AggregatePlan(context, select, tableRef, aggProjector, null, OrderBy.EMPTY_ORDER_BY, null, GroupBy.EMPTY_GROUP_BY, null);
+                    return new MutationPlan() {
+    
+                        @Override
+                        public PhoenixConnection getConnection() {
+                            return connection;
+                        }
+    
+                        @Override
+                        public ParameterMetaData getParameterMetaData() {
+                            return queryPlan.getContext().getBindManager().getParameterMetaData();
+                        }
+    
+                        @Override
+                        public MutationState execute() throws SQLException {
+                            ImmutableBytesWritable ptr = context.getTempPtr();
+                            tableRef.getTable().getIndexMaintainers(ptr);
+                            ServerCache cache = null;
+                            try {
+                                if (ptr.getLength() > 0) {
+                                    IndexMetaDataCacheClient client = new IndexMetaDataCacheClient(connection, tableRef);
+                                    cache = client.addIndexMetadataCache(context.getScanRanges(), ptr);
+                                    byte[] uuidValue = cache.getId();
+                                    scan.setAttribute(PhoenixIndexCodec.INDEX_UUID, uuidValue);
+                                }
+                                ResultIterator iterator = aggPlan.iterator();
+                                try {
+                                    Tuple row = iterator.next();
+                                    final long mutationCount = (Long)aggProjector.getColumnProjector(0).getValue(row, PDataType.LONG, ptr);
+                                    return new MutationState(maxSize, connection) {
+                                        @Override
+                                        public long getUpdateCount() {
+                                            return mutationCount;
+                                        }
+                                    };
+                                } finally {
+                                    iterator.close();
+                                }
+                            } finally {
+                                if (cache != null) {
+                                    cache.close();
+                                }
+                            }
+                        }
+    
+                        @Override
+                        public ExplainPlan getExplainPlan() throws SQLException {
+                            List<String> queryPlanSteps =  aggPlan.getExplainPlan().getPlanSteps();
+                            List<String> planSteps = Lists.newArrayListWithExpectedSize(queryPlanSteps.size()+1);
+                            planSteps.add("UPSERT ROWS");
+                            planSteps.addAll(queryPlanSteps);
+                            return new ExplainPlan(planSteps);
+                        }
+                    };
+                }
+            }
+
+            ////////////////////////////////////////////////////////////////////
+            // UPSERT SELECT run client-side
+            /////////////////////////////////////////////////////////////////////
+            return new MutationPlan() {
+
+                @Override
+                public PhoenixConnection getConnection() {
+                    return connection;
+                }
+                
+                @Override
+                public ParameterMetaData getParameterMetaData() {
+                    return queryPlan.getContext().getBindManager().getParameterMetaData();
+                }
+
+                @Override
+                public MutationState execute() throws SQLException {
+                    ResultIterator iterator = queryPlan.iterator();
+                    if (upsertParallelIteratorFactory == null) {
+                        return upsertSelect(statement, tableRef, projector, iterator, columnIndexes, pkSlotIndexes);
+                    }
+                    upsertParallelIteratorFactory.setRowProjector(projector);
+                    upsertParallelIteratorFactory.setColumnIndexes(columnIndexes);
+                    upsertParallelIteratorFactory.setPkSlotIndexes(pkSlotIndexes);
+                    Tuple tuple;
+                    long totalRowCount = 0;
+                    while ((tuple=iterator.next()) != null) {// Runs query
+                        KeyValue kv = tuple.getValue(0);
+                        totalRowCount += PDataType.LONG.getCodec().decodeLong(kv.getBuffer(), kv.getValueOffset(), null);
+                    }
+                    // Return total number of rows that have been updated. In the case of auto commit being off
+                    // the mutations will all be in the mutation state of the current connection.
+                    return new MutationState(maxSize, statement.getConnection(), totalRowCount);
+                }
+
+                @Override
+                public ExplainPlan getExplainPlan() throws SQLException {
+                    List<String> queryPlanSteps =  queryPlan.getExplainPlan().getPlanSteps();
+                    List<String> planSteps = Lists.newArrayListWithExpectedSize(queryPlanSteps.size()+1);
+                    planSteps.add("UPSERT SELECT");
+                    planSteps.addAll(queryPlanSteps);
+                    return new ExplainPlan(planSteps);
+                }
+                
+            };
+        }
+
+            
+        ////////////////////////////////////////////////////////////////////
+        // UPSERT VALUES
+        /////////////////////////////////////////////////////////////////////
+        int nodeIndex = 0;
+        // Allocate array based on size of all columns in table,
+        // since some values may not be set (if they're nullable).
+        final StatementContext context = new StatementContext(statement, resolver, statement.getParameters(), new Scan());
+        ImmutableBytesWritable ptr = context.getTempPtr();
+        UpsertValuesCompiler expressionBuilder = new UpsertValuesCompiler(context);
+        List<Expression> constantExpressions = Lists.newArrayListWithExpectedSize(valueNodes.size());
+        // First build all the expressions, as with sequences we want to collect them all first
+        // and initialize them in one batch
+        for (ParseNode valueNode : valueNodes) {
+            if (!valueNode.isStateless()) {
+                throw new SQLExceptionInfo.Builder(SQLExceptionCode.VALUE_IN_UPSERT_NOT_CONSTANT).build().buildException();
+            }
+            PColumn column = allColumns.get(columnIndexes[nodeIndex]);
+            expressionBuilder.setColumn(column);
+            constantExpressions.add(valueNode.accept(expressionBuilder));
+            nodeIndex++;
+        }
+        final SequenceManager sequenceManager = context.getSequenceManager();
+        sequenceManager.initSequences();
+        // Next evaluate all the expressions
+        nodeIndex = 0;
+        final byte[][] values = new byte[nValuesToSet][];
+        for (Expression constantExpression : constantExpressions) {
+            PColumn column = allColumns.get(columnIndexes[nodeIndex]);
+            constantExpression.evaluate(null, ptr);
+            Object value = null;
+            byte[] byteValue = ByteUtil.copyKeyBytesIfNecessary(ptr);
+            if (constantExpression.getDataType() != null) {
+                // If ColumnModifier from expression in SELECT doesn't match the
+                // column being projected into then invert the bits.
+                if (constantExpression.getColumnModifier() != column.getColumnModifier()) {
+                    byte[] tempByteValue = Arrays.copyOf(byteValue, byteValue.length);
+                    byteValue = ColumnModifier.SORT_DESC.apply(byteValue, 0, tempByteValue, 0, byteValue.length);
+                }
+                value = constantExpression.getDataType().toObject(byteValue);
+                if (!constantExpression.getDataType().isCoercibleTo(column.getDataType(), value)) { 
+                    throw TypeMismatchException.newException(
+                        constantExpression.getDataType(), column.getDataType(), "expression: "
+                                + constantExpression.toString() + " in column " + column);
+                }
+                if (!column.getDataType().isSizeCompatible(constantExpression.getDataType(),
+                        value, byteValue, constantExpression.getMaxLength(),
+                        column.getMaxLength(), constantExpression.getScale(), column.getScale())) { 
+                    throw new SQLExceptionInfo.Builder(
+                        SQLExceptionCode.DATA_INCOMPATIBLE_WITH_TYPE).setColumnName(column.getName().getString())
+                        .setMessage("value=" + constantExpression.toString()).build().buildException();
+                }
+            }
+            byteValue = column.getDataType().coerceBytes(byteValue, value,
+                    constantExpression.getDataType(), constantExpression.getMaxLength(), constantExpression.getScale(),
+                    column.getMaxLength(), column.getScale());
+            byte[] viewValue = overlapViewColumns.get(column);
+            if (viewValue != null && Bytes.compareTo(byteValue, viewValue) != 0) {
+                throw new SQLExceptionInfo.Builder(
+                        SQLExceptionCode.CANNOT_UPDATE_VIEW_COLUMN)
+                        .setColumnName(column.getName().getString())
+                        .setMessage("value=" + constantExpression.toString()).build().buildException();
+            }
+            values[nodeIndex] = byteValue;
+            nodeIndex++;
+        }
+        // Add columns based on view
+        for (byte[] value : addViewColumns.values()) {
+            values[nodeIndex++] = value;
+        }
+        if (isTenantSpecific) {
+            values[nodeIndex++] = connection.getTenantId().getBytes();
+        }
+        return new MutationPlan() {
+
+            @Override
+            public PhoenixConnection getConnection() {
+                return connection;
+            }
+
+            @Override
+            public ParameterMetaData getParameterMetaData() {
+                return context.getBindManager().getParameterMetaData();
+            }
+
+            @Override
+            public MutationState execute() { // TODO: add throws SQLException
+                try {
+                    sequenceManager.incrementSequenceValues();
+                } catch (SQLException e) {
+                    throw new RuntimeException(e); // Will get unwrapped
+                }
+                Map<ImmutableBytesPtr, Map<PColumn, byte[]>> mutation = Maps.newHashMapWithExpectedSize(1);
+                setValues(values, pkSlotIndexes, columnIndexes, tableRef.getTable(), mutation);
+                return new MutationState(tableRef, mutation, 0, maxSize, connection);
+            }
+
+            @Override
+            public ExplainPlan getExplainPlan() throws SQLException {
+                List<String> planSteps = Lists.newArrayListWithExpectedSize(2);
+                if (context.getSequenceManager().getSequenceCount() > 0) {
+                    planSteps.add("CLIENT RESERVE " + context.getSequenceManager().getSequenceCount() + " SEQUENCES");
+                }
+                planSteps.add("PUT SINGLE ROW");
+                return new ExplainPlan(planSteps);
+            }
+
+        };
+    }
+    
+    private static final class UpsertValuesCompiler extends ExpressionCompiler {
+        private PColumn column;
+        
+        private UpsertValuesCompiler(StatementContext context) {
+            super(context);
+        }
+
+        public void setColumn(PColumn column) {
+            this.column = column;
+        }
+        
+        @Override
+        public Expression visit(BindParseNode node) throws SQLException {
+            if (isTopLevel()) {
+                context.getBindManager().addParamMetaData(node, column);
+                Object value = context.getBindManager().getBindValue(node);
+                return LiteralExpression.newConstant(value, column.getDataType(), column.getColumnModifier(), true);
+            }
+            return super.visit(node);
+        }    
+        
+        @Override
+        public Expression visit(LiteralParseNode node) throws SQLException {
+            if (isTopLevel()) {
+                return LiteralExpression.newConstant(node.getValue(), column.getDataType(), column.getColumnModifier(), true);
+            }
+            return super.visit(node);
+        }
+        
+        
+        @Override
+        public Expression visit(SequenceValueParseNode node) throws SQLException {
+            return context.getSequenceManager().newSequenceReference(node);
+        }
+    }
+    
+
+    // ExpressionCompiler needs a context
+    private static class ViewValuesMapBuilder extends ExpressionCompiler {
+        private ColumnRef columnRef;
+        private Map<ColumnRef, byte[]> viewColumns = Maps.newHashMapWithExpectedSize(5);
+
+        private ViewValuesMapBuilder(StatementContext context) {
+            super(context);
+        }
+        
+        public Map<ColumnRef, byte[]> getViewColumns() {
+            return viewColumns;
+        }
+
+        @Override
+        protected ColumnRef resolveColumn(ColumnParseNode node) throws SQLException {
+            return columnRef = super.resolveColumn(node);
+        }
+
+        @Override
+        public Expression visitLeave(IsNullParseNode node, List<Expression> children) throws SQLException {
+            viewColumns.put(columnRef, ByteUtil.EMPTY_BYTE_ARRAY);
+            return super.visitLeave(node, children);
+        }
+        
+        @Override
+        public Expression visitLeave(ComparisonParseNode node, List<Expression> children) throws SQLException {
+            Expression literal = children.get(1);
+            ImmutableBytesWritable ptr = context.getTempPtr();
+            literal.evaluate(null, ptr);
+            PColumn column = columnRef.getColumn();
+            column.getDataType().coerceBytes(ptr, literal.getDataType(), literal.getColumnModifier(), column.getColumnModifier());
+            viewColumns.put(columnRef, ByteUtil.copyKeyBytesIfNecessary(ptr));
+            return super.visitLeave(node, children);
+        }
+    }
+    
+    private static SelectStatement addTenantAndViewConstants(PTable table, SelectStatement select, String tenantId, Map<ColumnRef, byte[]> addViewColumns) {
+        if (tenantId == null && addViewColumns.isEmpty()) {
+            return select;
+        }
+        List<AliasedNode> selectNodes = newArrayListWithCapacity(select.getSelect().size() + 1 + addViewColumns.size());
+        selectNodes.addAll(select.getSelect());
+        for (Map.Entry<ColumnRef, byte[]> entry : addViewColumns.entrySet()) {
+            ColumnRef ref = entry.getKey();
+            PColumn column = ref.getColumn();
+            byte[] byteValue = entry.getValue();
+            Object value = column.getDataType().toObject(byteValue);
+            selectNodes.add(new AliasedNode(null, new LiteralParseNode(value)));
+        }
+        if (table.isMultiTenant() && tenantId != null) {
+            selectNodes.add(new AliasedNode(null, new LiteralParseNode(tenantId)));
+        }
+        
+        return SelectStatement.create(select, selectNodes);
+    }
+    
+    /**
+     * Check that none of no columns in our updatable VIEW are changing values.
+     * @param tableRef
+     * @param overlapViewColumns
+     * @param targetColumns
+     * @param projector
+     * @throws SQLException
+     */
+    private static void throwIfNotUpdatable(TableRef tableRef, Map<PColumn, byte[]> overlapViewColumns,
+            List<PColumn> targetColumns, RowProjector projector, boolean sameTable) throws SQLException {
+        PTable table = tableRef.getTable();
+        if (table.getViewType() == ViewType.UPDATABLE && !overlapViewColumns.isEmpty()) {
+            ImmutableBytesWritable ptr = new ImmutableBytesWritable();
+            for (int i = 0; i < targetColumns.size(); i++) {
+                // Must make new column if position has changed
+                PColumn targetColumn = targetColumns.get(i);
+                byte[] value = overlapViewColumns.get(targetColumn);
+                if (value != null) {
+                    Expression source = projector.getColumnProjector(i).getExpression();
+                    if (source == null) { // FIXME: is this possible?
+                    } else if (source.isStateless()) {
+                        source.evaluate(null, ptr);
+                        if (Bytes.compareTo(ptr.get(), ptr.getOffset(), ptr.getLength(), value, 0, value.length) == 0) {
+                            continue;
+                        }
+                    // TODO: we had a ColumnRef already in our map before
+                    } else if (sameTable && source.equals(new ColumnRef(tableRef, targetColumn.getPosition()).newColumnExpression())) {
+                        continue;
+                    }
+                    // TODO: one other check we could do is if the source is an updatable VIEW,
+                    // check if the source column is a VIEW column with the same constant value
+                    // as expected.
+                    throw new SQLExceptionInfo.Builder(
+                            SQLExceptionCode.CANNOT_UPDATE_VIEW_COLUMN)
+                            .setColumnName(targetColumn.getName().getString())
+                            .build().buildException();
+                }
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..6f76f0a
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereCompiler.java
@@ -0,0 +1,214 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.compile;
+
+import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
+import java.util.Iterator;
+import java.util.Set;
+
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.filter.Filter;
+
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Sets;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.KeyValueColumnExpression;
+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.RowKeyComparisonFilter;
+import org.apache.phoenix.filter.SingleCFCQKeyValueComparisonFilter;
+import org.apache.phoenix.filter.SingleCQKeyValueComparisonFilter;
+import org.apache.phoenix.parse.ColumnParseNode;
+import org.apache.phoenix.parse.FilterableStatement;
+import org.apache.phoenix.parse.HintNode.Hint;
+import org.apache.phoenix.parse.ParseNode;
+import org.apache.phoenix.parse.ParseNodeFactory;
+import org.apache.phoenix.schema.AmbiguousColumnException;
+import org.apache.phoenix.schema.ColumnNotFoundException;
+import org.apache.phoenix.schema.ColumnRef;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.schema.TypeMismatchException;
+import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.ScanUtil;
+import org.apache.phoenix.util.SchemaUtil;
+
+
+/**
+ *
+ * Class to build the filter of a scan
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class WhereCompiler {
+    protected static final ParseNodeFactory NODE_FACTORY = new ParseNodeFactory();
+
+    private WhereCompiler() {
+    }
+
+    /**
+     * Pushes where clause filter expressions into scan by building and setting a filter.
+     * @param context the shared context during query compilation
+     * @param statement TODO
+     * @throws SQLException if mismatched types are found, bind value do not match binds,
+     * or invalid function arguments are encountered.
+     * @throws SQLFeatureNotSupportedException if an unsupported expression is encountered.
+     * @throws ColumnNotFoundException if column name could not be resolved
+     * @throws AmbiguousColumnException if an unaliased column name is ambiguous across multiple tables
+     */
+    public static Expression compile(StatementContext context, FilterableStatement statement) throws SQLException {
+        return compileWhereClause(context, statement, Sets.<Expression>newHashSet());
+    }
+
+    /**
+     * Used for testing to get access to the expressions that were used to form the start/stop key of the scan
+     * @param statement TODO
+     */
+    public static Expression compileWhereClause(StatementContext context, FilterableStatement statement,
+            Set<Expression> extractedNodes) throws SQLException {
+        WhereExpressionCompiler whereCompiler = new WhereExpressionCompiler(context);
+        ParseNode where = statement.getWhere();
+        Expression expression = where == null ? LiteralExpression.newConstant(true,PDataType.BOOLEAN,true) : where.accept(whereCompiler);
+        if (whereCompiler.isAggregate()) {
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode.AGGREGATE_IN_WHERE).build().buildException();
+        }
+        if (expression.getDataType() != PDataType.BOOLEAN) {
+            throw TypeMismatchException.newException(PDataType.BOOLEAN, expression.getDataType(), expression.toString());
+        }
+        
+        expression = WhereOptimizer.pushKeyExpressionsToScan(context, statement, expression, extractedNodes);
+        setScanFilter(context, statement, expression, whereCompiler.disambiguateWithFamily);
+
+        return expression;
+    }
+    
+    private static class WhereExpressionCompiler extends ExpressionCompiler {
+        private boolean disambiguateWithFamily;
+
+        WhereExpressionCompiler(StatementContext context) {
+            super(context);
+        }
+
+        @Override
+        protected ColumnRef resolveColumn(ColumnParseNode node) throws SQLException {
+            ColumnRef ref = super.resolveColumn(node);
+            PTable table = ref.getTable();
+            // Track if we need to compare KeyValue during filter evaluation
+            // using column family. If the column qualifier is enough, we
+            // just use that.
+            try {
+                if (!SchemaUtil.isPKColumn(ref.getColumn())) {
+                    table.getColumn(ref.getColumn().getName().getString());
+                }
+            } catch (AmbiguousColumnException e) {
+                disambiguateWithFamily = true;
+            }
+            return ref;
+         }
+    }
+
+    private static final class Counter {
+        public enum Count {NONE, SINGLE, MULTIPLE};
+        private Count count = Count.NONE;
+        private KeyValueColumnExpression column;
+
+        public void increment(KeyValueColumnExpression column) {
+            switch (count) {
+                case NONE:
+                    count = Count.SINGLE;
+                    this.column = column;
+                    break;
+                case SINGLE:
+                    count = column.equals(this.column) ? Count.SINGLE : Count.MULTIPLE;
+                    break;
+                case MULTIPLE:
+                    break;
+
+            }
+        }
+        public Count getCount() {
+            return count;
+        }
+    }
+
+    /**
+     * Sets the start/stop key range based on the whereClause expression.
+     * @param context the shared context during query compilation
+     * @param whereClause the final where clause expression.
+     */
+    private static void setScanFilter(StatementContext context, FilterableStatement statement, Expression whereClause, boolean disambiguateWithFamily) {
+        Filter filter = null;
+        Scan scan = context.getScan();
+        assert scan.getFilter() == null;
+
+        if (LiteralExpression.isFalse(whereClause)) {
+            context.setScanRanges(ScanRanges.NOTHING);
+        } else if (whereClause != null && !LiteralExpression.isTrue(whereClause)) {
+            final Counter counter = new Counter();
+            whereClause.accept(new KeyValueExpressionVisitor() {
+
+                @Override
+                public Iterator<Expression> defaultIterator(Expression node) {
+                    // Stop traversal once we've found multiple KeyValue columns
+                    if (counter.getCount() == Counter.Count.MULTIPLE) {
+                        return Iterators.emptyIterator();
+                    }
+                    return super.defaultIterator(node);
+                }
+
+                @Override
+                public Void visit(KeyValueColumnExpression expression) {
+                    counter.increment(expression);
+                    return null;
+                }
+            });
+            switch (counter.getCount()) {
+            case NONE:
+                PTable table = context.getResolver().getTables().get(0).getTable();
+                byte[] essentialCF = table.getType() == PTableType.VIEW 
+                        ? ByteUtil.EMPTY_BYTE_ARRAY 
+                        : SchemaUtil.getEmptyColumnFamily(table.getColumnFamilies());
+                filter = new RowKeyComparisonFilter(whereClause, essentialCF);
+                break;
+            case SINGLE:
+                filter = disambiguateWithFamily ? new SingleCFCQKeyValueComparisonFilter(whereClause) : new SingleCQKeyValueComparisonFilter(whereClause);
+                break;
+            case MULTIPLE:
+                filter = disambiguateWithFamily ? new MultiCFCQKeyValueComparisonFilter(whereClause) : new MultiCQKeyValueComparisonFilter(whereClause);
+                break;
+            }
+        }
+
+        scan.setFilter(filter);
+        ScanRanges scanRanges = context.getScanRanges();
+        boolean forcedSkipScan = statement.getHint().hasHint(Hint.SKIP_SCAN);
+        boolean forcedRangeScan = statement.getHint().hasHint(Hint.RANGE_SCAN);
+        if (forcedSkipScan || (scanRanges.useSkipScanFilter() && !forcedRangeScan)) {
+            ScanUtil.andFilterAtBeginning(scan, scanRanges.getSkipScanFilter());
+        }
+    }
+}


[49/51] [partial] Initial commit of master branch from github

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/IndexLogRollSynchronizer.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/IndexLogRollSynchronizer.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/IndexLogRollSynchronizer.java
new file mode 100644
index 0000000..96e94ca
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/IndexLogRollSynchronizer.java
@@ -0,0 +1,125 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index;
+
+import java.io.IOException;
+import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
+import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+
+/**
+ * Ensure that the log isn't rolled while we are the in middle of doing a pending index write.
+ * <p>
+ * The problem we are trying to solve is the following sequence:
+ * <ol>
+ * <li>Write to the indexed table</li>
+ * <li>Write the index-containing WALEdit</li>
+ * <li>Start writing to the index tables in the postXXX hook</li>
+ * <li>WAL gets rolled and archived</li>
+ * <li>An index update fails, in which case we should kill ourselves to get WAL replay</li>
+ * <li>Since the WAL got archived, we won't get the replay of the index writes</li>
+ * </ol>
+ * <p>
+ * The usual course of events should be:
+ * <ol>
+ * <li>In a preXXX hook,
+ * <ol>
+ * <li>Build the {@link WALEdit} + index information</li>
+ * <li>Lock the {@link IndexLogRollSynchronizer#logArchiveLock}</li>
+ * <ul>
+ * <li>This is a reentrant readlock on the WAL archiving, so we can make multiple WAL/index updates
+ * concurrently</li>
+ * </ul>
+ * </li>
+ * </ol>
+ * </li>
+ * <li>Pass that {@link WALEdit} to the WAL, ensuring its durable and replayable</li>
+ * <li>In the corresponding postXXX,
+ * <ol>
+ * <li>make the updates to the index tables</li>
+ * <li>Unlock {@link IndexLogRollSynchronizer#logArchiveLock}</li>
+ * </ol>
+ * </li> </ol>
+ * <p>
+ * <tt>this</tt> should be added as a {@link WALActionsListener} by updating
+ */
+public class IndexLogRollSynchronizer implements WALActionsListener {
+
+  private static final Log LOG = LogFactory.getLog(IndexLogRollSynchronizer.class);
+  private WriteLock logArchiveLock;
+
+  public IndexLogRollSynchronizer(WriteLock logWriteLock){
+    this.logArchiveLock = logWriteLock;
+  }
+
+
+  @Override
+  public void preLogArchive(Path oldPath, Path newPath) throws IOException {
+    //take a write lock on the index - any pending index updates will complete before we finish
+    LOG.debug("Taking INDEX_UPDATE writelock");
+    logArchiveLock.lock();
+    LOG.debug("Got the INDEX_UPDATE writelock");
+  }
+  
+  @Override
+  public void postLogArchive(Path oldPath, Path newPath) throws IOException {
+    // done archiving the logs, any WAL updates will be replayed on failure
+    LOG.debug("Releasing INDEX_UPDATE writelock");
+    logArchiveLock.unlock();
+  }
+
+  @Override
+  public void logCloseRequested() {
+    // don't care- before this is called, all the HRegions are closed, so we can't get any new
+    // requests and all pending request can finish before the WAL closes.
+  }
+
+  @Override
+  public void preLogRoll(Path oldPath, Path newPath) throws IOException {
+    // noop
+  }
+
+  @Override
+  public void postLogRoll(Path oldPath, Path newPath) throws IOException {
+    // noop
+  }
+
+  @Override
+  public void logRollRequested() {
+    // noop
+  }
+
+  @Override
+  public void visitLogEntryBeforeWrite(HRegionInfo info, HLogKey logKey, WALEdit logEdit) {
+    // noop
+  }
+
+  @Override
+  public void visitLogEntryBeforeWrite(HTableDescriptor htd, HLogKey logKey, WALEdit logEdit) {
+    // noop
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/Indexer.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/Indexer.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/Indexer.java
new file mode 100644
index 0000000..ce7ef87
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/Indexer.java
@@ -0,0 +1,706 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index;
+
+import static org.apache.hadoop.hbase.index.util.IndexManagementUtil.rethrowIndexingException;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Coprocessor;
+import org.apache.hadoop.hbase.CoprocessorEnvironment;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Durability;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.regionserver.InternalScanner;
+import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
+import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
+import org.apache.hadoop.hbase.regionserver.ScanType;
+import org.apache.hadoop.hbase.regionserver.Store;
+import org.apache.hadoop.hbase.regionserver.wal.HLog;
+import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.util.Pair;
+
+import com.google.common.collect.Multimap;
+import org.apache.hadoop.hbase.index.builder.IndexBuildManager;
+import org.apache.hadoop.hbase.index.builder.IndexBuilder;
+import org.apache.hadoop.hbase.index.builder.IndexBuildingFailureException;
+import org.apache.hadoop.hbase.index.table.HTableInterfaceReference;
+import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+import org.apache.hadoop.hbase.index.util.IndexManagementUtil;
+import org.apache.hadoop.hbase.index.wal.IndexedKeyValue;
+import org.apache.hadoop.hbase.index.write.IndexFailurePolicy;
+import org.apache.hadoop.hbase.index.write.IndexWriter;
+import org.apache.hadoop.hbase.index.write.recovery.PerRegionIndexWriteCache;
+import org.apache.hadoop.hbase.index.write.recovery.StoreFailuresInCachePolicy;
+import org.apache.hadoop.hbase.index.write.recovery.TrackingParallelWriterIndexCommitter;
+import org.apache.phoenix.util.MetaDataUtil;
+
+/**
+ * Do all the work of managing index updates from a single coprocessor. All Puts/Delets are passed
+ * to an {@link IndexBuilder} to determine the actual updates to make.
+ * <p>
+ * If the WAL is enabled, these updates are then added to the WALEdit and attempted to be written to
+ * the WAL after the WALEdit has been saved. If any of the index updates fail, this server is
+ * immediately terminated and we rely on WAL replay to attempt the index updates again (see
+ * {@link #preWALRestore(ObserverContext, HRegionInfo, HLogKey, WALEdit)}).
+ * <p>
+ * If the WAL is disabled, the updates are attempted immediately. No consistency guarantees are made
+ * if the WAL is disabled - some or none of the index updates may be successful. All updates in a
+ * single batch must have the same durability level - either everything gets written to the WAL or
+ * nothing does. Currently, we do not support mixed-durability updates within a single batch. If you
+ * want to have different durability levels, you only need to split the updates into two different
+ * batches.
+ */
+public class Indexer extends BaseRegionObserver {
+
+  private static final Log LOG = LogFactory.getLog(Indexer.class);
+
+  /** WAL on this server */
+  private HLog log;
+  protected IndexWriter writer;
+  protected IndexBuildManager builder;
+
+  /** Configuration key for the {@link IndexBuilder} to use */
+  public static final String INDEX_BUILDER_CONF_KEY = "index.builder";
+
+  // Setup out locking on the index edits/WAL so we can be sure that we don't lose a roll a WAL edit
+  // before an edit is applied to the index tables
+  private static final ReentrantReadWriteLock INDEX_READ_WRITE_LOCK = new ReentrantReadWriteLock(
+      true);
+  public static final ReadLock INDEX_UPDATE_LOCK = INDEX_READ_WRITE_LOCK.readLock();
+
+  /**
+   * Configuration key for if the indexer should check the version of HBase is running. Generally,
+   * you only want to ignore this for testing or for custom versions of HBase.
+   */
+  public static final String CHECK_VERSION_CONF_KEY = "com.saleforce.hbase.index.checkversion";
+
+  private static final String INDEX_RECOVERY_FAILURE_POLICY_KEY = "org.apache.hadoop.hbase.index.recovery.failurepolicy";
+
+  /**
+   * Marker {@link KeyValue} to indicate that we are doing a batch operation. Needed because the
+   * coprocessor framework throws away the WALEdit from the prePut/preDelete hooks when checking a
+   * batch if there were no {@link KeyValue}s attached to the {@link WALEdit}. When you get down to
+   * the preBatch hook, there won't be any WALEdits to which to add the index updates.
+   */
+  private static KeyValue BATCH_MARKER = new KeyValue();
+
+  /**
+   * cache the failed updates to the various regions. Used for making the WAL recovery mechanisms
+   * more robust in the face of recoverying index regions that were on the same server as the
+   * primary table region
+   */
+  private PerRegionIndexWriteCache failedIndexEdits = new PerRegionIndexWriteCache();
+
+  /**
+   * IndexWriter for writing the recovered index edits. Separate from the main indexer since we need
+   * different write/failure policies
+   */
+  private IndexWriter recoveryWriter;
+
+  private boolean stopped;
+  private boolean disabled;
+
+  public static final String RecoveryFailurePolicyKeyForTesting = INDEX_RECOVERY_FAILURE_POLICY_KEY;
+
+    public static final int INDEXING_SUPPORTED_MAJOR_VERSION = MetaDataUtil
+            .encodeMaxPatchVersion(0, 94);
+    public static final int INDEXING_SUPPORTED__MIN_MAJOR_VERSION = MetaDataUtil
+            .encodeVersion("0.94.0");
+    private static final int INDEX_WAL_COMPRESSION_MINIMUM_SUPPORTED_VERSION = MetaDataUtil
+            .encodeVersion("0.94.9");
+
+  @Override
+  public void start(CoprocessorEnvironment e) throws IOException {
+      try {
+        final RegionCoprocessorEnvironment env = (RegionCoprocessorEnvironment) e;
+        String serverName = env.getRegionServerServices().getServerName().getServerName();
+        if (env.getConfiguration().getBoolean(CHECK_VERSION_CONF_KEY, true)) {
+          // make sure the right version <-> combinations are allowed.
+          String errormsg = Indexer.validateVersion(env.getHBaseVersion(), env.getConfiguration());
+          if (errormsg != null) {
+            IOException ioe = new IOException(errormsg);
+            env.getRegionServerServices().abort(errormsg, ioe);
+            throw ioe;
+          }
+        }
+    
+        this.builder = new IndexBuildManager(env);
+    
+        // get a reference to the WAL
+        log = env.getRegionServerServices().getWAL();
+        // add a synchronizer so we don't archive a WAL that we need
+        log.registerWALActionsListener(new IndexLogRollSynchronizer(INDEX_READ_WRITE_LOCK.writeLock()));
+    
+        // setup the actual index writer
+        this.writer = new IndexWriter(env, serverName + "-index-writer");
+    
+        // setup the recovery writer that does retries on the failed edits
+        TrackingParallelWriterIndexCommitter recoveryCommmiter =
+            new TrackingParallelWriterIndexCommitter();
+    
+        try {
+          // get the specified failure policy. We only ever override it in tests, but we need to do it
+          // here
+          Class<? extends IndexFailurePolicy> policyClass =
+              env.getConfiguration().getClass(INDEX_RECOVERY_FAILURE_POLICY_KEY,
+                StoreFailuresInCachePolicy.class, IndexFailurePolicy.class);
+          IndexFailurePolicy policy =
+              policyClass.getConstructor(PerRegionIndexWriteCache.class).newInstance(failedIndexEdits);
+          LOG.debug("Setting up recovery writter with committer: " + recoveryCommmiter.getClass()
+              + " and failure policy: " + policy.getClass());
+          recoveryWriter =
+              new IndexWriter(recoveryCommmiter, policy, env, serverName + "-recovery-writer");
+        } catch (Exception ex) {
+          throw new IOException("Could not instantiate recovery failure policy!", ex);
+        }
+      } catch (NoSuchMethodError ex) {
+          disabled = true;
+          super.start(e);
+          LOG.error("Must be too early a version of HBase. Disabled coprocessor ", ex);
+      }
+  }
+
+  @Override
+  public void stop(CoprocessorEnvironment e) throws IOException {
+    if (this.stopped) {
+      return;
+    }
+    if (this.disabled) {
+        super.stop(e);
+        return;
+      }
+    this.stopped = true;
+    String msg = "Indexer is being stopped";
+    this.builder.stop(msg);
+    this.writer.stop(msg);
+    this.recoveryWriter.stop(msg);
+  }
+
+  @Override
+  public void prePut(final ObserverContext<RegionCoprocessorEnvironment> c, final Put put,
+      final WALEdit edit, final boolean writeToWAL) throws IOException {
+      if (this.disabled) {
+          super.prePut(c, put, edit, writeToWAL);
+          return;
+        }
+    // just have to add a batch marker to the WALEdit so we get the edit again in the batch
+    // processing step. We let it throw an exception here because something terrible has happened.
+    edit.add(BATCH_MARKER);
+  }
+
+  @Override
+  public void preDelete(ObserverContext<RegionCoprocessorEnvironment> e, Delete delete,
+      WALEdit edit, boolean writeToWAL) throws IOException {
+      if (this.disabled) {
+          super.preDelete(e, delete, edit, writeToWAL);
+          return;
+        }
+    try {
+      preDeleteWithExceptions(e, delete, edit, writeToWAL);
+      return;
+    } catch (Throwable t) {
+      rethrowIndexingException(t);
+    }
+    throw new RuntimeException(
+        "Somehow didn't return an index update but also didn't propagate the failure to the client!");
+  }
+
+  public void preDeleteWithExceptions(ObserverContext<RegionCoprocessorEnvironment> e,
+      Delete delete, WALEdit edit, boolean writeToWAL) throws Exception {
+    // if we are making the update as part of a batch, we need to add in a batch marker so the WAL
+    // is retained
+    if (this.builder.getBatchId(delete) != null) {
+      edit.add(BATCH_MARKER);
+      return;
+    }
+
+    // get the mapping for index column -> target index table
+    Collection<Pair<Mutation, byte[]>> indexUpdates = this.builder.getIndexUpdate(delete);
+
+    if (doPre(indexUpdates, edit, writeToWAL)) {
+      takeUpdateLock("delete");
+    }
+  }
+
+  @Override
+  public void preBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
+      MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp) throws IOException {
+      if (this.disabled) {
+          super.preBatchMutate(c, miniBatchOp);
+          return;
+        }
+    try {
+      preBatchMutateWithExceptions(c, miniBatchOp);
+      return;
+    } catch (Throwable t) {
+      rethrowIndexingException(t);
+    }
+    throw new RuntimeException(
+        "Somehow didn't return an index update but also didn't propagate the failure to the client!");
+  }
+
+  @SuppressWarnings("deprecation")
+  public void preBatchMutateWithExceptions(ObserverContext<RegionCoprocessorEnvironment> c,
+      MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp) throws Throwable {
+
+    // first group all the updates for a single row into a single update to be processed
+    Map<ImmutableBytesPtr, MultiMutation> mutations =
+        new HashMap<ImmutableBytesPtr, MultiMutation>();
+    boolean durable = false;
+    for (int i = 0; i < miniBatchOp.size(); i++) {
+      // remove the batch keyvalue marker - its added for all puts
+      WALEdit edit = miniBatchOp.getWalEdit(i);
+      // we don't have a WALEdit for immutable index cases, which still see this path
+      // we could check is indexing is enable for the mutation in prePut and then just skip this
+      // after checking here, but this saves us the checking again.
+      if (edit != null) {
+        KeyValue kv = edit.getKeyValues().remove(0);
+        assert kv == BATCH_MARKER : "Expected batch marker from the WALEdit, but got: " + kv;
+      }
+      Pair<Mutation, Integer> op = miniBatchOp.getOperation(i);
+      Mutation m = op.getFirst();
+      // skip this mutation if we aren't enabling indexing
+      // unfortunately, we really should ask if the raw mutation (rather than the combined mutation)
+      // should be indexed, which means we need to expose another method on the builder. Such is the
+      // way optimization go though.
+      if (!this.builder.isEnabled(m)) {
+        continue;
+      }
+      
+      // figure out if this is batch is durable or not
+      if(!durable){
+        durable = m.getDurability() != Durability.SKIP_WAL;
+      }
+
+      // add the mutation to the batch set
+      ImmutableBytesPtr row = new ImmutableBytesPtr(m.getRow());
+      MultiMutation stored = mutations.get(row);
+      // we haven't seen this row before, so add it
+      if (stored == null) {
+        stored = new MultiMutation(row, m.getWriteToWAL());
+        mutations.put(row, stored);
+      }
+      stored.addAll(m);
+    }
+    
+    // early exit if it turns out we don't have any edits
+    if (mutations.entrySet().size() == 0) {
+      return;
+    }
+
+    // dump all the index updates into a single WAL. They will get combined in the end anyways, so
+    // don't worry which one we get
+    WALEdit edit = miniBatchOp.getWalEdit(0);
+
+    // get the index updates for all elements in this batch
+    Collection<Pair<Mutation, byte[]>> indexUpdates =
+        this.builder.getIndexUpdate(miniBatchOp, mutations.values());
+    // write them
+    if (doPre(indexUpdates, edit, durable)) {
+      takeUpdateLock("batch mutation");
+    }
+  }
+
+  private void takeUpdateLock(String opDesc) throws IndexBuildingFailureException {
+    boolean interrupted = false;
+    // lock the log, so we are sure that index write gets atomically committed
+    LOG.debug("Taking INDEX_UPDATE readlock for " + opDesc);
+    // wait for the update lock
+    while (!this.stopped) {
+      try {
+        INDEX_UPDATE_LOCK.lockInterruptibly();
+        LOG.debug("Got the INDEX_UPDATE readlock for " + opDesc);
+        // unlock the lock so the server can shutdown, if we find that we have stopped since getting
+        // the lock
+        if (this.stopped) {
+          INDEX_UPDATE_LOCK.unlock();
+          throw new IndexBuildingFailureException(
+              "Found server stop after obtaining the update lock, killing update attempt");
+        }
+        break;
+      } catch (InterruptedException e) {
+        LOG.info("Interrupted while waiting for update lock. Ignoring unless stopped");
+        interrupted = true;
+      }
+    }
+    if (interrupted) {
+      Thread.currentThread().interrupt();
+    }
+  }
+
+  private class MultiMutation extends Mutation {
+
+    private ImmutableBytesPtr rowKey;
+
+    public MultiMutation(ImmutableBytesPtr rowkey, boolean writeToWal) {
+      this.rowKey = rowkey;
+      this.writeToWAL = writeToWal;
+    }
+
+    /**
+     * @param stored
+     */
+    @SuppressWarnings("deprecation")
+    public void addAll(Mutation stored) {
+      // add all the kvs
+      for (Entry<byte[], List<KeyValue>> kvs : stored.getFamilyMap().entrySet()) {
+        byte[] family = kvs.getKey();
+        List<KeyValue> list = getKeyValueList(family, kvs.getValue().size());
+        list.addAll(kvs.getValue());
+        familyMap.put(family, list);
+      }
+
+      // add all the attributes, not overriding already stored ones
+      for (Entry<String, byte[]> attrib : stored.getAttributesMap().entrySet()) {
+        if (this.getAttribute(attrib.getKey()) == null) {
+          this.setAttribute(attrib.getKey(), attrib.getValue());
+        }
+      }
+      if (stored.getWriteToWAL()) {
+        this.writeToWAL = true;
+      }
+    }
+
+    private List<KeyValue> getKeyValueList(byte[] family, int hint) {
+      List<KeyValue> list = familyMap.get(family);
+      if (list == null) {
+        list = new ArrayList<KeyValue>(hint);
+      }
+      return list;
+    }
+
+    @Override
+    public byte[] getRow(){
+      return this.rowKey.copyBytesIfNecessary();
+    }
+
+    @Override
+    public int hashCode() {
+      return this.rowKey.hashCode();
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      return o == null ? false : o.hashCode() == this.hashCode();
+    }
+
+    @Override
+    public void readFields(DataInput arg0) throws IOException {
+      throw new UnsupportedOperationException("MultiMutations cannot be read/written");
+    }
+
+    @Override
+    public void write(DataOutput arg0) throws IOException {
+      throw new UnsupportedOperationException("MultiMutations cannot be read/written");
+    }
+  }
+
+  /**
+   * Add the index updates to the WAL, or write to the index table, if the WAL has been disabled
+   * @return <tt>true</tt> if the WAL has been updated.
+   * @throws IOException
+   */
+  private boolean doPre(Collection<Pair<Mutation, byte[]>> indexUpdates, final WALEdit edit,
+      final boolean writeToWAL) throws IOException {
+    // no index updates, so we are done
+    if (indexUpdates == null || indexUpdates.size() == 0) {
+      return false;
+    }
+
+    // if writing to wal is disabled, we never see the WALEdit updates down the way, so do the index
+    // update right away
+    if (!writeToWAL) {
+      try {
+        this.writer.write(indexUpdates);
+        return false;
+      } catch (Throwable e) {
+        LOG.error("Failed to update index with entries:" + indexUpdates, e);
+        IndexManagementUtil.rethrowIndexingException(e);
+      }
+    }
+
+    // we have all the WAL durability, so we just update the WAL entry and move on
+    for (Pair<Mutation, byte[]> entry : indexUpdates) {
+      edit.add(new IndexedKeyValue(entry.getSecond(), entry.getFirst()));
+    }
+
+    return true;
+  }
+
+  @Override
+  public void postPut(ObserverContext<RegionCoprocessorEnvironment> e, Put put, WALEdit edit,
+      boolean writeToWAL) throws IOException {
+      if (this.disabled) {
+          super.postPut(e, put, edit, writeToWAL);
+          return;
+        }
+    doPost(edit, put, writeToWAL);
+  }
+
+  @Override
+  public void postDelete(ObserverContext<RegionCoprocessorEnvironment> e, Delete delete,
+      WALEdit edit, boolean writeToWAL) throws IOException {
+      if (this.disabled) {
+          super.postDelete(e, delete, edit, writeToWAL);
+          return;
+        }
+    doPost(edit,delete, writeToWAL);
+  }
+
+  @Override
+  public void postBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
+      MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp) throws IOException {
+      if (this.disabled) {
+          super.postBatchMutate(c, miniBatchOp);
+          return;
+        }
+    this.builder.batchCompleted(miniBatchOp);
+    // noop for the rest of the indexer - its handled by the first call to put/delete
+  }
+
+  private void doPost(WALEdit edit, Mutation m, boolean writeToWAL) throws IOException {
+    try {
+      doPostWithExceptions(edit, m, writeToWAL);
+      return;
+    } catch (Throwable e) {
+      rethrowIndexingException(e);
+    }
+    throw new RuntimeException(
+        "Somehow didn't complete the index update, but didn't return succesfully either!");
+  }
+
+  private void doPostWithExceptions(WALEdit edit, Mutation m, boolean writeToWAL) throws Exception {
+    //short circuit, if we don't need to do any work
+    if (!writeToWAL || !this.builder.isEnabled(m)) {
+      // already did the index update in prePut, so we are done
+      return;
+    }
+
+    // there is a little bit of excess here- we iterate all the non-indexed kvs for this check first
+    // and then do it again later when getting out the index updates. This should be pretty minor
+    // though, compared to the rest of the runtime
+    IndexedKeyValue ikv = getFirstIndexedKeyValue(edit);
+    /*
+     * early exit - we have nothing to write, so we don't need to do anything else. NOTE: we don't
+     * release the WAL Rolling lock (INDEX_UPDATE_LOCK) since we never take it in doPre if there are
+     * no index updates.
+     */
+    if (ikv == null) {
+      return;
+    }
+
+    /*
+     * only write the update if we haven't already seen this batch. We only want to write the batch
+     * once (this hook gets called with the same WALEdit for each Put/Delete in a batch, which can
+     * lead to writing all the index updates for each Put/Delete).
+     */
+    if (!ikv.getBatchFinished()) {
+      Collection<Pair<Mutation, byte[]>> indexUpdates = extractIndexUpdate(edit);
+
+      // the WAL edit is kept in memory and we already specified the factory when we created the
+      // references originally - therefore, we just pass in a null factory here and use the ones
+      // already specified on each reference
+      try {
+          writer.writeAndKillYourselfOnFailure(indexUpdates);
+      } finally {
+        // With a custom kill policy, we may throw instead of kill the server.
+        // Without doing this in a finally block (at least with the mini cluster),
+        // the region server never goes down.
+
+        // mark the batch as having been written. In the single-update case, this never gets check
+        // again, but in the batch case, we will check it again (see above).
+        ikv.markBatchFinished();
+      
+        // release the lock on the index, we wrote everything properly
+        // we took the lock for each Put/Delete, so we have to release it a matching number of times
+        // batch cases only take the lock once, so we need to make sure we don't over-release the
+        // lock.
+        LOG.debug("Releasing INDEX_UPDATE readlock");
+        INDEX_UPDATE_LOCK.unlock();
+      }
+    }
+  }
+
+  /**
+   * Search the {@link WALEdit} for the first {@link IndexedKeyValue} present
+   * @param edit {@link WALEdit}
+   * @return the first {@link IndexedKeyValue} in the {@link WALEdit} or <tt>null</tt> if not
+   *         present
+   */
+  private IndexedKeyValue getFirstIndexedKeyValue(WALEdit edit) {
+    for (KeyValue kv : edit.getKeyValues()) {
+      if (kv instanceof IndexedKeyValue) {
+        return (IndexedKeyValue) kv;
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Extract the index updates from the WAL Edit
+   * @param edit to search for index updates
+   * @return the mutations to apply to the index tables
+   */
+  private Collection<Pair<Mutation, byte[]>> extractIndexUpdate(WALEdit edit) {
+    Collection<Pair<Mutation, byte[]>> indexUpdates = new ArrayList<Pair<Mutation, byte[]>>();
+    for (KeyValue kv : edit.getKeyValues()) {
+      if (kv instanceof IndexedKeyValue) {
+        IndexedKeyValue ikv = (IndexedKeyValue) kv;
+        indexUpdates.add(new Pair<Mutation, byte[]>(ikv.getMutation(), ikv.getIndexTable()));
+      }
+    }
+
+    return indexUpdates;
+  }
+
+  @Override
+  public void postOpen(final ObserverContext<RegionCoprocessorEnvironment> c) {
+    Multimap<HTableInterfaceReference, Mutation> updates = failedIndexEdits.getEdits(c.getEnvironment().getRegion());
+    
+    if (this.disabled) {
+        super.postOpen(c);
+        return;
+      }
+    LOG.info("Found some outstanding index updates that didn't succeed during"
+        + " WAL replay - attempting to replay now.");
+    //if we have no pending edits to complete, then we are done
+    if (updates == null || updates.size() == 0) {
+      return;
+    }
+    
+    // do the usual writer stuff, killing the server again, if we can't manage to make the index
+    // writes succeed again
+    try {
+        writer.writeAndKillYourselfOnFailure(updates);
+    } catch (IOException e) {
+        LOG.error("Exception thrown instead of killing server during index writing", e);
+    }
+  }
+
+  @Override
+  public void preWALRestore(ObserverContext<RegionCoprocessorEnvironment> env, HRegionInfo info,
+      HLogKey logKey, WALEdit logEdit) throws IOException {
+      if (this.disabled) {
+          super.preWALRestore(env, info, logKey, logEdit);
+          return;
+        }
+    // TODO check the regions in transition. If the server on which the region lives is this one,
+    // then we should rety that write later in postOpen.
+    // we might be able to get even smarter here and pre-split the edits that are server-local
+    // into their own recovered.edits file. This then lets us do a straightforward recovery of each
+    // region (and more efficiently as we aren't writing quite as hectically from this one place).
+
+    /*
+     * Basically, we let the index regions recover for a little while long before retrying in the
+     * hopes they come up before the primary table finishes.
+     */
+    Collection<Pair<Mutation, byte[]>> indexUpdates = extractIndexUpdate(logEdit);
+    recoveryWriter.writeAndKillYourselfOnFailure(indexUpdates);
+  }
+
+  /**
+   * Create a custom {@link InternalScanner} for a compaction that tracks the versions of rows that
+   * are removed so we can clean then up from the the index table(s).
+   * <p>
+   * This is not yet implemented - its not clear if we should even mess around with the Index table
+   * for these rows as those points still existed. TODO: v2 of indexing
+   */
+  @Override
+  public InternalScanner preCompactScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
+      Store store, List<? extends KeyValueScanner> scanners, ScanType scanType, long earliestPutTs,
+      InternalScanner s) throws IOException {
+    return super.preCompactScannerOpen(c, store, scanners, scanType, earliestPutTs, s);
+  }
+
+  /**
+   * Exposed for testing!
+   * @return the currently instantiated index builder
+   */
+  public IndexBuilder getBuilderForTesting() {
+    return this.builder.getBuilderForTesting();
+  }
+
+    /**
+     * Validate that the version and configuration parameters are supported
+     * @param hbaseVersion current version of HBase on which <tt>this</tt> coprocessor is installed
+     * @param conf configuration to check for allowed parameters (e.g. WAL Compression only if >=
+     *            0.94.9)
+     * @return <tt>null</tt> if the version is supported, the error message to display otherwise
+     */
+    public static String validateVersion(String hbaseVersion, Configuration conf) {
+        int encodedVersion = MetaDataUtil.encodeVersion(hbaseVersion);
+        // above 0.94 everything should be supported
+        if (encodedVersion > INDEXING_SUPPORTED_MAJOR_VERSION) {
+            return null;
+        }
+        // check to see if its at least 0.94
+        if (encodedVersion < INDEXING_SUPPORTED__MIN_MAJOR_VERSION) {
+            return "Indexing not supported for versions older than 0.94.X";
+        }
+        // if less than 0.94.9, we need to check if WAL Compression is enabled
+        if (encodedVersion < INDEX_WAL_COMPRESSION_MINIMUM_SUPPORTED_VERSION) {
+            if (conf.getBoolean(HConstants.ENABLE_WAL_COMPRESSION, false)) {
+                return "Indexing not supported with WAL Compression for versions of HBase older than 0.94.9 - found version:"
+                        + hbaseVersion;
+            }
+        }
+        return null;
+    }
+
+  /**
+   * Enable indexing on the given table
+   * @param desc {@link HTableDescriptor} for the table on which indexing should be enabled
+   * @param builder class to use when building the index for this table
+   * @param properties map of custom configuration options to make available to your
+   *          {@link IndexBuilder} on the server-side
+   * @throws IOException the Indexer coprocessor cannot be added
+   */
+  public static void enableIndexing(HTableDescriptor desc, Class<? extends IndexBuilder> builder,
+      Map<String, String> properties) throws IOException {
+    if (properties == null) {
+      properties = new HashMap<String, String>();
+    }
+    properties.put(Indexer.INDEX_BUILDER_CONF_KEY, builder.getName());
+    desc.addCoprocessor(Indexer.class.getName(), null, Coprocessor.PRIORITY_USER, properties);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/ValueGetter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/ValueGetter.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/ValueGetter.java
new file mode 100644
index 0000000..54c4fbc
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/ValueGetter.java
@@ -0,0 +1,38 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.index.covered.update.ColumnReference;
+import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+
+public interface ValueGetter {
+
+  /**
+   * Get the most recent (largest timestamp) for the given column reference
+   * @param ref to match against an underlying key value. Uses the passed object to match the
+   *          keyValue via {@link ColumnReference#matches}
+   * @return the stored value for the given {@link ColumnReference}, or <tt>null</tt> if no value is
+   *         present.
+   * @throws IOException if there is an error accessing the underlying data storage
+   */
+  public ImmutableBytesPtr getLatestValue(ColumnReference ref) throws IOException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/builder/BaseIndexBuilder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/builder/BaseIndexBuilder.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/builder/BaseIndexBuilder.java
new file mode 100644
index 0000000..73b7421
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/builder/BaseIndexBuilder.java
@@ -0,0 +1,99 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index.builder;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
+import org.apache.hadoop.hbase.util.Pair;
+
+import org.apache.hadoop.hbase.index.covered.CoveredColumnsIndexBuilder;
+
+/**
+ * Basic implementation of the {@link IndexBuilder} that doesn't do any actual work of indexing.
+ * <p>
+ * You should extend this class, rather than implementing IndexBuilder directly to maintain
+ * compatability going forward.
+ * <p>
+ * Generally, you should consider using one of the implemented IndexBuilders (e.g
+ * {@link CoveredColumnsIndexBuilder}) as there is a lot of work required to keep an index table
+ * up-to-date.
+ */
+public abstract class BaseIndexBuilder implements IndexBuilder {
+
+  private static final Log LOG = LogFactory.getLog(BaseIndexBuilder.class);
+  protected boolean stopped;
+
+  @Override
+  public void extendBaseIndexBuilderInstead() { }
+  
+  @Override
+  public void setup(RegionCoprocessorEnvironment conf) throws IOException {
+    // noop
+  }
+
+  @Override
+  public void batchStarted(MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp) throws IOException {
+    // noop
+  }
+
+  @Override
+  public void batchCompleted(MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp) {
+    // noop
+  }
+  
+  /**
+   * By default, we always attempt to index the mutation. Commonly this can be slow (because the
+   * framework spends the time to do the indexing, only to realize that you don't need it) or not
+   * ideal (if you want to turn on/off indexing on a table without completely reloading it).
+ * @throws IOException 
+   */
+  @Override
+  public boolean isEnabled(Mutation m) throws IOException {
+    return true; 
+  }
+
+  /**
+   * {@inheritDoc}
+   * <p>
+   * By default, assumes that all mutations should <b>not be batched</b>. That is to say, each
+   * mutation always applies to different rows, even if they are in the same batch, or are
+   * independent updates.
+   */
+  @Override
+  public byte[] getBatchId(Mutation m) {
+    return null;
+  }
+
+  @Override
+  public void stop(String why) {
+    LOG.debug("Stopping because: " + why);
+    this.stopped = true;
+  }
+
+  @Override
+  public boolean isStopped() {
+    return this.stopped;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/builder/IndexBuildManager.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/builder/IndexBuildManager.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/builder/IndexBuildManager.java
new file mode 100644
index 0000000..0752446
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/builder/IndexBuildManager.java
@@ -0,0 +1,216 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index.builder;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ExecutionException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
+import org.apache.hadoop.hbase.util.Pair;
+
+import org.apache.hadoop.hbase.index.Indexer;
+import org.apache.hadoop.hbase.index.parallel.QuickFailingTaskRunner;
+import org.apache.hadoop.hbase.index.parallel.Task;
+import org.apache.hadoop.hbase.index.parallel.TaskBatch;
+import org.apache.hadoop.hbase.index.parallel.ThreadPoolBuilder;
+import org.apache.hadoop.hbase.index.parallel.ThreadPoolManager;
+
+/**
+ * Manage the building of index updates from primary table updates.
+ * <p>
+ * Internally, parallelizes updates through a thread-pool to a delegate index builder. Underlying
+ * {@link IndexBuilder} <b>must be thread safe</b> for each index update.
+ */
+public class IndexBuildManager implements Stoppable {
+
+  private static final Log LOG = LogFactory.getLog(IndexBuildManager.class);
+  private final IndexBuilder delegate;
+  private QuickFailingTaskRunner pool;
+  private boolean stopped;
+
+  /**
+   * Set the number of threads with which we can concurrently build index updates. Unused threads
+   * will be released, but setting the number of threads too high could cause frequent swapping and
+   * resource contention on the server - <i>tune with care</i>. However, if you are spending a lot
+   * of time building index updates, it could be worthwhile to spend the time to tune this parameter
+   * as it could lead to dramatic increases in speed.
+   */
+  public static final String NUM_CONCURRENT_INDEX_BUILDER_THREADS_CONF_KEY = "index.builder.threads.max";
+  /** Default to a single thread. This is the safest course of action, but the slowest as well */
+  private static final int DEFAULT_CONCURRENT_INDEX_BUILDER_THREADS = 10;
+  /**
+   * Amount of time to keep idle threads in the pool. After this time (seconds) we expire the
+   * threads and will re-create them as needed, up to the configured max
+   */
+  private static final String INDEX_BUILDER_KEEP_ALIVE_TIME_CONF_KEY =
+      "index.builder.threads.keepalivetime";
+
+  /**
+   * @param env environment in which <tt>this</tt> is running. Used to setup the
+   *          {@link IndexBuilder} and executor
+   * @throws IOException if an {@link IndexBuilder} cannot be correctly steup
+   */
+  public IndexBuildManager(RegionCoprocessorEnvironment env) throws IOException {
+    this(getIndexBuilder(env), new QuickFailingTaskRunner(ThreadPoolManager.getExecutor(
+      getPoolBuilder(env), env)));
+  }
+
+  private static IndexBuilder getIndexBuilder(RegionCoprocessorEnvironment e) throws IOException {
+    Configuration conf = e.getConfiguration();
+    Class<? extends IndexBuilder> builderClass =
+        conf.getClass(Indexer.INDEX_BUILDER_CONF_KEY, null, IndexBuilder.class);
+    try {
+      IndexBuilder builder = builderClass.newInstance();
+      builder.setup(e);
+      return builder;
+    } catch (InstantiationException e1) {
+      throw new IOException("Couldn't instantiate index builder:" + builderClass
+          + ", disabling indexing on table " + e.getRegion().getTableDesc().getNameAsString());
+    } catch (IllegalAccessException e1) {
+      throw new IOException("Couldn't instantiate index builder:" + builderClass
+          + ", disabling indexing on table " + e.getRegion().getTableDesc().getNameAsString());
+    }
+  }
+
+  private static ThreadPoolBuilder getPoolBuilder(RegionCoprocessorEnvironment env) {
+    String serverName = env.getRegionServerServices().getServerName().getServerName();
+    return new ThreadPoolBuilder(serverName + "-index-builder", env.getConfiguration()).
+        setCoreTimeout(INDEX_BUILDER_KEEP_ALIVE_TIME_CONF_KEY).
+        setMaxThread(NUM_CONCURRENT_INDEX_BUILDER_THREADS_CONF_KEY,
+          DEFAULT_CONCURRENT_INDEX_BUILDER_THREADS);
+  }
+
+  public IndexBuildManager(IndexBuilder builder, QuickFailingTaskRunner pool) {
+    this.delegate = builder;
+    this.pool = pool;
+  }
+
+
+  public Collection<Pair<Mutation, byte[]>> getIndexUpdate(
+      MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp,
+      Collection<? extends Mutation> mutations) throws Throwable {
+    // notify the delegate that we have started processing a batch
+    this.delegate.batchStarted(miniBatchOp);
+
+    // parallelize each mutation into its own task
+    // each task is cancelable via two mechanisms: (1) underlying HRegion is closing (which would
+    // fail lookups/scanning) and (2) by stopping this via the #stop method. Interrupts will only be
+    // acknowledged on each thread before doing the actual lookup, but after that depends on the
+    // underlying builder to look for the closed flag.
+    TaskBatch<Collection<Pair<Mutation, byte[]>>> tasks =
+        new TaskBatch<Collection<Pair<Mutation, byte[]>>>(mutations.size());
+    for (final Mutation m : mutations) {
+      tasks.add(new Task<Collection<Pair<Mutation, byte[]>>>() {
+
+        @Override
+        public Collection<Pair<Mutation, byte[]>> call() throws IOException {
+          return delegate.getIndexUpdate(m);
+        }
+
+      });
+    }
+    List<Collection<Pair<Mutation, byte[]>>> allResults = null;
+    try {
+      allResults = pool.submitUninterruptible(tasks);
+    } catch (CancellationException e) {
+      throw e;
+    } catch (ExecutionException e) {
+      LOG.error("Found a failed index update!");
+      throw e.getCause();
+    }
+
+    // we can only get here if we get successes from each of the tasks, so each of these must have a
+    // correct result
+    Collection<Pair<Mutation, byte[]>> results = new ArrayList<Pair<Mutation, byte[]>>();
+    for (Collection<Pair<Mutation, byte[]>> result : allResults) {
+      assert result != null : "Found an unsuccessful result, but didn't propagate a failure earlier";
+      results.addAll(result);
+    }
+
+    return results;
+  }
+
+  public Collection<Pair<Mutation, byte[]>> getIndexUpdate(Delete delete) throws IOException {
+    // all we get is a single update, so it would probably just go slower if we needed to queue it
+    // up. It will increase underlying resource contention a little bit, but the mutation case is
+    // far more common, so let's not worry about it for now.
+    // short circuit so we don't waste time.
+    if (!this.delegate.isEnabled(delete)) {
+      return null;
+    }
+
+    return delegate.getIndexUpdate(delete);
+
+  }
+
+  public Collection<Pair<Mutation, byte[]>> getIndexUpdateForFilteredRows(
+      Collection<KeyValue> filtered) throws IOException {
+    // this is run async, so we can take our time here
+    return delegate.getIndexUpdateForFilteredRows(filtered);
+  }
+
+  public void batchCompleted(MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp) {
+    delegate.batchCompleted(miniBatchOp);
+  }
+
+  public void batchStarted(MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp)
+      throws IOException {
+    delegate.batchStarted(miniBatchOp);
+  }
+
+  public boolean isEnabled(Mutation m) throws IOException {
+    return delegate.isEnabled(m);
+  }
+
+  public byte[] getBatchId(Mutation m) {
+    return delegate.getBatchId(m);
+  }
+
+  @Override
+  public void stop(String why) {
+    if (stopped) {
+      return;
+    }
+    this.stopped = true;
+    this.delegate.stop(why);
+    this.pool.stop(why);
+  }
+
+  @Override
+  public boolean isStopped() {
+    return this.stopped;
+  }
+
+  public IndexBuilder getBuilderForTesting() {
+    return this.delegate;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/builder/IndexBuilder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/builder/IndexBuilder.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/builder/IndexBuilder.java
new file mode 100644
index 0000000..8e49d6d
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/builder/IndexBuilder.java
@@ -0,0 +1,144 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index.builder;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
+import org.apache.hadoop.hbase.util.Pair;
+
+import org.apache.hadoop.hbase.index.Indexer;
+
+/**
+ * Interface to build updates ({@link Mutation}s) to the index tables, based on the primary table
+ * updates.
+ * <p>
+ * Either all the index updates will be applied to all tables or the primary table will kill itself
+ * and will attempt to replay the index edits through the WAL replay mechanism.
+ */
+public interface IndexBuilder extends Stoppable {
+
+  /** Helper method signature to ensure people don't attempt to extend this class directly */
+  public void extendBaseIndexBuilderInstead();
+
+  /**
+   * This is always called exactly once on install of {@link Indexer}, before any calls
+   * {@link #getIndexUpdate} on
+   * @param env in which the builder is running
+   * @throws IOException on failure to setup the builder
+   */
+  public void setup(RegionCoprocessorEnvironment env) throws IOException;
+
+  /**
+   * Your opportunity to update any/all index tables based on the update of the primary table row.
+   * Its up to your implementation to ensure that timestamps match between the primary and index
+   * tables.
+   * <p>
+   * The mutation is a generic mutation (not a {@link Put} or a {@link Delete}), as it actually
+   * corresponds to a batch update. Its important to note that {@link Put}s always go through the
+   * batch update code path, so a single {@link Put} will come through here and update the primary
+   * table as the only update in the mutation.
+   * <p>
+   * Implementers must ensure that this method is thread-safe - it could (and probably will) be
+   * called concurrently for different mutations, which may or may not be part of the same batch.
+   * @param mutation update to the primary table to be indexed.
+   * @return a Map of the mutations to make -> target index table name
+   * @throws IOException on failure
+   */
+  public Collection<Pair<Mutation, byte[]>> getIndexUpdate(Mutation mutation) throws IOException;
+
+  /**
+   * The counter-part to {@link #getIndexUpdate(Mutation)} - your opportunity to update any/all
+   * index tables based on the delete of the primary table row. This is only called for cases where
+   * the client sends a single delete ({@link HTable#delete}). We separate this method from
+   * {@link #getIndexUpdate(Mutation)} only for the ease of implementation as the delete path has
+   * subtly different semantics for updating the families/timestamps from the generic batch path.
+   * <p>
+   * Its up to your implementation to ensure that timestamps match between the primary and index
+   * tables.
+   * <p>
+   * Implementers must ensure that this method is thread-safe - it could (and probably will) be
+   * called concurrently for different mutations, which may or may not be part of the same batch.
+   * @param delete {@link Delete} to the primary table that may be indexed
+   * @return a {@link Map} of the mutations to make -> target index table name
+   * @throws IOException on failure
+   */
+  public Collection<Pair<Mutation, byte[]>> getIndexUpdate(Delete delete) throws IOException;
+
+  /**
+   * Build an index update to cleanup the index when we remove {@link KeyValue}s via the normal
+   * flush or compaction mechanisms.
+   * @param filtered {@link KeyValue}s that previously existed, but won't be included in further
+   *          output from HBase.
+   * @return a {@link Map} of the mutations to make -> target index table name
+   * @throws IOException on failure
+   */
+  public Collection<Pair<Mutation, byte[]>> getIndexUpdateForFilteredRows(
+      Collection<KeyValue> filtered)
+      throws IOException;
+
+  /**
+   * Notification that a batch of updates has successfully been written.
+   * @param miniBatchOp the full batch operation that was written
+   */
+  public void batchCompleted(MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp);
+
+  /**
+   * Notification that a batch has been started.
+   * <p>
+   * Unfortunately, the way HBase has the coprocessor hooks setup, this is actually called
+   * <i>after</i> the {@link #getIndexUpdate} methods. Therefore, you will likely need an attribute
+   * on your {@link Put}/{@link Delete} to indicate it is a batch operation.
+   * @param miniBatchOp the full batch operation to be written
+ * @throws IOException 
+   */
+  public void batchStarted(MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp) throws IOException;
+
+  /**
+   * This allows the codec to dynamically change whether or not indexing should take place for a
+   * table. If it doesn't take place, we can save a lot of time on the regular Put patch. By making
+   * it dynamic, we can save offlining and then onlining a table just to turn indexing on.
+   * <p>
+   * We can also be smart about even indexing a given update here too - if the update doesn't
+   * contain any columns that we care about indexing, we can save the effort of analyzing the put
+   * and further.
+   * @param m mutation that should be indexed.
+   * @return <tt>true</tt> if indexing is enabled for the given table. This should be on a per-table
+   *         basis, as each codec is instantiated per-region.
+ * @throws IOException 
+   */
+  public boolean isEnabled(Mutation m) throws IOException;
+
+  /**
+   * @param m mutation that has been received by the indexer and is waiting to be indexed
+   * @return the ID of batch to which the Mutation belongs, or <tt>null</tt> if the mutation is not
+   *         part of a batch.
+   */
+  public byte[] getBatchId(Mutation m);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/builder/IndexBuildingFailureException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/builder/IndexBuildingFailureException.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/builder/IndexBuildingFailureException.java
new file mode 100644
index 0000000..1c89b7f
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/builder/IndexBuildingFailureException.java
@@ -0,0 +1,50 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index.builder;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+
+/**
+ * Unexpected failure while building index updates that wasn't caused by an {@link IOException}.
+ * This should be used if there is some basic issue with indexing - and no matter of retries will
+ * fix it.
+ */
+@SuppressWarnings("serial")
+public class IndexBuildingFailureException extends DoNotRetryIOException {
+
+  /**
+   * Constructor for over the wire propagation. Generally, shouldn't be used since index failure
+   * should have an underlying cause to propagate.
+   * @param msg reason for the failure
+   */
+  public IndexBuildingFailureException(String msg) {
+    super(msg);
+  }
+
+  /**
+   * @param msg reason
+   * @param cause underlying cause for the failure
+   */
+  public IndexBuildingFailureException(String msg, Throwable cause) {
+    super(msg, cause);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/Batch.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/Batch.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/Batch.java
new file mode 100644
index 0000000..8b7815c
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/Batch.java
@@ -0,0 +1,62 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index.covered;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.hbase.KeyValue;
+
+/**
+ * A collection of {@link KeyValue KeyValues} to the primary table
+ */
+public class Batch {
+
+  private static final long pointDeleteCode = KeyValue.Type.Delete.getCode();
+  private final long timestamp;
+  private List<KeyValue> batch = new ArrayList<KeyValue>();
+  private boolean allPointDeletes = true;
+
+  /**
+   * @param ts
+   */
+  public Batch(long ts) {
+    this.timestamp = ts;
+  }
+
+  public void add(KeyValue kv){
+    if (pointDeleteCode != kv.getType()) {
+      allPointDeletes = false;
+    }
+    batch.add(kv);
+  }
+
+  public boolean isAllPointDeletes() {
+    return allPointDeletes;
+  }
+
+  public long getTimestamp() {
+    return this.timestamp;
+  }
+
+  public List<KeyValue> getKvs() {
+    return this.batch;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/CoveredColumns.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/CoveredColumns.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/CoveredColumns.java
new file mode 100644
index 0000000..4846dc8
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/CoveredColumns.java
@@ -0,0 +1,51 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index.covered;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.hadoop.hbase.index.covered.update.ColumnReference;
+
+/**
+ * Manage a set of {@link ColumnReference}s for the {@link LocalTableState}.
+ */
+public class CoveredColumns {
+
+  Set<ColumnReference> columns = new HashSet<ColumnReference>();
+
+  public Collection<? extends ColumnReference> findNonCoveredColumns(
+      Collection<? extends ColumnReference> columns2) {
+    List<ColumnReference> uncovered = new ArrayList<ColumnReference>();
+    for (ColumnReference column : columns2) {
+      if (!columns.contains(column)) {
+        uncovered.add(column);
+      }
+    }
+    return uncovered;
+  }
+
+  public void addColumn(ColumnReference column) {
+    this.columns.add(column);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/CoveredColumnsIndexBuilder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/CoveredColumnsIndexBuilder.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/CoveredColumnsIndexBuilder.java
new file mode 100644
index 0000000..1e48d5c
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/CoveredColumnsIndexBuilder.java
@@ -0,0 +1,490 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index.covered;
+
+import java.io.IOException;
+import java.lang.reflect.Constructor;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.Pair;
+
+import com.google.common.collect.Lists;
+import com.google.common.primitives.Longs;
+import org.apache.hadoop.hbase.index.builder.BaseIndexBuilder;
+import org.apache.hadoop.hbase.index.covered.data.LocalHBaseState;
+import org.apache.hadoop.hbase.index.covered.data.LocalTable;
+import org.apache.hadoop.hbase.index.covered.update.ColumnTracker;
+import org.apache.hadoop.hbase.index.covered.update.IndexUpdateManager;
+import org.apache.hadoop.hbase.index.covered.update.IndexedColumnGroup;
+
+/**
+ * Build covered indexes for phoenix updates.
+ * <p>
+ * Before any call to prePut/preDelete, the row has already been locked. This ensures that we don't
+ * need to do any extra synchronization in the IndexBuilder.
+ * <p>
+ * NOTE: This implementation doesn't cleanup the index when we remove a key-value on compaction or
+ * flush, leading to a bloated index that needs to be cleaned up by a background process.
+ */
+public class CoveredColumnsIndexBuilder extends BaseIndexBuilder {
+
+  private static final Log LOG = LogFactory.getLog(CoveredColumnsIndexBuilder.class);
+  public static final String CODEC_CLASS_NAME_KEY = "org.apache.hadoop.hbase.index.codec.class";
+
+  protected RegionCoprocessorEnvironment env;
+  protected IndexCodec codec;
+  protected LocalHBaseState localTable;
+
+  @Override
+  public void setup(RegionCoprocessorEnvironment env) throws IOException {
+    this.env = env;
+    // setup the phoenix codec. Generally, this will just be in standard one, but abstracting here
+    // so we can use it later when generalizing covered indexes
+    Configuration conf = env.getConfiguration();
+    Class<? extends IndexCodec> codecClass =
+        conf.getClass(CODEC_CLASS_NAME_KEY, null, IndexCodec.class);
+    try {
+      Constructor<? extends IndexCodec> meth = codecClass.getDeclaredConstructor(new Class[0]);
+      meth.setAccessible(true);
+      this.codec = meth.newInstance();
+      this.codec.initialize(env);
+    } catch (IOException e) {
+      throw e;
+    } catch (Exception e) {
+      throw new IOException(e);
+    }
+    
+    this.localTable = new LocalTable(env);
+  }
+
+  @Override
+  public Collection<Pair<Mutation, byte[]>> getIndexUpdate(Mutation mutation) throws IOException {
+    // build the index updates for each group
+    IndexUpdateManager updateMap = new IndexUpdateManager();
+
+    batchMutationAndAddUpdates(updateMap, mutation);
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Found index updates for Mutation: " + mutation + "\n" + updateMap);
+    }
+
+    return updateMap.toMap();
+  }
+
+  /**
+   * Split the mutation into batches based on the timestamps of each keyvalue. We need to check each
+   * key-value in the update to see if it matches the others. Generally, this will be the case, but
+   * you can add kvs to a mutation that don't all have the timestamp, so we need to manage
+   * everything in batches based on timestamp.
+   * <p>
+   * Adds all the updates in the {@link Mutation} to the state, as a side-effect.
+   * @param updateMap index updates into which to add new updates. Modified as a side-effect.
+   * @param state current state of the row for the mutation.
+   * @param m mutation to batch
+ * @throws IOException 
+   */
+  private void batchMutationAndAddUpdates(IndexUpdateManager manager, Mutation m) throws IOException {
+    // split the mutation into timestamp-based batches
+    Collection<Batch> batches = createTimestampBatchesFromMutation(m);
+
+    // create a state manager, so we can manage each batch
+    LocalTableState state = new LocalTableState(env, localTable, m);
+
+    // go through each batch of keyvalues and build separate index entries for each
+    boolean cleanupCurrentState = true;
+    for (Batch batch : batches) {
+      /*
+       * We have to split the work between the cleanup and the update for each group because when we
+       * update the current state of the row for the current batch (appending the mutations for the
+       * current batch) the next group will see that as the current state, which will can cause the
+       * a delete and a put to be created for the next group.
+       */
+      if (addMutationsForBatch(manager, batch, state, cleanupCurrentState)) {
+        cleanupCurrentState = false;
+      }
+    }
+  }
+
+  /**
+   * Batch all the {@link KeyValue}s in a {@link Mutation} by timestamp. Updates any
+   * {@link KeyValue} with a timestamp == {@link HConstants#LATEST_TIMESTAMP} to the timestamp at
+   * the time the method is called.
+   * @param m {@link Mutation} from which to extract the {@link KeyValue}s
+   * @return the mutation, broken into batches and sorted in ascending order (smallest first)
+   */
+  protected Collection<Batch> createTimestampBatchesFromMutation(Mutation m) {
+    Map<Long, Batch> batches = new HashMap<Long, Batch>();
+    for (List<KeyValue> family : m.getFamilyMap().values()) {
+      createTimestampBatchesFromKeyValues(family, batches);
+    }
+    // sort the batches
+    List<Batch> sorted = new ArrayList<Batch>(batches.values());
+    Collections.sort(sorted, new Comparator<Batch>() {
+      @Override
+      public int compare(Batch o1, Batch o2) {
+        return Longs.compare(o1.getTimestamp(), o2.getTimestamp());
+      }
+    });
+    return sorted;
+  }
+
+  /**
+   * Batch all the {@link KeyValue}s in a collection of kvs by timestamp. Updates any
+   * {@link KeyValue} with a timestamp == {@link HConstants#LATEST_TIMESTAMP} to the timestamp at
+   * the time the method is called.
+   * @param kvs {@link KeyValue}s to break into batches
+   * @param batches to update with the given kvs
+   */
+  protected void createTimestampBatchesFromKeyValues(Collection<KeyValue> kvs,
+      Map<Long, Batch> batches) {
+    long now = EnvironmentEdgeManager.currentTimeMillis();
+    byte[] nowBytes = Bytes.toBytes(now);
+
+    // batch kvs by timestamp
+    for (KeyValue kv : kvs) {
+      long ts = kv.getTimestamp();
+      // override the timestamp to the current time, so the index and primary tables match
+      // all the keys with LATEST_TIMESTAMP will then be put into the same batch
+      if (kv.updateLatestStamp(nowBytes)) {
+        ts = now;
+      }
+      Batch batch = batches.get(ts);
+      if (batch == null) {
+        batch = new Batch(ts);
+        batches.put(ts, batch);
+      }
+      batch.add(kv);
+    }
+  }
+
+  /**
+   * For a single batch, get all the index updates and add them to the updateMap
+   * <p>
+   * This method manages cleaning up the entire history of the row from the given timestamp forward
+   * for out-of-order (e.g. 'back in time') updates.
+   * <p>
+   * If things arrive out of order (client is using custom timestamps) we should still see the index
+   * in the correct order (assuming we scan after the out-of-order update in finished). Therefore,
+   * we when we aren't the most recent update to the index, we need to delete the state at the
+   * current timestamp (similar to above), but also issue a delete for the added index updates at
+   * the next newest timestamp of any of the columns in the update; we need to cleanup the insert so
+   * it looks like it was also deleted at that next newest timestamp. However, its not enough to
+   * just update the one in front of us - that column will likely be applied to index entries up the
+   * entire history in front of us, which also needs to be fixed up.
+   * <p>
+   * However, the current update usually will be the most recent thing to be added. In that case,
+   * all we need to is issue a delete for the previous index row (the state of the row, without the
+   * update applied) at the current timestamp. This gets rid of anything currently in the index for
+   * the current state of the row (at the timestamp). Then we can just follow that by applying the
+   * pending update and building the index update based on the new row state.
+   * @param updateMap map to update with new index elements
+   * @param batch timestamp-based batch of edits
+   * @param state local state to update and pass to the codec
+   * @param requireCurrentStateCleanup <tt>true</tt> if we should should attempt to cleanup the
+   *          current state of the table, in the event of a 'back in time' batch. <tt>false</tt>
+   *          indicates we should not attempt the cleanup, e.g. an earlier batch already did the
+   *          cleanup.
+   * @return <tt>true</tt> if we cleaned up the current state forward (had a back-in-time put),
+   *         <tt>false</tt> otherwise
+ * @throws IOException 
+   */
+  private boolean addMutationsForBatch(IndexUpdateManager updateMap, Batch batch,
+      LocalTableState state, boolean requireCurrentStateCleanup) throws IOException {
+
+    // need a temporary manager for the current batch. It should resolve any conflicts for the
+    // current batch. Essentially, we can get the case where a batch doesn't change the current
+    // state of the index (all Puts are covered by deletes), in which case we don't want to add
+    // anything
+    // A. Get the correct values for the pending state in the batch
+    // A.1 start by cleaning up the current state - as long as there are key-values in the batch
+    // that are indexed, we need to change the current state of the index. Its up to the codec to
+    // determine if we need to make any cleanup given the pending update.
+    long batchTs = batch.getTimestamp();
+    state.setPendingUpdates(batch.getKvs());
+    addCleanupForCurrentBatch(updateMap, batchTs, state);
+
+    // A.2 do a single pass first for the updates to the current state
+    state.applyPendingUpdates();
+    long minTs = addUpdateForGivenTimestamp(batchTs, state, updateMap);
+    // if all the updates are the latest thing in the index, we are done - don't go and fix history
+    if (ColumnTracker.isNewestTime(minTs)) {
+      return false;
+    }
+
+    // A.3 otherwise, we need to roll up through the current state and get the 'correct' view of the
+    // index. after this, we have the correct view of the index, from the batch up to the index
+    while(!ColumnTracker.isNewestTime(minTs) ){
+      minTs = addUpdateForGivenTimestamp(minTs, state, updateMap);
+    }
+
+    // B. only cleanup the current state if we need to - its a huge waste of effort otherwise.
+   if (requireCurrentStateCleanup) {
+      // roll back the pending update. This is needed so we can remove all the 'old' index entries.
+      // We don't need to do the puts here, but just the deletes at the given timestamps since we
+      // just want to completely hide the incorrect entries.
+      state.rollback(batch.getKvs());
+      // setup state
+      state.setPendingUpdates(batch.getKvs());
+
+      // cleanup the pending batch. If anything in the correct history is covered by Deletes used to
+      // 'fix' history (same row key and ts), we just drop the delete (we don't want to drop both
+      // because the update may have a different set of columns or value based on the update).
+      cleanupIndexStateFromBatchOnward(updateMap, batchTs, state);
+
+      // have to roll the state forward again, so the current state is correct
+      state.applyPendingUpdates();
+      return true;
+    }
+    return false;
+  }
+
+  private long addUpdateForGivenTimestamp(long ts, LocalTableState state,
+      IndexUpdateManager updateMap) throws IOException {
+    state.setCurrentTimestamp(ts);
+    ts = addCurrentStateMutationsForBatch(updateMap, state);
+    return ts;
+  }
+
+  private void addCleanupForCurrentBatch(IndexUpdateManager updateMap, long batchTs,
+      LocalTableState state) throws IOException {
+    // get the cleanup for the current state
+    state.setCurrentTimestamp(batchTs);
+    addDeleteUpdatesToMap(updateMap, state, batchTs);
+    // ignore any index tracking from the delete
+    state.resetTrackedColumns();
+  }
+  
+  /**
+   * Add the necessary mutations for the pending batch on the local state. Handles rolling up
+   * through history to determine the index changes after applying the batch (for the case where the
+   * batch is back in time).
+   * @param updateMap to update with index mutations
+   * @param batch to apply to the current state
+   * @param state current state of the table
+   * @return the minimum timestamp across all index columns requested. If
+   *         {@link ColumnTracker#isNewestTime(long)} returns <tt>true</tt> on the returned
+   *         timestamp, we know that this <i>was not a back-in-time update</i>.
+ * @throws IOException 
+   */
+  private long
+      addCurrentStateMutationsForBatch(IndexUpdateManager updateMap, LocalTableState state) throws IOException {
+
+    // get the index updates for this current batch
+    Iterable<IndexUpdate> upserts = codec.getIndexUpserts(state);
+    state.resetTrackedColumns();
+
+    /*
+     * go through all the pending updates. If we are sure that all the entries are the latest
+     * timestamp, we can just add the index updates and move on. However, if there are columns that
+     * we skip past (based on the timestamp of the batch), we need to roll back up the history.
+     * Regardless of whether or not they are the latest timestamp, the entries here are going to be
+     * correct for the current batch timestamp, so we add them to the updates. The only thing we
+     * really care about it if we need to roll up the history and fix it as we go.
+     */
+    // timestamp of the next update we need to track
+    long minTs = ColumnTracker.NO_NEWER_PRIMARY_TABLE_ENTRY_TIMESTAMP;
+    List<IndexedColumnGroup> columnHints = new ArrayList<IndexedColumnGroup>();
+    for (IndexUpdate update : upserts) {
+      // this is the one bit where we check the timestamps
+      final ColumnTracker tracker = update.getIndexedColumns();
+      long trackerTs = tracker.getTS();
+      // update the next min TS we need to track
+      if (trackerTs < minTs) {
+        minTs = tracker.getTS();
+      }
+      // track index hints for the next round. Hint if we need an update for that column for the
+      // next timestamp. These columns clearly won't need to update as we go through time as they
+      // already match the most recent possible thing.
+      boolean needsCleanup = false;
+      if (tracker.hasNewerTimestamps()) {
+        columnHints.add(tracker);
+        // this update also needs to be cleaned up at the next timestamp because it not the latest.
+        needsCleanup = true;
+      }
+
+
+      // only make the put if the index update has been setup
+      if (update.isValid()) {
+        byte[] table = update.getTableName();
+        Mutation mutation = update.getUpdate();
+        updateMap.addIndexUpdate(table, mutation);
+
+        // only make the cleanup if we made a put and need cleanup
+        if (needsCleanup) {
+          // there is a TS for the interested columns that is greater than the columns in the
+          // put. Therefore, we need to issue a delete at the same timestamp
+          Delete d = new Delete(mutation.getRow());
+          d.setTimestamp(tracker.getTS());
+          updateMap.addIndexUpdate(table, d);
+        }
+      }
+    }
+    return minTs;
+  }
+
+  /**
+   * Cleanup the index based on the current state from the given batch. Iterates over each timestamp
+   * (for the indexed rows) for the current state of the table and cleans up all the existing
+   * entries generated by the codec.
+   * <p>
+   * Adds all pending updates to the updateMap
+   * @param updateMap updated with the pending index updates from the codec
+   * @param batchTs timestamp from which we should cleanup
+   * @param state current state of the primary table. Should already by setup to the correct state
+   *          from which we want to cleanup.
+ * @throws IOException 
+   */
+  private void cleanupIndexStateFromBatchOnward(IndexUpdateManager updateMap,
+      long batchTs, LocalTableState state) throws IOException {
+    // get the cleanup for the current state
+    state.setCurrentTimestamp(batchTs);
+    addDeleteUpdatesToMap(updateMap, state, batchTs);
+    Set<ColumnTracker> trackers = state.getTrackedColumns();
+    long minTs = ColumnTracker.NO_NEWER_PRIMARY_TABLE_ENTRY_TIMESTAMP;
+    for (ColumnTracker tracker : trackers) {
+      if (tracker.getTS() < minTs) {
+        minTs = tracker.getTS();
+      }
+    }
+    state.resetTrackedColumns();
+    if (!ColumnTracker.isNewestTime(minTs)) {
+      state.setHints(Lists.newArrayList(trackers));
+      cleanupIndexStateFromBatchOnward(updateMap, minTs, state);
+    }
+  }
+
+
+  /**
+   * Get the index deletes from the codec {@link IndexCodec#getIndexDeletes(TableState)} and then
+   * add them to the update map.
+   * <p>
+   * Expects the {@link LocalTableState} to already be correctly setup (correct timestamp, updates
+   * applied, etc).
+ * @throws IOException 
+   */
+  protected void
+      addDeleteUpdatesToMap(IndexUpdateManager updateMap,
+      LocalTableState state, long ts) throws IOException {
+    Iterable<IndexUpdate> cleanup = codec.getIndexDeletes(state);
+    if (cleanup != null) {
+      for (IndexUpdate d : cleanup) {
+        if (!d.isValid()) {
+          continue;
+        }
+        // override the timestamps in the delete to match the current batch.
+        Delete remove = (Delete)d.getUpdate();
+        remove.setTimestamp(ts);
+        updateMap.addIndexUpdate(d.getTableName(), remove);
+      }
+    }
+  }
+
+  @Override
+  public Collection<Pair<Mutation, byte[]>> getIndexUpdate(Delete d) throws IOException {
+    // stores all the return values
+    IndexUpdateManager updateMap = new IndexUpdateManager();
+
+    // We have to figure out which kind of delete it is, since we need to do different things if its
+    // a general (row) delete, versus a delete of just a single column or family
+    Map<byte[], List<KeyValue>> families = d.getFamilyMap();
+
+    /*
+     * Option 1: its a row delete marker, so we just need to delete the most recent state for each
+     * group, as of the specified timestamp in the delete. This can happen if we have a single row
+     * update and it is part of a batch mutation (prepare doesn't happen until later... maybe a
+     * bug?). In a single delete, this delete gets all the column families appended, so the family
+     * map won't be empty by the time it gets here.
+     */
+    if (families.size() == 0) {
+      LocalTableState state = new LocalTableState(env, localTable, d);
+      // get a consistent view of name
+      long now = d.getTimeStamp();
+      if (now == HConstants.LATEST_TIMESTAMP) {
+        now = EnvironmentEdgeManager.currentTimeMillis();
+        // update the delete's idea of 'now' to be consistent with the index
+        d.setTimestamp(now);
+      }
+      // get deletes from the codec
+      // we only need to get deletes and not add puts because this delete covers all columns
+      addDeleteUpdatesToMap(updateMap, state, now);
+
+      /*
+       * Update the current state for all the kvs in the delete. Generally, we would just iterate
+       * the family map, but since we go here, the family map is empty! Therefore, we need to fake a
+       * bunch of family deletes (just like hos HRegion#prepareDelete works). This is just needed
+       * for current version of HBase that has an issue where the batch update doesn't update the
+       * deletes before calling the hook.
+       */
+      byte[] deleteRow = d.getRow();
+      for (byte[] family : this.env.getRegion().getTableDesc().getFamiliesKeys()) {
+        state.addPendingUpdates(new KeyValue(deleteRow, family, null, now,
+            KeyValue.Type.DeleteFamily));
+      }
+    } else {
+      // Option 2: Its actually a bunch single updates, which can have different timestamps.
+      // Therefore, we need to do something similar to the put case and batch by timestamp
+      batchMutationAndAddUpdates(updateMap, d);
+    }
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Found index updates for Delete: " + d + "\n" + updateMap);
+    }
+
+    return updateMap.toMap();
+  }
+
+  @Override
+  public Collection<Pair<Mutation, byte[]>> getIndexUpdateForFilteredRows(
+      Collection<KeyValue> filtered) throws IOException {
+    // TODO Implement IndexBuilder.getIndexUpdateForFilteredRows
+    return null;
+  }
+
+  /**
+   * Exposed for testing!
+   * @param codec codec to use for this instance of the builder
+   */
+  public void setIndexCodecForTesting(IndexCodec codec) {
+    this.codec = codec;
+  }
+
+  @Override
+  public boolean isEnabled(Mutation m) throws IOException {
+    // ask the codec to see if we should even attempt indexing
+    return this.codec.isEnabled(m);
+  }
+}
\ No newline at end of file


[40/51] [partial] Initial commit of master branch from github

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..4bbb357
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java
@@ -0,0 +1,1133 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.compile;
+
+import static org.apache.phoenix.schema.SaltingUtil.SALTING_COLUMN;
+
+import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+import org.apache.phoenix.expression.AndExpression;
+import org.apache.phoenix.expression.CoerceExpression;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.jdbc.PhoenixStatement;
+import org.apache.phoenix.join.ScanProjector;
+import org.apache.phoenix.parse.AliasedNode;
+import org.apache.phoenix.parse.AndParseNode;
+import org.apache.phoenix.parse.BetweenParseNode;
+import org.apache.phoenix.parse.BindTableNode;
+import org.apache.phoenix.parse.CaseParseNode;
+import org.apache.phoenix.parse.CastParseNode;
+import org.apache.phoenix.parse.ColumnParseNode;
+import org.apache.phoenix.parse.ComparisonParseNode;
+import org.apache.phoenix.parse.ConcreteTableNode;
+import org.apache.phoenix.parse.DerivedTableNode;
+import org.apache.phoenix.parse.EqualParseNode;
+import org.apache.phoenix.parse.FunctionParseNode;
+import org.apache.phoenix.parse.InListParseNode;
+import org.apache.phoenix.parse.IsNullParseNode;
+import org.apache.phoenix.parse.JoinTableNode;
+import org.apache.phoenix.parse.JoinTableNode.JoinType;
+import org.apache.phoenix.parse.LikeParseNode;
+import org.apache.phoenix.parse.NamedTableNode;
+import org.apache.phoenix.parse.NotParseNode;
+import org.apache.phoenix.parse.OrParseNode;
+import org.apache.phoenix.parse.OrderByNode;
+import org.apache.phoenix.parse.ParseNode;
+import org.apache.phoenix.parse.ParseNodeFactory;
+import org.apache.phoenix.parse.SelectStatement;
+import org.apache.phoenix.parse.StatelessTraverseAllParseNodeVisitor;
+import org.apache.phoenix.parse.TableName;
+import org.apache.phoenix.parse.TableNode;
+import org.apache.phoenix.parse.TableNodeVisitor;
+import org.apache.phoenix.parse.TraverseNoParseNodeVisitor;
+import org.apache.phoenix.parse.WildcardParseNode;
+import org.apache.phoenix.schema.AmbiguousColumnException;
+import org.apache.phoenix.schema.ColumnNotFoundException;
+import org.apache.phoenix.schema.ColumnRef;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PColumnImpl;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.PName;
+import org.apache.phoenix.schema.PNameFactory;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTableImpl;
+import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.util.SchemaUtil;
+
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ListMultimap;
+import com.google.common.collect.Lists;
+
+
+public class JoinCompiler {
+    
+    public enum ColumnRefType {
+        PREFILTER,
+        JOINLOCAL,
+        GENERAL,
+    }
+    
+    public static class JoinSpec {
+        private TableRef mainTable;
+        private List<AliasedNode> select; // all basic nodes related to mainTable, no aggregation.
+        private List<ParseNode> preFilters;
+        private List<ParseNode> postFilters;
+        private List<JoinTable> joinTables;
+        private Map<ColumnRef, ColumnRefType> columnRefs;
+        
+        private JoinSpec(SelectStatement statement, ColumnResolver resolver) throws SQLException {
+        	List<AliasedNode> selectList = statement.getSelect();
+            List<TableNode> tableNodes = statement.getFrom();
+            assert (tableNodes.size() > 1);
+            Iterator<TableNode> iter = tableNodes.iterator();
+            Iterator<TableRef> tableRefIter = resolver.getTables().iterator();
+            iter.next();
+            this.mainTable = tableRefIter.next();
+            this.select = extractFromSelect(selectList, mainTable, resolver);
+            this.joinTables = new ArrayList<JoinTable>(tableNodes.size() - 1);
+            this.preFilters = new ArrayList<ParseNode>();
+            this.postFilters = new ArrayList<ParseNode>();
+            ColumnParseNodeVisitor generalRefVisitor = new ColumnParseNodeVisitor(resolver);
+            ColumnParseNodeVisitor joinLocalRefVisitor = new ColumnParseNodeVisitor(resolver);
+            ColumnParseNodeVisitor prefilterRefVisitor = new ColumnParseNodeVisitor(resolver);            
+            boolean hasRightJoin = false;
+            TableNode tableNode = null;
+            while (iter.hasNext()) {
+                tableNode = iter.next();
+                if (!(tableNode instanceof JoinTableNode))
+                    throw new SQLFeatureNotSupportedException("Full joins not supported.");
+                JoinTableNode joinTableNode = (JoinTableNode) tableNode;
+                JoinTable joinTable = new JoinTable(joinTableNode, tableRefIter.next(), selectList, resolver);
+                joinTables.add(joinTable);
+                for (ParseNode prefilter : joinTable.preFilters) {
+                    prefilter.accept(prefilterRefVisitor);
+                }
+                for (ParseNode condition : joinTable.conditions) {
+                    ComparisonParseNode comparisonNode = (ComparisonParseNode) condition;
+                    comparisonNode.getLHS().accept(generalRefVisitor);
+                    comparisonNode.getRHS().accept(joinLocalRefVisitor);
+                }
+                if (joinTable.getType() == JoinType.Right) {
+                	hasRightJoin = true;
+                }
+            }
+            if (statement.getWhere() != null) {
+            	if (hasRightJoin) {
+            		// conditions can't be pushed down to the scan filter.
+            		postFilters.add(statement.getWhere());
+            	} else {
+            		statement.getWhere().accept(new WhereNodeVisitor(resolver));
+            		for (ParseNode prefilter : preFilters) {
+            		    prefilter.accept(prefilterRefVisitor);
+            		}
+            	}
+            	for (ParseNode postfilter : postFilters) {
+            		postfilter.accept(generalRefVisitor);
+            	}
+            }
+            for (AliasedNode node : selectList) {
+                node.getNode().accept(generalRefVisitor);
+            }
+            if (statement.getGroupBy() != null) {
+                for (ParseNode node : statement.getGroupBy()) {
+                    node.accept(generalRefVisitor);
+                }
+            }
+            if (statement.getHaving() != null) {
+                statement.getHaving().accept(generalRefVisitor);
+            }
+            if (statement.getOrderBy() != null) {
+                for (OrderByNode node : statement.getOrderBy()) {
+                    node.getNode().accept(generalRefVisitor);
+                }
+            }
+            this.columnRefs = new HashMap<ColumnRef, ColumnRefType>();
+            for (ColumnRef ref : generalRefVisitor.getColumnRefMap().keySet()) {
+                columnRefs.put(ref, ColumnRefType.GENERAL);
+            }
+            for (ColumnRef ref : joinLocalRefVisitor.getColumnRefMap().keySet()) {
+                if (!columnRefs.containsKey(ref))
+                    columnRefs.put(ref, ColumnRefType.JOINLOCAL);
+            }
+            for (ColumnRef ref : prefilterRefVisitor.getColumnRefMap().keySet()) {
+                if (!columnRefs.containsKey(ref))
+                    columnRefs.put(ref, ColumnRefType.PREFILTER);
+            }            
+        }
+        
+        private JoinSpec(TableRef table, List<AliasedNode> select, List<ParseNode> preFilters, 
+                List<ParseNode> postFilters, List<JoinTable> joinTables, Map<ColumnRef, ColumnRefType> columnRefs) {
+            this.mainTable = table;
+            this.select = select;
+            this.preFilters = preFilters;
+            this.postFilters = postFilters;
+            this.joinTables = joinTables;
+            this.columnRefs = columnRefs;
+        }
+        
+        public TableRef getMainTable() {
+            return mainTable;
+        }
+        
+        public List<AliasedNode> getSelect() {
+            return select;
+        }
+        
+        public List<ParseNode> getPreFilters() {
+            return preFilters;
+        }
+        
+        public List<ParseNode> getPostFilters() {
+            return postFilters;
+        }
+        
+        public List<JoinTable> getJoinTables() {
+            return joinTables;
+        }
+        
+        public ParseNode getPreFiltersCombined() {
+            if (preFilters == null || preFilters.isEmpty())
+                return null;
+            
+            if (preFilters.size() == 1)
+                return preFilters.get(0);
+            
+            return NODE_FACTORY.and(preFilters);
+        }
+        
+        public Expression compilePostFilterExpression(StatementContext context) throws SQLException {
+        	if (postFilters == null || postFilters.isEmpty())
+        		return null;
+        	
+            ExpressionCompiler expressionCompiler = new ExpressionCompiler(context);
+            List<Expression> expressions = new ArrayList<Expression>(postFilters.size());
+            for (ParseNode postFilter : postFilters) {
+                expressionCompiler.reset();
+                Expression expression = postFilter.accept(expressionCompiler);
+                expressions.add(expression);
+            }
+            
+            if (expressions.size() == 1)
+            	return expressions.get(0);
+            
+            return new AndExpression(expressions);
+        }
+
+        public void projectColumns(Scan scan, TableRef table) {
+            if (isWildCardSelect(select)) {
+                scan.getFamilyMap().clear();
+                return;
+            }
+            for (ColumnRef columnRef : columnRefs.keySet()) {
+                if (columnRef.getTableRef().equals(table)
+                        && !SchemaUtil.isPKColumn(columnRef.getColumn())) {
+                    scan.addColumn(columnRef.getColumn().getFamilyName().getBytes(), columnRef.getColumn().getName().getBytes());
+                }
+            }
+        }
+        
+        public ProjectedPTableWrapper createProjectedTable(TableRef tableRef, boolean retainPKColumns) throws SQLException {
+        	List<PColumn> projectedColumns = new ArrayList<PColumn>();
+        	List<Expression> sourceExpressions = new ArrayList<Expression>();
+        	ListMultimap<String, String> columnNameMap = ArrayListMultimap.<String, String>create();
+            PTable table = tableRef.getTable();
+            boolean hasSaltingColumn = retainPKColumns && table.getBucketNum() != null;
+            if (retainPKColumns) {
+            	for (PColumn column : table.getPKColumns()) {
+            		addProjectedColumn(projectedColumns, sourceExpressions, columnNameMap,
+            				column, tableRef, column.getFamilyName(), hasSaltingColumn);
+            	}
+            }
+            if (isWildCardSelect(select)) {
+            	for (PColumn column : table.getColumns()) {
+            		if (!retainPKColumns || !SchemaUtil.isPKColumn(column)) {
+            			addProjectedColumn(projectedColumns, sourceExpressions, columnNameMap,
+            					column, tableRef, PNameFactory.newName(ScanProjector.VALUE_COLUMN_FAMILY), hasSaltingColumn);
+            		}
+            	}
+            } else {
+                for (Map.Entry<ColumnRef, ColumnRefType> e : columnRefs.entrySet()) {
+                    ColumnRef columnRef = e.getKey();
+                    if (e.getValue() != ColumnRefType.PREFILTER 
+                            && columnRef.getTableRef().equals(tableRef)
+                            && (!retainPKColumns || !SchemaUtil.isPKColumn(columnRef.getColumn()))) {
+                    	PColumn column = columnRef.getColumn();
+            			addProjectedColumn(projectedColumns, sourceExpressions, columnNameMap,
+            					column, tableRef, PNameFactory.newName(ScanProjector.VALUE_COLUMN_FAMILY), hasSaltingColumn);
+                    }
+                }            	
+            }
+            
+            PTable t = PTableImpl.makePTable(PNameFactory.newName(PROJECTED_TABLE_SCHEMA), table.getName(), PTableType.JOIN, table.getIndexState(),
+                        table.getTimeStamp(), table.getSequenceNumber(), table.getPKName(), retainPKColumns ? table.getBucketNum() : null,
+                        projectedColumns, table.getParentTableName(), table.getIndexes(),
+                        table.isImmutableRows(), Collections.<PName>emptyList(), null, null, table.isWALDisabled(), table.isMultiTenant(), table.getViewType());
+            return new ProjectedPTableWrapper(t, columnNameMap, sourceExpressions);
+        }
+        
+        private static void addProjectedColumn(List<PColumn> projectedColumns, List<Expression> sourceExpressions,
+        		ListMultimap<String, String> columnNameMap, PColumn sourceColumn, TableRef sourceTable, PName familyName, boolean hasSaltingColumn) 
+        throws SQLException {
+            if (sourceColumn == SALTING_COLUMN)
+                return;
+            
+        	int position = projectedColumns.size() + (hasSaltingColumn ? 1 : 0);
+        	PTable table = sourceTable.getTable();
+        	PName colName = sourceColumn.getName();
+        	PName name = sourceTable.getTableAlias() == null ? null : PNameFactory.newName(getProjectedColumnName(null, sourceTable.getTableAlias(), colName.getString()));
+        	PName fullName = getProjectedColumnName(table.getSchemaName(), table.getTableName(), colName);
+        	if (name == null) {
+        	    name = fullName;
+        	} else {
+        		columnNameMap.put(fullName.getString(), name.getString());
+        	}
+            columnNameMap.put(colName.getString(), name.getString());
+    		PColumnImpl column = new PColumnImpl(name, familyName, sourceColumn.getDataType(), 
+    				sourceColumn.getMaxLength(), sourceColumn.getScale(), sourceColumn.isNullable(), 
+    				position, sourceColumn.getColumnModifier(), sourceColumn.getArraySize());
+        	Expression sourceExpression = new ColumnRef(sourceTable, sourceColumn.getPosition()).newColumnExpression();
+        	projectedColumns.add(column);
+        	sourceExpressions.add(sourceExpression);
+        }
+        
+        public boolean hasPostReference(TableRef table) {
+            if (isWildCardSelect(select)) 
+                return true;
+            
+            for (Map.Entry<ColumnRef, ColumnRefType> e : columnRefs.entrySet()) {
+                if (e.getValue() == ColumnRefType.GENERAL && e.getKey().getTableRef().equals(table)) {
+                    return true;
+                }
+            }
+            
+            return false;
+        }
+        
+        private class WhereNodeVisitor  extends TraverseNoParseNodeVisitor<Void> {
+            private ColumnResolver resolver;
+            
+            public WhereNodeVisitor(ColumnResolver resolver) {
+                this.resolver = resolver;
+            }
+            
+            private Void leaveBooleanNode(ParseNode node,
+                    List<Void> l) throws SQLException {
+                ColumnParseNodeVisitor visitor = new ColumnParseNodeVisitor(resolver);
+                node.accept(visitor);
+                ColumnParseNodeVisitor.ContentType type = visitor.getContentType(mainTable);
+                if (type == ColumnParseNodeVisitor.ContentType.NONE 
+                        || type == ColumnParseNodeVisitor.ContentType.SELF_ONLY) {
+                    preFilters.add(node);
+                } else {
+                    postFilters.add(node);
+                }
+                return null;
+            }
+
+            @Override
+            public Void visitLeave(LikeParseNode node,
+                    List<Void> l) throws SQLException {                
+                return leaveBooleanNode(node, l);
+            }
+
+            @Override
+            public boolean visitEnter(AndParseNode node) {
+                return true;
+            }
+            
+            @Override
+            public Void visitLeave(OrParseNode node, List<Void> l)
+                    throws SQLException {
+                return leaveBooleanNode(node, l);
+            }
+
+            @Override
+            public Void visitLeave(ComparisonParseNode node, List<Void> l) 
+                    throws SQLException {
+                return leaveBooleanNode(node, l);
+            }
+
+            @Override
+            public Void visitLeave(NotParseNode node, List<Void> l)
+                    throws SQLException {
+                return leaveBooleanNode(node, l);
+            }
+
+            @Override
+            public Void visitLeave(InListParseNode node,
+                    List<Void> l) throws SQLException {
+                return leaveBooleanNode(node, l);
+            }
+            
+            @Override
+            public Void visitLeave(IsNullParseNode node, List<Void> l) 
+                    throws SQLException {
+                return leaveBooleanNode(node, l);
+            }
+            
+            @Override
+            public Void visitLeave(FunctionParseNode node, List<Void> l) 
+            		throws SQLException {
+            	return leaveBooleanNode(node, l);
+            }
+            
+            @Override
+            public Void visitLeave(BetweenParseNode node, List<Void> l) 
+            		throws SQLException {
+            	return leaveBooleanNode(node, l);
+            }
+            
+            @Override
+            public Void visitLeave(CaseParseNode node, List<Void> l) 
+            		throws SQLException {
+            	return leaveBooleanNode(node, l);
+            }
+            
+            @Override
+            public Void visitLeave(CastParseNode node, List<Void> l) 
+            		throws SQLException {
+            	return leaveBooleanNode(node, l);
+            }			
+        }
+    }
+    
+    public static JoinSpec getSubJoinSpecWithoutPostFilters(JoinSpec join) {
+        return new JoinSpec(join.mainTable, join.select, join.preFilters, new ArrayList<ParseNode>(), 
+                join.joinTables.subList(0, join.joinTables.size() - 1), join.columnRefs);
+    }
+    
+    public static class JoinTable {
+        private JoinType type;
+        private TableNode tableNode; // original table node
+        private TableRef table;
+        private List<AliasedNode> select; // all basic nodes related to this table, no aggregation.
+        private List<ParseNode> preFilters;
+        private List<ParseNode> conditions;
+        private SelectStatement subquery;
+        
+        private Set<TableRef> leftTableRefs;
+        
+        public JoinTable(JoinTableNode node, TableRef tableRef, List<AliasedNode> select, ColumnResolver resolver) throws SQLException {
+            if (!(node.getTable() instanceof ConcreteTableNode))
+                throw new SQLFeatureNotSupportedException("Subqueries not supported.");
+            
+            this.type = node.getType();
+            this.tableNode = node.getTable();
+            this.table = tableRef;
+            this.select = extractFromSelect(select,tableRef,resolver);
+            this.preFilters = new ArrayList<ParseNode>();
+            this.conditions = new ArrayList<ParseNode>();
+            this.leftTableRefs = new HashSet<TableRef>();
+            node.getOnNode().accept(new OnNodeVisitor(resolver));
+        }
+        
+        public JoinType getType() {
+            return type;
+        }
+        
+        public TableNode getTableNode() {
+            return tableNode;
+        }
+        
+        public TableRef getTable() {
+            return table;
+        }
+        
+        public List<AliasedNode> getSelect() {
+            return select;
+        }
+        
+        public List<ParseNode> getPreFilters() {
+            return preFilters;
+        }
+        
+        public List<ParseNode> getJoinConditions() {
+            return conditions;
+        }
+        
+        public SelectStatement getSubquery() {
+            return subquery;
+        }
+        
+        public Set<TableRef> getLeftTableRefs() {
+            return leftTableRefs;
+        }
+        
+        public ParseNode getPreFiltersCombined() {
+            if (preFilters == null || preFilters.isEmpty())
+                return null;
+            
+            if (preFilters.size() == 1)
+                return preFilters.get(0);
+            
+            return NODE_FACTORY.and(preFilters);
+        }
+        
+        public SelectStatement getAsSubquery() {
+            if (subquery != null)
+                return subquery;
+            
+            List<TableNode> from = new ArrayList<TableNode>(1);
+            from.add(tableNode);
+            return NODE_FACTORY.select(from, null, false, select, getPreFiltersCombined(), null, null, null, null, 0, false);
+        }
+        
+        public Pair<List<Expression>, List<Expression>> compileJoinConditions(StatementContext context, ColumnResolver leftResolver, ColumnResolver rightResolver) throws SQLException {
+        	ColumnResolver resolver = context.getResolver();
+            List<Pair<Expression, Expression>> compiled = new ArrayList<Pair<Expression, Expression>>(conditions.size());
+        	context.setResolver(leftResolver);
+            ExpressionCompiler expressionCompiler = new ExpressionCompiler(context);
+            for (ParseNode condition : conditions) {
+                assert (condition instanceof EqualParseNode);
+                EqualParseNode equalNode = (EqualParseNode) condition;
+                expressionCompiler.reset();
+                Expression left = equalNode.getLHS().accept(expressionCompiler);
+                compiled.add(new Pair<Expression, Expression>(left, null));
+            }
+        	context.setResolver(rightResolver);
+            expressionCompiler = new ExpressionCompiler(context);
+            Iterator<Pair<Expression, Expression>> iter = compiled.iterator();
+            for (ParseNode condition : conditions) {
+                Pair<Expression, Expression> p = iter.next();
+                EqualParseNode equalNode = (EqualParseNode) condition;
+                expressionCompiler.reset();
+                Expression right = equalNode.getRHS().accept(expressionCompiler);
+                Expression left = p.getFirst();
+                PDataType toType = getCommonType(left.getDataType(), right.getDataType());
+                if (left.getDataType() != toType) {
+                    left = CoerceExpression.create(left, toType);
+                    p.setFirst(left);
+                }
+                if (right.getDataType() != toType) {
+                    right = CoerceExpression.create(right, toType);
+                }
+                p.setSecond(right);
+            }
+            context.setResolver(resolver); // recover the resolver
+            Collections.sort(compiled, new Comparator<Pair<Expression, Expression>>() {
+                @Override
+                public int compare(Pair<Expression, Expression> o1, Pair<Expression, Expression> o2) {
+                    Expression e1 = o1.getFirst();
+                    Expression e2 = o2.getFirst();
+                    boolean isFixed1 = e1.getDataType().isFixedWidth();
+                    boolean isFixed2 = e2.getDataType().isFixedWidth();
+                    boolean isFixedNullable1 = e1.isNullable() &&isFixed1;
+                    boolean isFixedNullable2 = e2.isNullable() && isFixed2;
+                    if (isFixedNullable1 == isFixedNullable2) {
+                        if (isFixed1 == isFixed2) {
+                            return 0;
+                        } else if (isFixed1) {
+                            return -1;
+                        } else {
+                            return 1;
+                        }
+                    } else if (isFixedNullable1) {
+                        return 1;
+                    } else {
+                        return -1;
+                    }
+                }
+            });
+            List<Expression> lConditions = new ArrayList<Expression>(compiled.size());
+            List<Expression> rConditions = new ArrayList<Expression>(compiled.size());
+            for (Pair<Expression, Expression> pair : compiled) {
+                lConditions.add(pair.getFirst());
+                rConditions.add(pair.getSecond());
+            }
+            
+            return new Pair<List<Expression>, List<Expression>>(lConditions, rConditions);
+        }
+        
+        private PDataType getCommonType(PDataType lType, PDataType rType) throws SQLException {
+            if (lType == rType)
+                return lType;
+            
+            if (!lType.isComparableTo(rType))
+                throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_CONVERT_TYPE)
+                    .setMessage("On-clause LHS expression and RHS expression must be comparable. LHS type: " + lType + ", RHS type: " + rType)
+                    .build().buildException();
+
+            if ((lType == null || lType.isCoercibleTo(PDataType.TINYINT))
+                    && (rType == null || rType.isCoercibleTo(PDataType.TINYINT))) {
+                return lType == null ? rType : lType; // to preserve UNSIGNED type
+            }
+
+            if ((lType == null || lType.isCoercibleTo(PDataType.SMALLINT))
+                    && (rType == null || rType.isCoercibleTo(PDataType.SMALLINT))) {
+                return lType == null ? rType : lType; // to preserve UNSIGNED type
+            }
+
+            if ((lType == null || lType.isCoercibleTo(PDataType.INTEGER))
+                    && (rType == null || rType.isCoercibleTo(PDataType.INTEGER))) {
+                return lType == null ? rType : lType; // to preserve UNSIGNED type
+            }
+
+            if ((lType == null || lType.isCoercibleTo(PDataType.LONG))
+                    && (rType == null || rType.isCoercibleTo(PDataType.LONG))) {
+                return lType == null ? rType : lType; // to preserve UNSIGNED type
+            }
+
+            if ((lType == null || lType.isCoercibleTo(PDataType.DECIMAL))
+                    && (rType == null || rType.isCoercibleTo(PDataType.DECIMAL))) {
+                return PDataType.DECIMAL;
+            }
+
+            if ((lType == null || lType.isCoercibleTo(PDataType.DATE))
+                    && (rType == null || rType.isCoercibleTo(PDataType.DATE))) {
+                return lType == null ? rType : lType;
+            }
+
+            if ((lType == null || lType.isCoercibleTo(PDataType.TIMESTAMP))
+                    && (rType == null || rType.isCoercibleTo(PDataType.TIMESTAMP))) {
+                return lType == null ? rType : lType;
+            }
+
+            if ((lType == null || lType.isCoercibleTo(PDataType.VARCHAR))
+                    && (rType == null || rType.isCoercibleTo(PDataType.VARCHAR))) {
+                return PDataType.VARCHAR;
+            }
+
+            if ((lType == null || lType.isCoercibleTo(PDataType.BOOLEAN))
+                    && (rType == null || rType.isCoercibleTo(PDataType.BOOLEAN))) {
+                return PDataType.BOOLEAN;
+            }
+
+            return PDataType.VARBINARY;
+        }
+        
+        private class OnNodeVisitor  extends TraverseNoParseNodeVisitor<Void> {
+            private ColumnResolver resolver;
+            
+            public OnNodeVisitor(ColumnResolver resolver) {
+                this.resolver = resolver;
+            }
+            
+            private Void leaveNonEqBooleanNode(ParseNode node,
+                    List<Void> l) throws SQLException {
+                ColumnParseNodeVisitor visitor = new ColumnParseNodeVisitor(resolver);
+                node.accept(visitor);
+                ColumnParseNodeVisitor.ContentType type = visitor.getContentType(table);
+                if (type == ColumnParseNodeVisitor.ContentType.NONE 
+                        || type == ColumnParseNodeVisitor.ContentType.SELF_ONLY) {
+                    preFilters.add(node);
+                } else {
+                    throwUnsupportedJoinConditionException();
+                }
+                return null;
+            }
+
+            @Override
+            public Void visitLeave(LikeParseNode node,
+                    List<Void> l) throws SQLException {                
+                return leaveNonEqBooleanNode(node, l);
+            }
+
+            @Override
+            public boolean visitEnter(AndParseNode node) {
+                return true;
+            }
+            
+            @Override
+            public Void visitLeave(OrParseNode node, List<Void> l)
+                    throws SQLException {
+                return leaveNonEqBooleanNode(node, l);
+            }
+
+            @Override
+            public Void visitLeave(ComparisonParseNode node, List<Void> l) 
+                    throws SQLException {
+                if (!(node instanceof EqualParseNode))
+                    return leaveNonEqBooleanNode(node, l);
+                ColumnParseNodeVisitor lhsVisitor = new ColumnParseNodeVisitor(resolver);
+                ColumnParseNodeVisitor rhsVisitor = new ColumnParseNodeVisitor(resolver);
+                node.getLHS().accept(lhsVisitor);
+                node.getRHS().accept(rhsVisitor);
+                ColumnParseNodeVisitor.ContentType lhsType = lhsVisitor.getContentType(table);
+                ColumnParseNodeVisitor.ContentType rhsType = rhsVisitor.getContentType(table);
+                if ((lhsType == ColumnParseNodeVisitor.ContentType.SELF_ONLY || lhsType == ColumnParseNodeVisitor.ContentType.NONE)
+                		&& (rhsType == ColumnParseNodeVisitor.ContentType.SELF_ONLY || rhsType == ColumnParseNodeVisitor.ContentType.NONE)) {
+                    preFilters.add(node);
+                } else if (lhsType == ColumnParseNodeVisitor.ContentType.FOREIGN_ONLY 
+                		&& rhsType == ColumnParseNodeVisitor.ContentType.SELF_ONLY) {
+                    conditions.add(node);
+                    leftTableRefs.addAll(lhsVisitor.getTableRefSet());
+                } else if (rhsType == ColumnParseNodeVisitor.ContentType.FOREIGN_ONLY 
+                		&& lhsType == ColumnParseNodeVisitor.ContentType.SELF_ONLY) {
+                    conditions.add(NODE_FACTORY.equal(node.getRHS(), node.getLHS()));
+                    leftTableRefs.addAll(rhsVisitor.getTableRefSet());
+                } else {
+                	throwUnsupportedJoinConditionException();
+                }
+                return null;
+            }
+
+            @Override
+            public Void visitLeave(NotParseNode node, List<Void> l)
+                    throws SQLException {
+                return leaveNonEqBooleanNode(node, l);
+            }
+
+            @Override
+            public Void visitLeave(InListParseNode node,
+                    List<Void> l) throws SQLException {
+                return leaveNonEqBooleanNode(node, l);
+            }
+            
+            @Override
+            public Void visitLeave(IsNullParseNode node, List<Void> l) 
+                    throws SQLException {
+                return leaveNonEqBooleanNode(node, l);
+            }
+            
+            @Override
+            public Void visitLeave(FunctionParseNode node, List<Void> l) 
+            		throws SQLException {
+            	return leaveNonEqBooleanNode(node, l);
+            }
+            
+            @Override
+            public Void visitLeave(BetweenParseNode node, List<Void> l) 
+            		throws SQLException {
+            	return leaveNonEqBooleanNode(node, l);
+            }
+            
+            @Override
+            public Void visitLeave(CaseParseNode node, List<Void> l) 
+            		throws SQLException {
+            	return leaveNonEqBooleanNode(node, l);
+            }
+            
+            @Override
+            public Void visitLeave(CastParseNode node, List<Void> l) 
+            		throws SQLException {
+            	return leaveNonEqBooleanNode(node, l);
+            }
+
+            /*
+             * Conditions in the ON clause can only be:
+             * 1) an equal test between a self table expression and a foreign 
+             *    table expression.
+             * 2) a boolean condition referencing to the self table only.
+             * Otherwise, it can be ambiguous.
+             */
+            private void throwUnsupportedJoinConditionException() 
+            		throws SQLFeatureNotSupportedException {
+            	throw new SQLFeatureNotSupportedException("Does not support non-standard or non-equi join conditions.");
+            }			
+        }
+    }
+    
+    private static class ColumnParseNodeVisitor  extends StatelessTraverseAllParseNodeVisitor {
+        public enum ContentType {NONE, SELF_ONLY, FOREIGN_ONLY, COMPLEX};
+        
+        private ColumnResolver resolver;
+        private final Set<TableRef> tableRefSet;
+        private final Map<ColumnRef, ColumnParseNode> columnRefMap;
+       
+        public ColumnParseNodeVisitor(ColumnResolver resolver) {
+            this.resolver = resolver;
+            this.tableRefSet = new HashSet<TableRef>();
+            this.columnRefMap = new HashMap<ColumnRef, ColumnParseNode>();
+        }
+        
+        public void reset() {
+            this.tableRefSet.clear();
+            this.columnRefMap.clear();
+        }
+        
+        @Override
+        public Void visit(ColumnParseNode node) throws SQLException {
+            ColumnRef columnRef = resolver.resolveColumn(node.getSchemaName(), node.getTableName(), node.getName());
+            columnRefMap.put(columnRef, node);
+            tableRefSet.add(columnRef.getTableRef());
+            return null;                
+        }
+        
+        public Set<TableRef> getTableRefSet() {
+            return tableRefSet;
+        }
+        
+        public Map<ColumnRef, ColumnParseNode> getColumnRefMap() {
+            return columnRefMap;
+        }
+        
+        public ContentType getContentType(TableRef selfTable) {
+            if (tableRefSet.isEmpty())
+                return ContentType.NONE;
+            if (tableRefSet.size() > 1)
+                return ContentType.COMPLEX;
+            if (tableRefSet.contains(selfTable))
+                return ContentType.SELF_ONLY;
+            return ContentType.FOREIGN_ONLY;
+        }
+    }
+    
+    private static String PROJECTED_TABLE_SCHEMA = ".";
+    // for creation of new statements
+    private static ParseNodeFactory NODE_FACTORY = new ParseNodeFactory();
+    
+    private static boolean isWildCardSelect(List<AliasedNode> select) {
+        return (select.size() == 1 && select.get(0).getNode() == WildcardParseNode.INSTANCE);
+    }
+    
+    private static List<AliasedNode> extractFromSelect(List<AliasedNode> select, TableRef table, ColumnResolver resolver) throws SQLException {
+        List<AliasedNode> ret = new ArrayList<AliasedNode>();
+        if (isWildCardSelect(select)) {
+            ret.add(NODE_FACTORY.aliasedNode(null, WildcardParseNode.INSTANCE));
+            return ret;
+        }
+        
+        ColumnParseNodeVisitor visitor = new ColumnParseNodeVisitor(resolver);
+        for (AliasedNode node : select) {
+            node.getNode().accept(visitor);
+            ColumnParseNodeVisitor.ContentType type = visitor.getContentType(table);
+            if (type == ColumnParseNodeVisitor.ContentType.SELF_ONLY) {
+                ret.add(node);
+            } else if (type == ColumnParseNodeVisitor.ContentType.COMPLEX) {
+                for (Map.Entry<ColumnRef, ColumnParseNode> entry : visitor.getColumnRefMap().entrySet()) {
+                    if (entry.getKey().getTableRef().equals(table)) {
+                        ret.add(NODE_FACTORY.aliasedNode(null, entry.getValue()));
+                    }
+                }
+            }
+            visitor.reset();
+        }
+        return ret;
+    }
+    
+    public static JoinSpec getJoinSpec(StatementContext context, SelectStatement statement) throws SQLException {
+        return new JoinSpec(statement, context.getResolver());
+    }
+    
+    public static SelectStatement optimize(StatementContext context, SelectStatement select, PhoenixStatement statement) throws SQLException {
+        ColumnResolver resolver = context.getResolver();
+        JoinSpec join = new JoinSpec(select, resolver);
+        Map<TableRef, TableRef> replacement = new HashMap<TableRef, TableRef>();
+        List<TableNode> from = select.getFrom();
+        List<TableNode> newFrom = Lists.newArrayListWithExpectedSize(from.size());
+
+        class TableNodeRewriter implements TableNodeVisitor {
+            private TableRef table;
+            private TableNode replaced;
+            
+            TableNodeRewriter(TableRef table) {
+                this.table = table;
+            }
+            
+            public TableNode getReplacedTableNode() {
+                return replaced;
+            }
+
+            @Override
+            public void visit(BindTableNode boundTableNode) throws SQLException {
+                replaced = NODE_FACTORY.bindTable(boundTableNode.getAlias(), getReplacedTableName());
+            }
+
+            @Override
+            public void visit(JoinTableNode joinNode) throws SQLException {
+                joinNode.getTable().accept(this);
+                replaced = NODE_FACTORY.join(joinNode.getType(), joinNode.getOnNode(), replaced);
+            }
+
+            @Override
+            public void visit(NamedTableNode namedTableNode)
+                    throws SQLException {
+                replaced = NODE_FACTORY.namedTable(namedTableNode.getAlias(), getReplacedTableName(), namedTableNode.getDynamicColumns());
+            }
+
+            @Override
+            public void visit(DerivedTableNode subselectNode)
+                    throws SQLException {
+                throw new SQLFeatureNotSupportedException();
+            }
+            
+            private TableName getReplacedTableName() {
+                String schemaName = table.getTable().getSchemaName().getString();
+                schemaName = schemaName.length() == 0 ? null : '"' + schemaName + '"';
+                String tableName = '"' + table.getTable().getTableName().getString() + '"';
+                return NODE_FACTORY.table(schemaName, tableName);
+            }
+        };
+        
+        // get optimized plans for join tables
+        for (int i = 1; i < from.size(); i++) {
+            TableNode jNode = from.get(i);
+            assert (jNode instanceof JoinTableNode);
+            TableNode tNode = ((JoinTableNode) jNode).getTable();
+            for (JoinTable jTable : join.getJoinTables()) {
+                if (jTable.getTableNode() != tNode)
+                    continue;
+                TableRef table = jTable.getTable();
+                SelectStatement stmt = getSubqueryForOptimizedPlan(select, table, join.columnRefs, jTable.getPreFiltersCombined());
+                QueryPlan plan = context.getConnection().getQueryServices().getOptimizer().optimize(stmt, statement);
+                if (!plan.getTableRef().equals(table)) {
+                    TableNodeRewriter rewriter = new TableNodeRewriter(plan.getTableRef());
+                    jNode.accept(rewriter);
+                    newFrom.add(rewriter.getReplacedTableNode());
+                    replacement.put(table, plan.getTableRef());
+                } else {
+                    newFrom.add(jNode);
+                }
+            }
+        }
+        // get optimized plan for main table
+        TableRef table = join.getMainTable();
+        SelectStatement stmt = getSubqueryForOptimizedPlan(select, table, join.columnRefs, join.getPreFiltersCombined());
+        QueryPlan plan = context.getConnection().getQueryServices().getOptimizer().optimize(stmt, statement);
+        if (!plan.getTableRef().equals(table)) {
+            TableNodeRewriter rewriter = new TableNodeRewriter(plan.getTableRef());
+            from.get(0).accept(rewriter);
+            newFrom.add(0, rewriter.getReplacedTableNode());
+            replacement.put(table, plan.getTableRef());            
+        } else {
+            newFrom.add(0, from.get(0));
+        }
+        
+        if (replacement.isEmpty()) 
+            return select;
+        
+        return IndexStatementRewriter.translate(NODE_FACTORY.select(select, newFrom), resolver, replacement);        
+    }
+    
+    private static SelectStatement getSubqueryForOptimizedPlan(SelectStatement select, TableRef table, Map<ColumnRef, ColumnRefType> columnRefs, ParseNode where) {
+        TableName tName = NODE_FACTORY.table(table.getTable().getSchemaName().getString(), table.getTable().getTableName().getString());
+        List<AliasedNode> selectList = new ArrayList<AliasedNode>();
+        if (isWildCardSelect(select.getSelect())) {
+            selectList.add(NODE_FACTORY.aliasedNode(null, WildcardParseNode.INSTANCE));
+        } else {
+            for (ColumnRef colRef : columnRefs.keySet()) {
+                if (colRef.getTableRef().equals(table)) {
+                    selectList.add(NODE_FACTORY.aliasedNode(null, NODE_FACTORY.column(tName, '"' + colRef.getColumn().getName().getString() + '"', null)));
+                }
+            }
+        }
+        List<? extends TableNode> from = Collections.singletonList(NODE_FACTORY.namedTable(table.getTableAlias(), tName));
+
+        return NODE_FACTORY.select(from, select.getHint(), false, selectList, where, null, null, null, null, 0, false);
+    }
+    
+    /**
+     * Returns a boolean vector indicating whether the evaluation of join expressions
+     * can be evaluated at an early stage if the input JoinSpec can be taken as a
+     * star join. Otherwise returns null.  
+     * @param join the JoinSpec
+     * @return a boolean vector for a star join; or null for non star join.
+     */
+    public static boolean[] getStarJoinVector(JoinSpec join) {
+        assert(!join.getJoinTables().isEmpty());
+        
+        int count = join.getJoinTables().size();
+        boolean[] vector = new boolean[count];
+        for (int i = 0; i < count; i++) {
+        	JoinTable joinTable = join.getJoinTables().get(i);
+            if (joinTable.getType() != JoinType.Left 
+                    && joinTable.getType() != JoinType.Inner)
+                return null;
+            vector[i] = true;
+            Iterator<TableRef> iter = joinTable.getLeftTableRefs().iterator();
+            while (vector[i] == true && iter.hasNext()) {
+            	TableRef tableRef = iter.next();
+                if (!tableRef.equals(join.getMainTable())) {
+                    vector[i] = false;
+                }
+            }
+        }
+        
+        return vector;
+    }
+    
+    public static SelectStatement getSubqueryWithoutJoin(SelectStatement statement, JoinSpec join) {
+        return NODE_FACTORY.select(statement.getFrom().subList(0, 1), statement.getHint(), statement.isDistinct(), statement.getSelect(), join.getPreFiltersCombined(), statement.getGroupBy(), statement.getHaving(), statement.getOrderBy(), statement.getLimit(), statement.getBindCount(), statement.isAggregate());
+    }
+    
+    // Get the last join table select statement with fixed-up select and where nodes.
+    // Currently does NOT support last join table as a subquery.
+    public static SelectStatement getSubqueryForLastJoinTable(SelectStatement statement, JoinSpec join) throws SQLException {
+        List<JoinTable> joinTables = join.getJoinTables();
+        int count = joinTables.size();
+        assert (count > 0);
+        JoinTable lastJoinTable = joinTables.get(count - 1);
+        if (lastJoinTable.getSubquery() != null) {
+            throw new SQLFeatureNotSupportedException("Subqueries not supported.");
+        }
+        List<TableNode> from = new ArrayList<TableNode>(1);
+        from.add(lastJoinTable.getTableNode());
+        
+        return NODE_FACTORY.select(from, statement.getHint(), statement.isDistinct(), statement.getSelect(), lastJoinTable.getPreFiltersCombined(), statement.getGroupBy(), statement.getHaving(), statement.getOrderBy(), statement.getLimit(), statement.getBindCount(), statement.isAggregate());
+    }
+    
+    // Get subquery with fixed select and where nodes
+    public static SelectStatement getSubQueryWithoutLastJoin(SelectStatement statement, JoinSpec join) {
+        List<TableNode> from = statement.getFrom();
+        assert(from.size() > 1);
+        List<JoinTable> joinTables = join.getJoinTables();
+        int count = joinTables.size();
+        assert (count > 0);
+        List<AliasedNode> select = new ArrayList<AliasedNode>();
+        select.addAll(join.getSelect());
+        for (int i = 0; i < count - 1; i++) {
+            select.addAll(joinTables.get(i).getSelect());
+        }
+        
+        return NODE_FACTORY.select(from.subList(0, from.size() - 1), statement.getHint(), false, select, join.getPreFiltersCombined(), null, null, null, null, statement.getBindCount(), false);
+    }
+    
+    public static PTableWrapper mergeProjectedTables(PTableWrapper lWrapper, PTableWrapper rWrapper, boolean innerJoin) throws SQLException {
+    	PTable left = lWrapper.getTable();
+    	PTable right = rWrapper.getTable();
+    	List<PColumn> merged = new ArrayList<PColumn>();
+    	merged.addAll(left.getColumns());
+    	int position = merged.size();
+    	for (PColumn c : right.getColumns()) {
+    		if (!SchemaUtil.isPKColumn(c)) {
+    			PColumnImpl column = new PColumnImpl(c.getName(), 
+    					PNameFactory.newName(ScanProjector.VALUE_COLUMN_FAMILY), c.getDataType(), 
+    					c.getMaxLength(), c.getScale(), innerJoin ? c.isNullable() : true, position++, 
+    					c.getColumnModifier(), c.getArraySize());
+    			merged.add(column);
+    		}
+    	}
+        if (left.getBucketNum() != null) {
+            merged.remove(0);
+        }
+        PTable t = PTableImpl.makePTable(left.getSchemaName(), PNameFactory.newName(SchemaUtil.getTableName(left.getName().getString(), right.getName().getString())),
+                left.getType(), left.getIndexState(), left.getTimeStamp(), left.getSequenceNumber(), left.getPKName(), left.getBucketNum(), merged, left.getParentTableName(),
+                left.getIndexes(), left.isImmutableRows(), Collections.<PName>emptyList(), null, null, PTable.DEFAULT_DISABLE_WAL, left.isMultiTenant(), left.getViewType());
+
+        ListMultimap<String, String> mergedMap = ArrayListMultimap.<String, String>create();
+        mergedMap.putAll(lWrapper.getColumnNameMap());
+        mergedMap.putAll(rWrapper.getColumnNameMap());
+        
+        return new PTableWrapper(t, mergedMap);
+    }
+    
+    public static ScanProjector getScanProjector(ProjectedPTableWrapper table) {
+    	return new ScanProjector(table);
+    }
+    
+    public static class PTableWrapper {
+    	protected PTable table;
+    	protected ListMultimap<String, String> columnNameMap;
+    	
+    	protected PTableWrapper(PTable table, ListMultimap<String, String> columnNameMap) {
+    		this.table = table;
+    		this.columnNameMap = columnNameMap;
+    	}
+    	
+    	public PTable getTable() {
+    		return table;
+    	}
+    	
+    	public ListMultimap<String, String> getColumnNameMap() {
+    		return columnNameMap;
+    	}
+
+    	public List<String> getMappedColumnName(String name) {
+    		return columnNameMap.get(name);
+    	}
+    }
+    
+    public static class ProjectedPTableWrapper extends PTableWrapper {
+    	private List<Expression> sourceExpressions;
+    	
+    	protected ProjectedPTableWrapper(PTable table, ListMultimap<String, String> columnNameMap, List<Expression> sourceExpressions) {
+    		super(table, columnNameMap);
+    		this.sourceExpressions = sourceExpressions;
+    	}
+    	
+    	public Expression getSourceExpression(PColumn column) {
+    		return sourceExpressions.get(column.getPosition() - (table.getBucketNum() == null ? 0 : 1));
+    	}
+    }
+    
+    public static ColumnResolver getColumnResolver(PTableWrapper table) {
+    	return new JoinedTableColumnResolver(table);
+    }
+    
+    public static class JoinedTableColumnResolver implements ColumnResolver {
+    	private PTableWrapper table;
+    	private List<TableRef> tableRefs;
+    	
+    	private JoinedTableColumnResolver(PTableWrapper table) {
+    		this.table = table;
+    		TableRef tableRef = new TableRef(null, table.getTable(), 0, false);
+    		this.tableRefs = ImmutableList.of(tableRef);
+    	}
+
+		@Override
+		public List<TableRef> getTables() {
+			return tableRefs;
+		}
+		
+		public PTableWrapper getPTableWrapper() {
+			return table;
+		}
+
+		@Override
+		public ColumnRef resolveColumn(String schemaName, String tableName,
+				String colName) throws SQLException {
+			String name = getProjectedColumnName(schemaName, tableName, colName);
+			TableRef tableRef = tableRefs.get(0);
+			try {
+				PColumn column = tableRef.getTable().getColumn(name);
+				return new ColumnRef(tableRef, column.getPosition());
+			} catch (ColumnNotFoundException e) {
+				List<String> names = table.getMappedColumnName(name);
+				if (names.size() == 1) {
+					PColumn column = tableRef.getTable().getColumn(names.get(0));
+					return new ColumnRef(tableRef, column.getPosition());					
+				}
+				
+				if (names.size() > 1) {
+					throw new AmbiguousColumnException(name);
+				}
+				
+				throw e;
+			}
+		}
+    }
+    
+    private static String getProjectedColumnName(String schemaName, String tableName,
+			String colName) {
+    	return SchemaUtil.getColumnName(SchemaUtil.getTableName(schemaName, tableName), colName);
+    }
+    
+    private static PName getProjectedColumnName(PName schemaName, PName tableName,
+    		PName colName) {
+    	String name = getProjectedColumnName(schemaName.getString(), tableName.getString(), colName.getString());
+    	return PNameFactory.newName(name);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/compile/KeyPart.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/KeyPart.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/KeyPart.java
new file mode 100644
index 0000000..1c8b0eb
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/KeyPart.java
@@ -0,0 +1,79 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.compile;
+
+import java.util.List;
+
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.query.KeyRange;
+import org.apache.phoenix.schema.PColumn;
+
+/**
+ * 
+ * Interface that determines how a key part contributes to
+ * the forming of the key (start/stop of scan and SkipScanFilter)
+ * for each part of a multi-part primary key. It acts as the glue
+ * between a built-in function and the setting of the scan key
+ * during query compilation.
+ * 
+ * @author jtaylor
+ * @since 0.12
+ */
+public interface KeyPart {
+    /**
+     * Calculate the key range given an operator and the key on
+     * the RHS of an expression. For example, given the expression
+     * SUBSTR(foo,1,3) = 'bar', the key range would be ['bar','bas'),
+     * and if foo was fixed length, the upper and lower key range
+     * bytes would be filled out to the fixed length.
+     * @param op comparison operator (=, <=, <, >=, >, !=)
+     * @param rhs the constant on the RHS of an expression.
+     * @return the key range that encompasses the range for the
+     *  expression for which this keyPart is associated
+     *  
+     * @see org.apache.phoenix.expression.function.ScalarFunction#newKeyPart(KeyPart)
+     */
+    public KeyRange getKeyRange(CompareOp op, Expression rhs);
+    
+    /**
+     * Determines whether an expression gets extracted from the
+     * WHERE clause if it contributes toward the building of the
+     * scan key. For example, the SUBSTR built-in function may
+     * be extracted, since it may be completely represented
+     * through a key range. However, the REGEXP_SUBSTR must be
+     * left in the WHERE clause, since only the constant prefix
+     * part of the evaluation can be represented through a key
+     * range (i.e. rows may pass through that will fail when
+     * the REGEXP_SUBSTR is evaluated).
+     * 
+     * @return an empty list if the expression should remain in
+     * the WHEERE clause for post filtering or a singleton list
+     * containing the expression if it should be removed.
+     */
+    public List<Expression> getExtractNodes();
+    
+    /**
+     * Gets the primary key column associated with the start of this key part.
+     * @return the primary key column for this key part
+     */
+    public PColumn getColumn();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/compile/LimitCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/LimitCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/LimitCompiler.java
new file mode 100644
index 0000000..5c4cbb2
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/LimitCompiler.java
@@ -0,0 +1,110 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.compile;
+
+import java.sql.SQLException;
+
+import org.apache.phoenix.parse.*;
+import org.apache.phoenix.schema.*;
+
+
+public class LimitCompiler {
+    private static final ParseNodeFactory NODE_FACTORY = new ParseNodeFactory();
+    
+    public static final PDatum LIMIT_DATUM = new PDatum() {
+        @Override
+        public boolean isNullable() {
+            return false;
+        }
+        @Override
+        public PDataType getDataType() {
+            return PDataType.INTEGER;
+        }
+        @Override
+        public Integer getByteSize() {
+            return getDataType().getByteSize();
+        }
+        @Override
+        public Integer getMaxLength() {
+            return null;
+        }
+        @Override
+        public Integer getScale() {
+            return null;
+        }
+		@Override
+		public ColumnModifier getColumnModifier() {
+			return null;
+		}
+    };
+    
+    private LimitCompiler() {
+    }
+
+    public static Integer compile(StatementContext context, FilterableStatement statement) throws SQLException {
+        LimitNode limitNode = statement.getLimit();
+        if (limitNode == null) {
+            return null;
+        }
+        LimitParseNodeVisitor visitor = new LimitParseNodeVisitor(context);
+        limitNode.getLimitParseNode().accept(visitor);
+        return visitor.getLimit();
+    }
+    
+    private static class LimitParseNodeVisitor extends TraverseNoParseNodeVisitor<Void> {
+        private final StatementContext context;
+        private Integer limit;
+        
+        private LimitParseNodeVisitor(StatementContext context) {
+            this.context = context;
+        }
+        
+        public Integer getLimit() {
+            return limit;
+        }
+        
+        @Override
+        public Void visit(LiteralParseNode node) throws SQLException {
+            Object limitValue = node.getValue();
+            // If limit is null, leave this.limit set to zero
+            // This means that we've bound limit to null for the purpose of
+            // collecting parameter metadata.
+            if (limitValue != null) {
+                Integer limit = (Integer)LIMIT_DATUM.getDataType().toObject(limitValue, node.getType());
+                if (limit.intValue() >= 0) { // TODO: handle LIMIT 0
+                    this.limit = limit;
+                }
+            }
+            return null;
+        }
+    
+        @Override
+        public Void visit(BindParseNode node) throws SQLException {
+            Object value = context.getBindManager().getBindValue(node);
+            context.getBindManager().addParamMetaData(node, LIMIT_DATUM);
+            // Resolve the bind value, create a LiteralParseNode, and call the visit method for it.
+            // In this way, we can deal with just having a literal on one side of the expression.
+            visit(NODE_FACTORY.literal(value, LIMIT_DATUM.getDataType()));
+            return null;
+        }
+		
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/compile/MutatingParallelIteratorFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/MutatingParallelIteratorFactory.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/MutatingParallelIteratorFactory.java
new file mode 100644
index 0000000..b19af43
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/MutatingParallelIteratorFactory.java
@@ -0,0 +1,115 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.compile;
+
+import static org.apache.phoenix.query.QueryConstants.AGG_TIMESTAMP;
+import static org.apache.phoenix.query.QueryConstants.SINGLE_COLUMN;
+import static org.apache.phoenix.query.QueryConstants.SINGLE_COLUMN_FAMILY;
+import static org.apache.phoenix.query.QueryConstants.UNGROUPED_AGG_ROW_KEY;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.KeyValue;
+
+import org.apache.phoenix.execute.MutationState;
+import org.apache.phoenix.iterate.ParallelIterators.ParallelIteratorFactory;
+import org.apache.phoenix.iterate.PeekingResultIterator;
+import org.apache.phoenix.iterate.ResultIterator;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.query.ConnectionQueryServices;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.schema.tuple.SingleKeyValueTuple;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.KeyValueUtil;
+
+/**
+ * Factory class used to instantiate an iterator to handle mutations made during a parallel scan.
+ */
+public abstract class MutatingParallelIteratorFactory implements ParallelIteratorFactory {
+    protected final PhoenixConnection connection;
+    protected final TableRef tableRef;
+
+    protected MutatingParallelIteratorFactory(PhoenixConnection connection, TableRef tableRef) {
+        this.connection = connection;
+        this.tableRef = tableRef;
+    }
+    
+    /**
+     * Method that does the actual mutation work
+     */
+    abstract protected MutationState mutate(PhoenixConnection connection, ResultIterator iterator) throws SQLException;
+    
+    @Override
+    public PeekingResultIterator newIterator(ResultIterator iterator) throws SQLException {
+        // Clone the connection as it's not thread safe and will be operated on in parallel
+        final PhoenixConnection connection = new PhoenixConnection(this.connection);
+        MutationState state = mutate(connection, iterator);
+        long totalRowCount = state.getUpdateCount();
+        if (connection.getAutoCommit()) {
+            connection.getMutationState().join(state);
+            connection.commit();
+            ConnectionQueryServices services = connection.getQueryServices();
+            int maxSize = services.getProps().getInt(QueryServices.MAX_MUTATION_SIZE_ATTRIB,QueryServicesOptions.DEFAULT_MAX_MUTATION_SIZE);
+            state = new MutationState(maxSize, connection, totalRowCount);
+        }
+        final MutationState finalState = state;
+        byte[] value = PDataType.LONG.toBytes(totalRowCount);
+        KeyValue keyValue = KeyValueUtil.newKeyValue(UNGROUPED_AGG_ROW_KEY, SINGLE_COLUMN_FAMILY, SINGLE_COLUMN, AGG_TIMESTAMP, value, 0, value.length);
+        final Tuple tuple = new SingleKeyValueTuple(keyValue);
+        return new PeekingResultIterator() {
+            private boolean done = false;
+            
+            @Override
+            public Tuple next() throws SQLException {
+                if (done) {
+                    return null;
+                }
+                done = true;
+                return tuple;
+            }
+
+            @Override
+            public void explain(List<String> planSteps) {
+            }
+
+            @Override
+            public void close() throws SQLException {
+                try {
+                    // Join the child mutation states in close, since this is called in a single threaded manner
+                    // after the parallel results have been processed.
+                    if (!connection.getAutoCommit()) {
+                        MutatingParallelIteratorFactory.this.connection.getMutationState().join(finalState);
+                    }
+                } finally {
+                    connection.close();
+                }
+            }
+
+            @Override
+            public Tuple peek() throws SQLException {
+                return done ? null : tuple;
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/compile/MutationPlan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/MutationPlan.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/MutationPlan.java
new file mode 100644
index 0000000..11fd2cd
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/MutationPlan.java
@@ -0,0 +1,31 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.compile;
+
+import java.sql.SQLException;
+
+import org.apache.phoenix.execute.MutationState;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+
+
+public interface MutationPlan extends StatementPlan {
+    public PhoenixConnection getConnection();
+    public MutationState execute() throws SQLException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/compile/OrderByCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/OrderByCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/OrderByCompiler.java
new file mode 100644
index 0000000..61c0fcc
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/OrderByCompiler.java
@@ -0,0 +1,138 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.compile;
+
+
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.LinkedHashSet;
+import java.util.List;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.apache.phoenix.compile.GroupByCompiler.GroupBy;
+import org.apache.phoenix.compile.TrackOrderPreservingExpressionCompiler.Ordering;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.OrderByExpression;
+import org.apache.phoenix.parse.FilterableStatement;
+import org.apache.phoenix.parse.OrderByNode;
+import org.apache.phoenix.query.ConnectionQueryServices.Feature;
+import org.apache.phoenix.schema.ColumnModifier;
+
+/**
+ * Validates ORDER BY clause and builds up a list of referenced columns.
+ * 
+ * @author syyang
+ * @since 0.1
+ */
+public class OrderByCompiler {
+    public static class OrderBy {
+        public static final OrderBy EMPTY_ORDER_BY = new OrderBy(Collections.<OrderByExpression>emptyList());
+        /**
+         * Used to indicate that there was an ORDER BY, but it was optimized out because
+         * rows are already returned in this order. 
+         */
+        public static final OrderBy FWD_ROW_KEY_ORDER_BY = new OrderBy(Collections.<OrderByExpression>emptyList());
+        public static final OrderBy REV_ROW_KEY_ORDER_BY = new OrderBy(Collections.<OrderByExpression>emptyList());
+        
+        private final List<OrderByExpression> orderByExpressions;
+        
+        private OrderBy(List<OrderByExpression> orderByExpressions) {
+            this.orderByExpressions = ImmutableList.copyOf(orderByExpressions);
+        }
+
+        public List<OrderByExpression> getOrderByExpressions() {
+            return orderByExpressions;
+        }
+    }
+    /**
+     * Gets a list of columns in the ORDER BY clause
+     * @param context the query context for tracking various states
+     * associated with the given select statement
+     * @param statement TODO
+     * @param groupBy the list of columns in the GROUP BY clause
+     * @param limit the row limit or null if no limit
+     * @return the compiled ORDER BY clause
+     * @throws SQLException
+     */
+    public static OrderBy compile(StatementContext context,
+                                  FilterableStatement statement,
+                                  GroupBy groupBy, Integer limit) throws SQLException {
+        List<OrderByNode> orderByNodes = statement.getOrderBy();
+        if (orderByNodes.isEmpty()) {
+            return OrderBy.EMPTY_ORDER_BY;
+        }
+        // accumulate columns in ORDER BY
+        TrackOrderPreservingExpressionCompiler visitor = 
+                new TrackOrderPreservingExpressionCompiler(context, groupBy, 
+                        orderByNodes.size(), Ordering.ORDERED);
+        LinkedHashSet<OrderByExpression> orderByExpressions = Sets.newLinkedHashSetWithExpectedSize(orderByNodes.size());
+        for (OrderByNode node : orderByNodes) {
+            boolean isAscending = node.isAscending();
+            Expression expression = node.getNode().accept(visitor);
+            if (!expression.isStateless() && visitor.addEntry(expression, isAscending ? null : ColumnModifier.SORT_DESC)) {
+                // Detect mix of aggregate and non aggregates (i.e. ORDER BY txns, SUM(txns)
+                if (!visitor.isAggregate()) {
+                    if (statement.isAggregate() || statement.isDistinct()) {
+                        // Detect ORDER BY not in SELECT DISTINCT: SELECT DISTINCT count(*) FROM t ORDER BY x
+                        if (statement.isDistinct()) {
+                            throw new SQLExceptionInfo.Builder(SQLExceptionCode.ORDER_BY_NOT_IN_SELECT_DISTINCT)
+                            .setMessage(expression.toString()).build().buildException();
+                        }
+                        ExpressionCompiler.throwNonAggExpressionInAggException(expression.toString());
+                    }
+                }
+                if (expression.getDataType().isArrayType()) {
+                    throw new SQLExceptionInfo.Builder(SQLExceptionCode.ORDER_BY_ARRAY_NOT_SUPPORTED)
+                    .setMessage(expression.toString()).build().buildException();
+                }
+                if (expression.getColumnModifier() == ColumnModifier.SORT_DESC) {
+                    isAscending = !isAscending;
+                }
+                OrderByExpression orderByExpression = new OrderByExpression(expression, node.isNullsLast(), isAscending);
+                orderByExpressions.add(orderByExpression);
+            }
+            visitor.reset();
+        }
+       
+        if (orderByExpressions.isEmpty()) {
+            return OrderBy.EMPTY_ORDER_BY;
+        }
+        // If we're ordering by the order returned by the scan, we don't need an order by
+        if (visitor.isOrderPreserving()) {
+            if (visitor.isReverse()) {
+                if (context.getConnection().getQueryServices().supportsFeature(Feature.REVERSE_SCAN)) {
+                    return OrderBy.REV_ROW_KEY_ORDER_BY;
+                }
+            } else {
+                return OrderBy.FWD_ROW_KEY_ORDER_BY;
+            }
+        }
+
+        return new OrderBy(Lists.newArrayList(orderByExpressions.iterator()));
+    }
+
+
+    private OrderByCompiler() {
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..40fe1c3
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/PostDDLCompiler.java
@@ -0,0 +1,225 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.compile;
+
+import java.sql.ParameterMetaData;
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.phoenix.cache.ServerCacheClient.ServerCache;
+import org.apache.phoenix.compile.GroupByCompiler.GroupBy;
+import org.apache.phoenix.compile.OrderByCompiler.OrderBy;
+import org.apache.phoenix.coprocessor.UngroupedAggregateRegionObserver;
+import org.apache.phoenix.execute.AggregatePlan;
+import org.apache.phoenix.execute.MutationState;
+import org.apache.phoenix.iterate.ResultIterator;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixParameterMetaData;
+import org.apache.phoenix.jdbc.PhoenixStatement;
+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.PColumnFamily;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.ScanUtil;
+
+import com.google.common.collect.Lists;
+
+
+/**
+ * 
+ * Class that compiles plan to update data values after a DDL command
+ * executes.
+ *
+ * TODO: get rid of this ugly code and just go through the standard APIs.
+ * The only time we may still need this is to manage updating the empty
+ * key value, as we sometimes need to "go back through time" to adjust
+ * this.
+ * @author jtaylor
+ * @since 0.1
+ */
+public class PostDDLCompiler {
+    private final PhoenixConnection connection;
+
+    public PostDDLCompiler(PhoenixConnection connection) {
+        this.connection = connection;
+    }
+
+    public MutationPlan compile(final List<TableRef> tableRefs, final byte[] emptyCF, final byte[] projectCF, final List<PColumn> deleteList,
+            final long timestamp) throws SQLException {
+        
+        return new MutationPlan() {
+            
+            @Override
+            public PhoenixConnection getConnection() {
+                return connection;
+            }
+            
+            @Override
+            public ParameterMetaData getParameterMetaData() {
+                return PhoenixParameterMetaData.EMPTY_PARAMETER_META_DATA;
+            }
+            
+            @Override
+            public ExplainPlan getExplainPlan() throws SQLException {
+                return ExplainPlan.EMPTY_PLAN;
+            }
+            
+            @Override
+            public MutationState execute() throws SQLException {
+                if (tableRefs.isEmpty()) {
+                    return null;
+                }
+                boolean wasAutoCommit = connection.getAutoCommit();
+                try {
+                    connection.setAutoCommit(true);
+                    SQLException sqlE = null;
+                    if (deleteList == null && emptyCF == null) {
+                        return new MutationState(0, connection);
+                    }
+                    /*
+                     * Handles:
+                     * 1) deletion of all rows for a DROP TABLE and subsequently deletion of all rows for a DROP INDEX;
+                     * 2) deletion of all column values for a ALTER TABLE DROP COLUMN
+                     * 3) updating the necessary rows to have an empty KV
+                     */
+                    long totalMutationCount = 0;
+                    for (final TableRef tableRef : tableRefs) {
+                        Scan scan = new Scan();
+                        scan.setAttribute(UngroupedAggregateRegionObserver.UNGROUPED_AGG, QueryConstants.TRUE);
+                        SelectStatement select = SelectStatement.COUNT_ONE;
+                        // We need to use this tableRef
+                        ColumnResolver resolver = new ColumnResolver() {
+                            @Override
+                            public List<TableRef> getTables() {
+                                return Collections.singletonList(tableRef);
+                            }
+                            @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);
+                                return new ColumnRef(tableRef, column.getPosition());
+                            }
+                        };
+                        StatementContext context = new StatementContext(new PhoenixStatement(connection), resolver, Collections.<Object>emptyList(), scan);
+                        ScanUtil.setTimeRange(scan, timestamp);
+                        if (emptyCF != null) {
+                            scan.setAttribute(UngroupedAggregateRegionObserver.EMPTY_CF, emptyCF);
+                        }
+                        ServerCache cache = null;
+                        try {
+                            if (deleteList != null) {
+                                if (deleteList.isEmpty()) {
+                                    scan.setAttribute(UngroupedAggregateRegionObserver.DELETE_AGG, QueryConstants.TRUE);
+                                    // In the case of a row deletion, add index metadata so mutable secondary indexing works
+                                    /* TODO
+                                    ImmutableBytesWritable ptr = context.getTempPtr();
+                                    tableRef.getTable().getIndexMaintainers(ptr);
+                                    if (ptr.getLength() > 0) {
+                                        IndexMetaDataCacheClient client = new IndexMetaDataCacheClient(connection, tableRef);
+                                        cache = client.addIndexMetadataCache(context.getScanRanges(), ptr);
+                                        byte[] uuidValue = cache.getId();
+                                        scan.setAttribute(PhoenixIndexCodec.INDEX_UUID, uuidValue);
+                                    }
+                                    */
+                                } else {
+                                    // In the case of the empty key value column family changing, do not send the index
+                                    // metadata, as we're currently managing this from the client. It's possible for the
+                                    // data empty column family to stay the same, while the index empty column family
+                                    // changes.
+                                    PColumn column = deleteList.get(0);
+                                    if (emptyCF == null) {
+                                        scan.addColumn(column.getFamilyName().getBytes(), column.getName().getBytes());
+                                    }
+                                    scan.setAttribute(UngroupedAggregateRegionObserver.DELETE_CF, column.getFamilyName().getBytes());
+                                    scan.setAttribute(UngroupedAggregateRegionObserver.DELETE_CQ, column.getName().getBytes());
+                                }
+                            }
+                            List<byte[]> columnFamilies = Lists.newArrayListWithExpectedSize(tableRef.getTable().getColumnFamilies().size());
+                            if (projectCF == null) {
+                                for (PColumnFamily family : tableRef.getTable().getColumnFamilies()) {
+                                    columnFamilies.add(family.getName().getBytes());
+                                }
+                            } else {
+                                columnFamilies.add(projectCF);
+                            }
+                            // Need to project all column families into the scan, since we haven't yet created our empty key value
+                            RowProjector projector = ProjectionCompiler.compile(context, SelectStatement.COUNT_ONE, GroupBy.EMPTY_GROUP_BY);
+                            // Explicitly project these column families and don't project the empty key value,
+                            // since at this point we haven't added the empty key value everywhere.
+                            if (columnFamilies != null) {
+                                scan.getFamilyMap().clear();
+                                for (byte[] family : columnFamilies) {
+                                    scan.addFamily(family);
+                                }
+                                projector = new RowProjector(projector,false);
+                            }
+                            WhereCompiler.compile(context, select); // Push where clause into scan
+                            QueryPlan plan = new AggregatePlan(context, select, tableRef, projector, null, OrderBy.EMPTY_ORDER_BY, null, GroupBy.EMPTY_GROUP_BY, null);
+                            ResultIterator iterator = plan.iterator();
+                            try {
+                                Tuple row = iterator.next();
+                                ImmutableBytesWritable ptr = context.getTempPtr();
+                                totalMutationCount += (Long)projector.getColumnProjector(0).getValue(row, PDataType.LONG, ptr);
+                            } catch (SQLException e) {
+                                sqlE = e;
+                            } finally {
+                                try {
+                                    iterator.close();
+                                } catch (SQLException e) {
+                                    if (sqlE == null) {
+                                        sqlE = e;
+                                    } else {
+                                        sqlE.setNextException(e);
+                                    }
+                                } finally {
+                                    if (sqlE != null) {
+                                        throw sqlE;
+                                    }
+                                }
+                            }
+                        } finally {
+                            if (cache != null) { // Remove server cache if there is one
+                                cache.close();
+                            }
+                        }
+                        
+                    }
+                    final long count = totalMutationCount;
+                    return new MutationState(1, connection) {
+                        @Override
+                        public long getUpdateCount() {
+                            return count;
+                        }
+                    };
+                } finally {
+                    if (!wasAutoCommit) connection.setAutoCommit(wasAutoCommit);
+                }
+            }
+        };
+    }
+}


[50/51] [partial] Initial commit of master branch from github

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index 864c428..21a2c92 100644
--- a/.gitignore
+++ b/.gitignore
@@ -5,6 +5,7 @@
 
 # eclipse stuffs
 .settings/*
+*/.settings/
 .classpath
 .project
 

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/bin/csv-bulk-loader.sh
----------------------------------------------------------------------
diff --git a/bin/csv-bulk-loader.sh b/bin/csv-bulk-loader.sh
index 9b05e17..ded249e 100755
--- a/bin/csv-bulk-loader.sh
+++ b/bin/csv-bulk-loader.sh
@@ -36,5 +36,5 @@
 # -error                         Ignore error while reading rows from CSV ? (1 - YES | 0 - NO, defaults to 1) (optional)
 # -help                          Print all options (optional)
 
-phoenix_client_jar=$(find ../target/phoenix-*-client.jar)
+phoenix_client_jar=$(find ../phoenix-assembly/target/phoenix-*-client.jar)
 java -cp "$phoenix_client_jar" org.apache.phoenix.map.reduce.CSVBulkLoader "$@"

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/bin/performance.sh
----------------------------------------------------------------------
diff --git a/bin/performance.sh b/bin/performance.sh
index 24928aa..43ebaab 100755
--- a/bin/performance.sh
+++ b/bin/performance.sh
@@ -23,6 +23,7 @@
 
 # Note: This script is tested on Linux environment only. It should work on any Unix platform but is not tested.
 
+
 # command line arguments
 zookeeper=$1
 rowcount=$2
@@ -36,9 +37,9 @@ statements=""
 
 # Phoenix client jar. To generate new jars: $ mvn package -DskipTests
 current_dir=$(cd $(dirname $0);pwd)
-phoenix_jar_path="$current_dir/../target"
+phoenix_jar_path="$current_dir/../phoenix-assembly/target"
 phoenix_client_jar=$(find $phoenix_jar_path/phoenix-*-client.jar)
-testjar="$phoenix_jar_path/phoenix-*-tests.jar"
+testjar="$current_dir/../phoenix-core/target/phoenix-*-tests.jar"
 
 # HBase configuration folder path (where hbase-site.xml reside) for HBase/Phoenix client side property override
 hbase_config_path="$current_dir"

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/bin/psql.sh
----------------------------------------------------------------------
diff --git a/bin/psql.sh b/bin/psql.sh
index 316171a..c17f7c3 100755
--- a/bin/psql.sh
+++ b/bin/psql.sh
@@ -23,7 +23,7 @@
 
 # Phoenix client jar. To generate new jars: $ mvn package -DskipTests
 current_dir=$(cd $(dirname $0);pwd)
-phoenix_jar_path="$current_dir/../target"
+phoenix_jar_path="$current_dir/../phoenix-assembly/target"
 phoenix_client_jar=$(find $phoenix_jar_path/phoenix-*-client.jar)
 
 # HBase configuration folder path (where hbase-site.xml reside) for HBase/Phoenix client side property override

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/bin/sqlline.sh
----------------------------------------------------------------------
diff --git a/bin/sqlline.sh b/bin/sqlline.sh
index 6abda20..066f384 100755
--- a/bin/sqlline.sh
+++ b/bin/sqlline.sh
@@ -23,7 +23,7 @@
 
 # Phoenix client jar. To generate new jars: $ mvn package -DskipTests
 current_dir=$(cd $(dirname $0);pwd)
-phoenix_jar_path="$current_dir/../target"
+phoenix_jar_path="$current_dir/../phoenix-assembly/target"
 phoenix_client_jar=$(find $phoenix_jar_path/phoenix-*-client.jar)
 
 
@@ -36,4 +36,4 @@ if [ "$2" ]
   then sqlfile="--run=$2";
 fi
 
-java -cp ".:$phoenix_client_jar" -Dlog4j.configuration=file:$current_dir/log4j.properties sqlline.SqlLine -d org.apache.phoenix.jdbc.PhoenixDriver -u jdbc:phoenix:$1 -n none -p none --color=true --fastConnect=false --silent=true --verbose=false --isolation=TRANSACTION_READ_COMMITTED $sqlfile
+java -cp ".:$phoenix_client_jar" -Dlog4j.configuration=file:$current_dir/log4j.properties sqlline.SqlLine -d org.apache.phoenix.jdbc.PhoenixDriver -u jdbc:phoenix:$1 -n none -p none --color=true --fastConnect=false --verbose=true --isolation=TRANSACTION_READ_COMMITTED $sqlfile

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/bin/upgradeTo2.sh
----------------------------------------------------------------------
diff --git a/bin/upgradeTo2.sh b/bin/upgradeTo2.sh
index d2b9ec2..c2a4a0d 100755
--- a/bin/upgradeTo2.sh
+++ b/bin/upgradeTo2.sh
@@ -21,10 +21,9 @@
 #
 ############################################################################
 
-
 # Phoenix client jar. To generate new jars: $ mvn package -DskipTests
 current_dir=$(cd $(dirname $0);pwd)
-phoenix_jar_path="$current_dir/../target"
+phoenix_jar_path="$current_dir/../phoenix-assembly/target"
 phoenix_client_jar=$(find $phoenix_jar_path/phoenix-*-client.jar)
 
 # HBase configuration folder path (where hbase-site.xml reside) for HBase/Phoenix client side property override

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/build.txt
----------------------------------------------------------------------
diff --git a/build.txt b/build.txt
index a28a798..ca3304d 100644
--- a/build.txt
+++ b/build.txt
@@ -1,11 +1,38 @@
-# Building Phoenix
-================
+############################################################################
+#
+# Copyright 2010 The Apache Software Foundation
+#
+# 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.
+#
+############################################################################
+
+
+# Building Apache Phoenix
+=========================
 
 Phoenix uses Maven (3.X) to build all its necessary resources. 
 
 ## Building from source
 =======================
 
+On first setup, you may need to run
+	$ mvn install -DskipTests
+to install the local jars. This is a side-effect of multi-module maven projects
+
 1. To re-generate the antlr based files:
 	$ mvn process-sources
 
@@ -31,8 +58,10 @@ Use the m2e eclipse plugin and do Import->Maven Project and just pick the root '
 ===========
 Findbugs report is generated in /target/site
 	$ mvn site
+	
 
 ## Generate Apache Web Site
 ===========================
-	$ mvn clean site -Ddependency.locations.enabled=false
-	
+	$ mvn -pl phoenix-core site -Ddependency.locations.enabled=false
+
+Note: site is generated in phoenix-core/target/site

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/config/apache-access-logs.properties
----------------------------------------------------------------------
diff --git a/config/apache-access-logs.properties b/config/apache-access-logs.properties
new file mode 100644
index 0000000..2733cba
--- /dev/null
+++ b/config/apache-access-logs.properties
@@ -0,0 +1,35 @@
+###########################
+# Configuration for Flume
+##########################
+# TODO: Should be moved into the flume folder, but its a little bit of a pain in the assembly configuration
+# for now, just leaving it here - jyates
+#
+# configurationforagent
+agent.sources=spooling-source
+agent.sinks=phoenix-sink
+agent.channels=memoryChannel
+# configurationforchannel
+agent.channels.memoryChannel.type=memory
+agent.channels.memoryChannel.transactionCapacity=100
+agent.channels.memoryChannel.byteCapacityBufferPercentage=20
+# configurationforsource
+agent.sources.spooling-source.type=spooldir
+agent.sources.spooling-source.channels=memoryChannel
+agent.sources.spooling-source.spoolDir=/opt/logs
+# configurationforinterceptor
+agent.sources.spooling-source.interceptors=i1
+agent.sources.spooling-source.interceptors.i1.type=host
+agent.sources.spooling-source.interceptors.i1.hostHeader=f_host
+# configurationforsink
+agent.sinks.phoenix-sink.type=org.apache.phoenix.flume.sink.PhoenixSink
+agent.sinks.phoenix-sink.channel=memoryChannel
+agent.sinks.phoenix-sink.batchSize=100
+agent.sinks.phoenix-sink.table=APACHE_LOGS
+agent.sinks.phoenix-sink.ddl=CREATETABLEIFNOTEXISTSAPACHE_LOGS(uidVARCHARNOTNULL,hostVARCHAR,identityVARCHAR,userVARCHAR,timeVARCHAR,methodVARCHAR,requestVARCHAR,protocolVARCHAR,statusINTEGER,sizeINTEGER,refererVARCHAR,agentVARCHAR,f_hostVARCHARCONSTRAINTpkPRIMARYKEY(uid))
+agent.sinks.phoenix-sink.zookeeperQuorum=localhost
+agent.sinks.phoenix-sink.serializer=REGEX
+agent.sinks.phoenix-sink.serializer.rowkeyType=uuid
+agent.sinks.phoenix-sink.serializer.regex=([^]*)([^]*)([^]*)(-|\\[[^\\]]*\\])\"([^]+)([^]+)([^\"]+)\"(-|[0-9]*)(-|[0-9]*)(?:([^\"]*|\"[^\"]*\")([^\"]*|\"[^\"]*\"))?
+agent.sinks.phoenix-sink.serializer.columns=host,identity,user,time,method,request,protocol,status,size,referer,agent
+agent.sinks.phoenix-sink.serializer.headers=f_host
+

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/config/csv-bulk-load-config.properties
----------------------------------------------------------------------
diff --git a/config/csv-bulk-load-config.properties b/config/csv-bulk-load-config.properties
new file mode 100644
index 0000000..2d81808
--- /dev/null
+++ b/config/csv-bulk-load-config.properties
@@ -0,0 +1,5 @@
+mapreduce.map.output.compress=true
+mapreduce.map.output.compress.codec=org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.CompressionCodec
+io.sort.record.percent=0.2
+io.sort.factor=20
+mapred.tasktracker.map.tasks.maximum=10

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/dev/eclipse_prefs_phoenix.epf
----------------------------------------------------------------------
diff --git a/dev/eclipse_prefs_phoenix.epf b/dev/eclipse_prefs_phoenix.epf
index dd9e8f8..fb8df40 100644
--- a/dev/eclipse_prefs_phoenix.epf
+++ b/dev/eclipse_prefs_phoenix.epf
@@ -800,7 +800,7 @@ file_export_version=3.0
 /instance/org.eclipse.jdt.ui/org.eclipse.jdt.ui.packages.linktoeditor=true
 /instance/org.eclipse.jdt.ui/org.eclipse.jdt.ui.staticondemandthreshold=99
 /instance/org.eclipse.jdt.ui/org.eclipse.jdt.ui.text.code_templates_migrated=true
-/instance/org.eclipse.jdt.ui/org.eclipse.jdt.ui.text.custom_code_templates=<?xml version\="1.0" encoding\="UTF-8"?><templates><template autoinsert\="true" context\="gettercomment_context" deleted\="false" description\="Comment for getter method" enabled\="true" id\="org.eclipse.jdt.ui.text.codetemplates.gettercomment" name\="gettercomment">/**\n * @return Returns the ${bare_field_name}.\n */</template><template autoinsert\="true" context\="settercomment_context" deleted\="false" description\="Comment for setter method" enabled\="true" id\="org.eclipse.jdt.ui.text.codetemplates.settercomment" name\="settercomment">/**\n * @param ${param} The ${bare_field_name} to set.\n */</template><template autoinsert\="true" context\="typecomment_context" deleted\="false" description\="Comment for created types" enabled\="true" id\="org.eclipse.jdt.ui.text.codetemplates.typecomment" name\="typecomment">/**\n * Describe your class here.\n *\n * @author ${user}\n * @since 138\n */</template><templat
 e autoinsert\="true" context\="fieldcomment_context" deleted\="false" description\="Comment for fields" enabled\="true" id\="org.eclipse.jdt.ui.text.codetemplates.fieldcomment" name\="fieldcomment">/**\n * Comment for &lt;code&gt;${field}&lt;/code&gt;\n */</template></templates>
+/instance/org.eclipse.jdt.ui/org.eclipse.jdt.ui.text.custom_code_templates=<?xml version\="1.0" encoding\="UTF-8" standalone\="no"?><templates><template autoinsert\="true" context\="gettercomment_context" deleted\="false" description\="Comment for getter method" enabled\="true" id\="org.eclipse.jdt.ui.text.codetemplates.gettercomment" name\="gettercomment">/**\n * @return Returns the ${bare_field_name}.\n */</template><template autoinsert\="true" context\="settercomment_context" deleted\="false" description\="Comment for setter method" enabled\="true" id\="org.eclipse.jdt.ui.text.codetemplates.settercomment" name\="settercomment">/**\n * @param ${param} The ${bare_field_name} to set.\n */</template><template autoinsert\="true" context\="typecomment_context" deleted\="false" description\="Comment for created types" enabled\="true" id\="org.eclipse.jdt.ui.text.codetemplates.typecomment" name\="typecomment">/**\n * Describe your class here.\n *\n * @author ${user}\n * @since 138\n */</
 template><template autoinsert\="true" context\="fieldcomment_context" deleted\="false" description\="Comment for fields" enabled\="true" id\="org.eclipse.jdt.ui.text.codetemplates.fieldcomment" name\="fieldcomment">/**\n * Comment for &lt;code&gt;${field}&lt;/code&gt;\n */</template><template autoinsert\="false" context\="newtype_context" deleted\="false" description\="Newly created files" enabled\="true" id\="org.eclipse.jdt.ui.text.codetemplates.newtype" name\="newtype">/*\n * Copyright 2010 The Apache Software Foundation\n *\n * Licensed to the Apache Software Foundation (ASF) under one\n * or more contributor license agreements.  See the NOTICE file\n *distributed with this work for additional information\n * regarding copyright ownership.  The ASF licenses this file\n * to you under the Apache License, Version 2.0 (the\n * "License"); you maynot use this file except in compliance\n * with the License.  You may obtain a copy of the License at\n *\n * http://www.apache.org/licens
 es/LICENSE-2.0\n *\n * Unless required by applicablelaw or agreed to in writing, software\n * distributed under the License is distributed on an "AS IS" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n${package_declaration}\n\n${typecomment}\n${type_declaration}</template></templates>
 /instance/org.eclipse.jdt.ui/org.eclipse.jdt.ui.text.custom_templates=<?xml version\="1.0" encoding\="UTF-8"?><templates/>
 /instance/org.eclipse.jdt.ui/org.eclipse.jdt.ui.text.templates_migrated=true
 /instance/org.eclipse.jdt.ui/proposalOrderMigrated=true

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-assembly/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-assembly/pom.xml b/phoenix-assembly/pom.xml
new file mode 100644
index 0000000..9a89206
--- /dev/null
+++ b/phoenix-assembly/pom.xml
@@ -0,0 +1,115 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache</groupId>
+    <artifactId>phoenix</artifactId>
+    <version>3.0.0-SNAPSHOT</version>
+  </parent>
+  <artifactId>phoenix-assembly</artifactId>
+  <name>Phoenix Assebmly</name>
+  <description>Assemble Phoenix artifacts</description>
+  <packaging>pom</packaging>
+
+  <build>
+    <plugins>
+      <plugin>
+        <artifactId>maven-assembly-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>client</id>
+            <phase>package</phase>
+            <goals>
+              <goal>single</goal>
+            </goals>
+            <configuration>
+              <attach>false</attach>
+              <finalName>phoenix-${project.version}</finalName>
+              <archive>
+                <index>true</index>
+                <manifest>
+                  <addClasspath>true</addClasspath>
+                  <mainClass>org.apache.phoenix.util.PhoenixRuntime</mainClass>
+                  <addDefaultImplementationEntries>true</addDefaultImplementationEntries>
+                  <addDefaultSpecificationEntries>true</addDefaultSpecificationEntries>
+                </manifest>
+              </archive>
+              <descriptors>
+                <descriptor>src/build/client.xml</descriptor>
+              </descriptors>
+            </configuration>
+          </execution>
+          <execution>
+            <id>package-to-tar</id>
+            <phase>package</phase>
+            <goals>
+              <goal>single</goal>
+            </goals>
+            <configuration>
+            <finalName>phoenix-${project.version}</finalName>
+              <attach>false</attach>
+              <tarLongFileMode>gnu</tarLongFileMode>
+              <appendAssemblyId>false</appendAssemblyId>
+              <descriptors>
+                <descriptor>src/build/all.xml</descriptor>
+              </descriptors>
+            </configuration>
+          </execution>
+          <execution>
+            <id>client-minimal</id>
+            <phase>package</phase>
+            <goals>
+              <goal>single</goal>
+            </goals>
+            <configuration>
+         <finalName>phoenix-${project.version}</finalName>
+              <attach>false</attach>
+              <appendAssemblyId>true</appendAssemblyId>
+              <descriptors>
+               <!--build the phoenix client jar, but without HBase code. -->
+                <descriptor>src/build/client-without-hbase.xml</descriptor>
+               <!-- build the phoenix client jar, but without HBase (or its depenencies). -->
+                <descriptor>src/build/client-minimal.xml</descriptor>
+              </descriptors>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <!-- No jars created for this module -->
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <executions>
+          <execution>
+            <phase>prepare-package</phase>
+            <goals />
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+
+  <dependencies>
+    <!-- Depend on all other internal projects -->
+    <dependency>
+      <groupId>org.apache</groupId>
+      <artifactId>phoenix-core</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache</groupId>
+      <artifactId>phoenix-hadoop-compat</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache</groupId>
+      <artifactId>${compat.module}</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache</groupId>
+      <artifactId>phoenix-flume</artifactId>
+    </dependency>
+        <dependency>
+      <groupId>org.apache</groupId>
+      <artifactId>phoenix-pig</artifactId>
+    </dependency>
+  </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-assembly/src/build/all.xml
----------------------------------------------------------------------
diff --git a/phoenix-assembly/src/build/all.xml b/phoenix-assembly/src/build/all.xml
new file mode 100644
index 0000000..36910d5
--- /dev/null
+++ b/phoenix-assembly/src/build/all.xml
@@ -0,0 +1,139 @@
+<assembly xmlns="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.0"
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.0 http://maven.apache.org/xsd/assembly-1.1.0.xsd">
+  <!--This 'all' id is not appended to the produced bundle because we do this: http://maven.apache.org/plugins/maven-assembly-plugin/faq.html#required-classifiers -->
+  <id>all</id>
+  <formats>
+    <format>tar.gz</format>
+  </formats>
+  <includeBaseDirectory>true</includeBaseDirectory>
+  
+  <moduleSets>
+    <moduleSet>
+      <!-- Enable access to all projects in the current multimodule build. Eclipse 
+        says this is an error, but builds from the command line just fine. -->
+      <useAllReactorProjects>true</useAllReactorProjects>
+       <!-- Include all the sources in the top directory -->
+      <sources>
+         <fileSets>
+          <fileSet>
+            <!-- Make sure this excludes is same as the phoenix-hadoop2-compat
+                 excludes below -->
+            <excludes>
+              <exclude>target/</exclude>
+              <exclude>test/</exclude>
+              <exclude>.classpath</exclude>
+              <exclude>.project</exclude>
+              <exclude>.settings/</exclude>
+            </excludes>
+          </fileSet>
+        </fileSets>
+      </sources>
+      <!-- Binaries for the dependencies also go in the lib directory -->
+      <binaries>
+        <outputDirectory>lib</outputDirectory>
+        <unpack>false</unpack>
+      </binaries>
+    </moduleSet>
+  </moduleSets>
+
+  <fileSets>
+    <!--This one is weird.  When we assemble src, it'll be default profile which
+         at the moment is hadoop1.  But we should include the hadoop2 compat module
+         too so can build hadoop2 from src -->
+    <fileSet>
+      <directory>${project.basedir}/..</directory>
+      <fileMode>0644</fileMode>
+      <directoryMode>0755</directoryMode>
+      <includes>
+        <include>phoenix-*</include>
+        </includes>
+            <excludes>
+              <exclude>target/</exclude>
+              <exclude>test/</exclude>
+              <exclude>.classpath</exclude>
+              <exclude>.project</exclude>
+              <exclude>.settings/</exclude>
+            </excludes>
+    </fileSet>
+    <fileSet>
+      <!--Get misc project files -->
+      <directory>${project.basedir}/..</directory>
+      <outputDirectory>/</outputDirectory>
+      <includes>
+        <include>*.txt</include>
+        <include>*.md</include>
+        <include>pom.xml</include>
+      </includes>
+    </fileSet>
+    <!-- Top level directories -->
+    <fileSet>
+      <directory>${project.basedir}/../bin</directory>
+      <outputDirectory>bin</outputDirectory>
+      <fileMode>0644</fileMode>
+      <directoryMode>0755</directoryMode>
+    </fileSet>
+    <fileSet>
+      <directory>${project.basedir}/../dev</directory>
+      <fileMode>0644</fileMode>
+      <directoryMode>0755</directoryMode>
+    </fileSet>
+    <fileSet>
+      <directory>${project.basedir}/../docs</directory>
+      <fileMode>0644</fileMode>
+      <directoryMode>0755</directoryMode>
+    </fileSet>
+      <fileSet>
+      <directory>${project.basedir}/../examples</directory>
+      <fileMode>0644</fileMode>
+      <directoryMode>0755</directoryMode>
+    </fileSet>
+    <!-- Add the client jar. Expects the client jar packaging phase to already be run, 
+      which is determined by specification order in the pom. -->
+    <fileSet>
+      <directory>target</directory>
+      <outputDirectory>/</outputDirectory>
+      <includes>
+        <include>phoenix-*-client.jar</include>
+      </includes>
+    </fileSet>
+  </fileSets>
+
+  <!-- And add all of our dependencies -->
+  <dependencySets>
+    <dependencySet>
+      <!-- Unpack all the dependencies to class files, since java doesn't support 
+        jar of jars for running -->
+      <unpack>false</unpack>
+      <outputDirectory>/lib</outputDirectory>
+      <includes>
+        <include>commons-configuration:commons-configuration</include>
+        <include>commons-io:commons-io</include>
+        <include>commons-lang:commons-lang</include>
+        <include>commons-logging:commons-logging</include>
+        <include>com.google.guava:guava</include>
+        <include>org.apache.hadoop:hadoop*</include>
+        <include>com.google.protobuf:protobuf-java</include>
+        <include>org.slf4j:slf4j-api</include>
+        <include>org.slf4j:slf4j-log4j12</include>
+        <include>org.apache.zookeeper:zookeeper</include>
+        <include>log4j:log4j</include>
+        <include>org.apache.hbase:hbase*</include>
+        <include>net.sf.opencsv:opencsv</include>
+        <include>org.antlr:antlr</include>
+      </includes>
+    </dependencySet>
+    <!-- Separate dependency set to just pull in the jackson stuff since its test 
+      scoped and we only include 'runtime' scoped (which includes compile) dependencies -->
+    <dependencySet>
+      <unpack>false</unpack>
+      <scope>test</scope>
+      <!-- save these dependencies to the top-level -->
+      <outputDirectory>/lib</outputDirectory>
+      <includes>
+        <include>org.codehaus.jackson:jackson-core-asl</include>
+        <include>org.codehaus.jackson:jackson-mapper-asl</include>
+      </includes>
+    </dependencySet>
+  </dependencySets>
+</assembly>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-assembly/src/build/client-minimal.xml
----------------------------------------------------------------------
diff --git a/phoenix-assembly/src/build/client-minimal.xml b/phoenix-assembly/src/build/client-minimal.xml
new file mode 100644
index 0000000..3e6e4e9
--- /dev/null
+++ b/phoenix-assembly/src/build/client-minimal.xml
@@ -0,0 +1,16 @@
+<assembly xmlns="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.0"
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.0 http://maven.apache.org/xsd/assembly-1.1.0.xsd">
+  <!-- Often clients want to use Phoenix in an existing HBase environment (they have 
+    their own HBase version already built), so the standard HBase jar shouldn't be included 
+    (as with the regular client jar) as it will conflict the installed version. This 
+    profile does the same thing as the client.xml build, but excludes the hbase stuff. -->
+  <id>client-minimal</id>
+  <formats>
+    <format>jar</format>
+  </formats>
+  <includeBaseDirectory>false</includeBaseDirectory>
+  <componentDescriptors>
+    <componentDescriptor>src/build/components-minimal.xml</componentDescriptor>
+  </componentDescriptors>
+</assembly>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-assembly/src/build/client-without-hbase.xml
----------------------------------------------------------------------
diff --git a/phoenix-assembly/src/build/client-without-hbase.xml b/phoenix-assembly/src/build/client-without-hbase.xml
new file mode 100644
index 0000000..2933715
--- /dev/null
+++ b/phoenix-assembly/src/build/client-without-hbase.xml
@@ -0,0 +1,18 @@
+<assembly xmlns="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.0"
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.0 http://maven.apache.org/xsd/assembly-1.1.0.xsd">
+  <!-- Often clients want to use Phoenix in an existing HBase environment (they have 
+    their own HBase version already built), so the standard HBase jar shouldn't be included 
+    (as with the regular client jar) as it will conflict the installed version. This 
+    profile does the same thing as the client.xml build, but excludes the hbase stuff. -->
+  <id>client-without-hbase</id>
+  <formats>
+    <format>jar</format>
+  </formats>
+  <includeBaseDirectory>false</includeBaseDirectory>
+
+  <componentDescriptors>
+    <componentDescriptor>src/build/components-minimal.xml</componentDescriptor>
+    <componentDescriptor>src/build/components-major-client.xml</componentDescriptor>
+  </componentDescriptors>
+</assembly>

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-assembly/src/build/client.xml
----------------------------------------------------------------------
diff --git a/phoenix-assembly/src/build/client.xml b/phoenix-assembly/src/build/client.xml
new file mode 100644
index 0000000..3be4949
--- /dev/null
+++ b/phoenix-assembly/src/build/client.xml
@@ -0,0 +1,41 @@
+<assembly xmlns="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.0"
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.0 http://maven.apache.org/xsd/assembly-1.1.0.xsd">
+  <id>client</id>
+  <!-- All the dependencies (unpacked) necessary to run phoenix from a single, stand-alone jar -->
+  <formats>
+    <format>jar</format>
+  </formats>
+  <includeBaseDirectory>false</includeBaseDirectory>
+  
+  <componentDescriptors>
+    <componentDescriptor>src/build/components-minimal.xml</componentDescriptor>
+    <componentDescriptor>src/build/components-major-client.xml</componentDescriptor>
+  </componentDescriptors>
+
+  <!-- Unpack all the modules into the target jar -->
+  <moduleSets>
+    <moduleSet>
+      <useAllReactorProjects>true</useAllReactorProjects>
+      <binaries>
+      <outputDirectory>/</outputDirectory>
+        <unpack>true</unpack>
+      </binaries>
+    </moduleSet>
+  </moduleSets>
+
+  <dependencySets>
+    <dependencySet>
+      <!-- Unpack all the dependencies to class files, since java doesn't support 
+        jar of jars for running -->
+      <unpack>true</unpack>
+      <!-- save these dependencies to the top-level -->
+      <outputDirectory>/</outputDirectory>
+      <includes>
+        <include>jline:jline</include>
+        <include>sqlline:sqlline</include>
+        <include>org.apache.hbase:hbase*</include>
+      </includes>
+    </dependencySet>
+  </dependencySets>
+</assembly>

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-assembly/src/build/components-major-client.xml
----------------------------------------------------------------------
diff --git a/phoenix-assembly/src/build/components-major-client.xml b/phoenix-assembly/src/build/components-major-client.xml
new file mode 100644
index 0000000..bdbf701
--- /dev/null
+++ b/phoenix-assembly/src/build/components-major-client.xml
@@ -0,0 +1,32 @@
+<component>
+  <!-- Components that the client needs (except for HBase) -->
+  <dependencySets>
+    <dependencySet>
+      <!-- Unpack all the dependencies to class files, since java doesn't support 
+        jar of jars for running -->
+      <unpack>true</unpack>
+      <!-- save these dependencies to the top-level -->
+      <outputDirectory>/</outputDirectory>
+      <!-- Maybe a blacklist is easier? -->
+      <includes>
+        <!-- We use a newer version of guava than HBase - this might be an issue? -->
+        <include>com.google.guava:guava</include>
+        <!-- HBase also pulls in these dependencies on its own, should we include-them? -->
+        <include>com.google.protobuf:protobuf-java</include>
+        <include>org.slf4j:slf4j-api</include>
+        <include>org.slf4j:slf4j-log4j12</include>
+        <include>org.apache.zookeeper:zookeeper</include>
+        <include>log4j:log4j</include>
+        <include>org.apache.hadoop:hadoop*</include>
+        <include>commons-configuration:commons-configuration</include>
+        <include>commons-io:commons-io</include>
+        <include>commons-logging:commons-logging</include>
+        <include>commons-lang:commons-lang</include>
+        <include>commons-cli:commons-cli</include>
+        <include>org.codehaus.jackson:jackson-mapper-asl</include>
+        <include>org.codehaus.jackson:jackson-core-asl</include>
+        <include>org.xerial.snappy:snappy-java</include>
+      </includes>
+    </dependencySet>
+  </dependencySets>
+</component>

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-assembly/src/build/components-minimal.xml
----------------------------------------------------------------------
diff --git a/phoenix-assembly/src/build/components-minimal.xml b/phoenix-assembly/src/build/components-minimal.xml
new file mode 100644
index 0000000..172e398
--- /dev/null
+++ b/phoenix-assembly/src/build/components-minimal.xml
@@ -0,0 +1,47 @@
+<component>
+  <!-- Just the basic components that Phoenix pulls in, that is not a transitive dependency from Hadoop/HBase/Pig -->
+  <dependencySets>
+    <dependencySet>
+      <!-- Unpack all the dependencies to class files, since java doesn't support 
+        jar of jars for running -->
+      <unpack>true</unpack>
+      <!-- save these dependencies to the top-level -->
+      <outputDirectory>/</outputDirectory>
+      <!-- Just include the extra things that phoenix needs -->
+      <includes>
+        <include>net.sf.opencsv:opencsv</include>
+        <include>org.antlr:antlr*</include>
+      </includes>
+    </dependencySet>
+
+    <dependencySet>
+      <outputDirectory>/</outputDirectory>
+      <unpack>true</unpack>
+      <scope>system</scope>
+    </dependencySet>
+  </dependencySets>
+
+  <fileSets>
+    <fileSet>
+      <!--Get misc project files -->
+      <directory>${project.basedir}/..</directory>
+      <outputDirectory>/</outputDirectory>
+      <includes>
+        <include>*.txt*</include>
+        <include>*.md</include>
+        <include>NOTICE*</include>
+      </includes>
+      <excludes>
+        <exclude>build.txt</exclude>
+      </excludes>
+    </fileSet>
+    <fileSet>
+      <!--Get map-red-config properties files -->
+      <directory>${project.basedir}/../config</directory>
+      <outputDirectory>/</outputDirectory>
+      <includes>
+        <include>csv-bulk-load-config.properties</include>
+      </includes>
+    </fileSet>
+  </fileSets>
+</component>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-core/pom.xml b/phoenix-core/pom.xml
new file mode 100644
index 0000000..0e2b8e5
--- /dev/null
+++ b/phoenix-core/pom.xml
@@ -0,0 +1,319 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache</groupId>
+    <artifactId>phoenix</artifactId>
+    <version>3.0.0-SNAPSHOT</version>
+  </parent>
+  <artifactId>phoenix-core</artifactId>
+  <name>Phoenix Core</name>
+  <description>Core Phoenix codebase</description>
+
+  <licenses>
+      <license>
+          <name>The Apache Software License, Version 2.0</name>
+          <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
+          <distribution>repo</distribution>
+          <comments />
+      </license>
+  </licenses>
+
+  <organization>
+      <name>Apache Software Foundation</name>
+      <url>http://www.apache.org</url>
+  </organization>
+
+  <build>
+    <resources>
+      <resource>
+        <directory>src/main/resources</directory>
+        <targetPath>META-INF/services</targetPath>
+        <includes>
+          <include>java.sql.Driver</include>
+        </includes>
+      </resource>
+    </resources>
+
+    <plugins>
+      <!-- Add the ant-generated sources to the source path -->
+     <plugin>
+       <groupId>org.apache.maven.plugins</groupId>
+       <artifactId>maven-site-plugin</artifactId>
+       <version>3.2</version>
+       <dependencies>
+        <dependency>
+           <groupId>org.apache.maven.doxia</groupId>
+           <artifactId>doxia-module-markdown</artifactId>
+           <version>1.3</version>
+         </dependency>
+         <dependency>
+           <groupId>lt.velykis.maven.skins</groupId>
+           <artifactId>reflow-velocity-tools</artifactId>
+           <version>1.0.0</version>
+         </dependency>
+         <dependency>
+           <groupId>org.apache.velocity</groupId>
+           <artifactId>velocity</artifactId>
+           <version>1.7</version>
+         </dependency>
+       </dependencies>
+       <configuration>
+         <reportPlugins>
+           <plugin>
+             <groupId>org.codehaus.mojo</groupId>
+             <artifactId>findbugs-maven-plugin</artifactId>
+	         <version>2.5.2</version>
+           </plugin>
+         </reportPlugins>
+       </configuration>
+     </plugin>
+     <plugin>
+       <artifactId>exec-maven-plugin</artifactId>
+       <groupId>org.codehaus.mojo</groupId>
+       <version>1.2.1</version>
+       <executions>
+        <execution><!-- Run our version calculation script -->
+          <id>Merge Language Reference</id>
+           <phase>site</phase>
+           <goals>
+             <goal>exec</goal>
+           </goals>
+           <configuration>
+             <executable>${basedir}/src/site/bin/merge.sh</executable>
+           </configuration>
+         </execution>
+       </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>build-helper-maven-plugin</artifactId>
+        <version>${maven-build-helper-plugin.version}</version>
+        <executions>
+          <execution>
+            <id>add-source</id>
+            <phase>generate-sources</phase>
+            <goals>
+              <goal>add-source</goal>
+            </goals>
+            <configuration>
+              <sources>
+                <source>${antlr-output.dir}</source>
+                <source>${antlr-input.dir}</source>
+              </sources>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <!-- Compile the antlr sources -->
+      <plugin>
+        <groupId>org.antlr</groupId>
+        <artifactId>antlr3-maven-plugin</artifactId>
+        <version>3.5</version>
+        <executions>
+          <execution>
+            <goals>
+              <goal>antlr</goal>
+            </goals>
+          </execution>
+        </executions>
+        <configuration>
+          <outputDirectory>${antlr-output.dir}/org/apache/phoenix/parse</outputDirectory>
+        </configuration>
+      </plugin>
+      <!-- Run with -Dmaven.test.skip.exec=true to build -tests.jar without running 
+        tests (this is needed for upstream projects whose tests need this jar simply for 
+        compilation) -->
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <executions>
+          <execution>
+            <phase>prepare-package
+            </phase>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+            <configuration>
+              <archive>
+                <manifest>
+                  <mainClass>org.apache.phoenix.util.GeneratePerformanceData</mainClass>
+                </manifest>
+              </archive>
+            </configuration>
+          </execution>
+        </executions>
+        <configuration>
+          <!-- Exclude these 2 packages, because their dependency _binary_ files 
+            include the sources, and Maven 2.2 appears to add them to the sources to compile, 
+            weird -->
+          <excludes>
+            <exclude>org/apache/jute/**</exclude>
+            <exclude>org/apache/zookeeper/**</exclude>
+            <exclude>**/*.jsp</exclude>
+            <exclude>log4j.properties</exclude>
+          </excludes>
+        </configuration>
+      </plugin>
+      <!-- Setup eclipse -->
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-eclipse-plugin</artifactId>
+        <configuration>
+          <buildcommands>
+            <buildcommand>org.jamon.project.templateBuilder</buildcommand>
+            <buildcommand>org.eclipse.jdt.core.javabuilder</buildcommand>
+          </buildcommands>
+        </configuration>
+      </plugin>
+      <plugin>
+        <!--Make it so assembly:single does nothing in here -->
+        <artifactId>maven-assembly-plugin</artifactId>
+        <configuration>
+          <skipAssembly>true</skipAssembly>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
+
+  <dependencies>
+    <!-- Intra project dependencies -->
+    <dependency>
+      <groupId>org.apache</groupId>
+      <artifactId>phoenix-hadoop-compat</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache</groupId>
+      <artifactId>phoenix-hadoop-compat</artifactId>
+      <classifier>tests</classifier>
+      <scope>test</scope>
+    </dependency>
+    <!-- Make sure we have all the antlr dependencies -->
+    <dependency>
+      <groupId>org.antlr</groupId>
+      <artifactId>antlr</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.antlr</groupId>
+      <artifactId>antlr-runtime</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>net.sf.opencsv</groupId>
+      <artifactId>opencsv</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>jline</groupId>
+      <artifactId>jline</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>sqlline</groupId>
+      <artifactId>sqlline</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase</artifactId>
+    </dependency>
+    <!-- Findbugs Annotation -->
+    <dependency>
+      <groupId>net.sourceforge.findbugs</groupId>
+      <artifactId>annotations</artifactId>
+    </dependency>
+
+    <!-- Test Dependencies -->
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase</artifactId>
+      <type>test-jar</type>
+    </dependency>
+    <!-- Needed by HBase to run the minicluster -->
+    <dependency>
+      <groupId>org.codehaus.jackson</groupId>
+      <artifactId>jackson-core-asl</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.codehaus.jackson</groupId>
+      <artifactId>jackson-mapper-asl</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.codehaus.jackson</groupId>
+      <artifactId>jackson-jaxrs</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.codehaus.jackson</groupId>
+      <artifactId>jackson-xc</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+    </dependency>
+  </dependencies>
+
+  <profiles>
+    <!-- Profile for building against Hadoop 1. Active by default. Not used if another 
+      Hadoop profile is specified with mvn -Dhadoop.profile=foo -->
+    <profile>
+      <id>hadoop-1</id>
+      <activation>
+        <property>
+          <name>!hadoop.profile</name>
+        </property>
+      </activation>
+      <dependencies>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-core</artifactId>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-test</artifactId>
+        </dependency>
+      </dependencies>
+    </profile>
+
+    <!-- Profile for building against Hadoop 2. Activate using: mvn -Dhadoop.profile=2 -->
+    <profile>
+      <id>hadoop-2</id>
+      <activation>
+        <property>
+          <name>hadoop.profile</name>
+          <value>2</value>
+        </property>
+      </activation>
+      <dependencies>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-common</artifactId>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-annotations</artifactId>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-mapreduce-client-core</artifactId>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-minicluster</artifactId>
+        </dependency>
+      </dependencies>
+      <build>
+        <plugins>
+          <plugin>
+            <artifactId>maven-dependency-plugin</artifactId>
+            <version>${maven-dependency-plugin.version}</version>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+  </profiles>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/antlr3/PhoenixSQL.g
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/antlr3/PhoenixSQL.g b/phoenix-core/src/main/antlr3/PhoenixSQL.g
new file mode 100644
index 0000000..5dcfe81
--- /dev/null
+++ b/phoenix-core/src/main/antlr3/PhoenixSQL.g
@@ -0,0 +1,1136 @@
+/**
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.
+ */
+grammar PhoenixSQL;
+
+tokens
+{
+    SELECT='select';
+    FROM='from';
+    WHERE='where';
+    NOT='not';
+    AND='and';
+    OR='or';
+    NULL='null';
+    TRUE='true';
+    FALSE='false';
+    LIKE='like';
+    AS='as';
+    OUTER='outer';
+    ON='on';
+    IN='in';
+    GROUP='group';
+    HAVING='having';
+    ORDER='order';
+    BY='by';
+    ASC='asc';
+    DESC='desc';
+    NULLS='nulls';
+    LIMIT='limit';
+    FIRST='first';
+    LAST='last';
+    CASE='case';
+    WHEN='when';
+    THEN='then';
+    ELSE='else';
+    END='end';
+    EXISTS='exists';
+    IS='is';
+    FIRST='first';    
+    DISTINCT='distinct';
+    JOIN='join';
+    INNER='inner';
+    LEFT='left';
+    RIGHT='right';
+    FULL='full';
+    BETWEEN='between';
+    UPSERT='upsert';
+    INTO='into';
+    VALUES='values';
+    DELETE='delete';
+    CREATE='create';
+    DROP='drop';
+    PRIMARY='primary';
+    KEY='key';
+    ALTER='alter';
+    COLUMN='column';
+    TABLE='table';
+    ADD='add';
+    SPLIT='split';
+    EXPLAIN='explain';
+    VIEW='view';
+    IF='if';
+    CONSTRAINT='constraint';
+    TABLES='tables';
+    ALL='all';
+    INDEX='index';
+    INCLUDE='include';
+    WITHIN='within';
+    SET='set';
+    CAST='cast';
+    USABLE='usable';
+    UNUSABLE='unusable';
+    DISABLE='disable';
+    REBUILD='rebuild';
+    ARRAY='array';
+    SEQUENCE='sequence';
+    START='start';
+    WITH='with';
+    INCREMENT='increment';
+    NEXT='next';
+    CURRENT='current';
+    VALUE='value';
+    FOR='for';
+    CACHE='cache';
+    DERIVE='derive';
+}
+
+
+@parser::header {
+/**
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+///CLOVER:OFF
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.ListMultimap;
+import org.apache.hadoop.hbase.util.Pair;
+import java.math.BigDecimal;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Stack;
+import java.sql.SQLException;
+import org.apache.phoenix.expression.function.CountAggregateFunction;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.IllegalDataException;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.PIndexState;
+import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.util.SchemaUtil;
+}
+
+@lexer::header {
+/**
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+///CLOVER:OFF
+}
+
+// --------------------------------------
+// The Parser
+
+@parser::members
+{
+    
+    /**
+     * used to turn '?' binds into : binds.
+     */
+    private int anonBindNum;
+    private ParseNodeFactory factory;
+    private ParseContext.Stack contextStack = new ParseContext.Stack();
+
+    public void setParseNodeFactory(ParseNodeFactory factory) {
+        this.factory = factory;
+    }
+    
+    public boolean isCountFunction(String field) {
+        return CountAggregateFunction.NORMALIZED_NAME.equals(SchemaUtil.normalizeIdentifier(field));
+    }
+     
+    public int line(Token t) {
+        return t.getLine();
+    }
+
+    public int column(Token t) {
+        return t.getCharPositionInLine() + 1;
+    }
+    
+    private void throwRecognitionException(Token t) throws RecognitionException {
+        RecognitionException e = new RecognitionException();
+        e.token = t;
+        e.line = t.getLine();
+        e.charPositionInLine = t.getCharPositionInLine();
+        e.input = input;
+        throw e;
+    }
+    
+    public int getBindCount() {
+        return anonBindNum;
+    }
+    
+    public void resetBindCount() {
+        anonBindNum = 0;
+    }
+    
+    public String nextBind() {
+        return Integer.toString(++anonBindNum);
+    }
+    
+    public void updateBind(String namedBind){
+         int nBind = Integer.parseInt(namedBind);
+         if (nBind > anonBindNum) {
+             anonBindNum = nBind;
+         }
+    }
+
+    protected Object recoverFromMismatchedToken(IntStream input, int ttype, BitSet follow)
+        throws RecognitionException {
+        RecognitionException e = null;
+        // if next token is what we are looking for then "delete" this token
+        if (mismatchIsUnwantedToken(input, ttype)) {
+            e = new UnwantedTokenException(ttype, input);
+        } else if (mismatchIsMissingToken(input, follow)) {
+            Object inserted = getMissingSymbol(input, e, ttype, follow);
+            e = new MissingTokenException(ttype, input, inserted);
+        } else {
+            e = new MismatchedTokenException(ttype, input);
+        }
+        throw e;
+    }
+
+    public Object recoverFromMismatchedSet(IntStream input, RecognitionException e, BitSet follow)
+        throws RecognitionException
+    {
+        throw e;
+    }
+    
+    @Override
+    public String getErrorMessage(RecognitionException e, String[] tokenNames) {
+        if (e instanceof MismatchedTokenException) {
+            MismatchedTokenException mte = (MismatchedTokenException)e;
+            String txt = mte.token.getText();
+            String p = mte.token.getType() == -1 ? "EOF" : PARAPHRASE[mte.token.getType()];
+            String expecting = (mte.expecting < PARAPHRASE.length && mte.expecting >= 0) ? PARAPHRASE[mte.expecting] : null;
+            if (expecting == null) {
+                return "unexpected token (" + line(mte.token) + "," + column(mte.token) + "): " + (txt != null ? txt : p);
+            } else {
+                return "expecting " + expecting +
+                    ", found '" + (txt != null ? txt : p) + "'";
+            }
+        } else if (e instanceof NoViableAltException) {
+            //NoViableAltException nvae = (NoViableAltException)e;
+            return "unexpected token: (" + line(e.token) + "," + column(e.token) + ")" + getTokenErrorDisplay(e.token);
+        }
+        return super.getErrorMessage(e, tokenNames);
+     }
+
+    public String getTokenErrorDisplay(int t) {
+        String ret = PARAPHRASE[t];
+        if (ret == null) ret = "<UNKNOWN>";
+        return ret;
+    }
+
+
+    private String[] PARAPHRASE = new String[getTokenNames().length];
+    {
+        PARAPHRASE[NAME] = "a field or entity name";
+        PARAPHRASE[NUMBER] = "a number";
+        PARAPHRASE[EQ] = "an equals sign";
+        PARAPHRASE[LT] = "a left angle bracket";
+        PARAPHRASE[GT] = "a right angle bracket";
+        PARAPHRASE[COMMA] = "a comma";
+        PARAPHRASE[LPAREN] = "a left parentheses";
+        PARAPHRASE[RPAREN] = "a right parentheses";
+        PARAPHRASE[SEMICOLON] = "a semi-colon";
+        PARAPHRASE[COLON] = "a colon";
+        PARAPHRASE[LSQUARE] = "left square bracket";
+        PARAPHRASE[RSQUARE] = "right square bracket";
+        PARAPHRASE[LCURLY] = "left curly bracket";
+        PARAPHRASE[RCURLY] = "right curly bracket";
+        PARAPHRASE[AT] = "at";
+        PARAPHRASE[MINUS] = "a subtraction";
+        PARAPHRASE[TILDE] = "a tilde";
+        PARAPHRASE[PLUS] = "an addition";
+        PARAPHRASE[ASTERISK] = "an asterisk";
+        PARAPHRASE[DIVIDE] = "a division";
+        PARAPHRASE[FIELDCHAR] = "a field character";
+        PARAPHRASE[LETTER] = "an ansi letter";
+        PARAPHRASE[POSINTEGER] = "a positive integer";
+        PARAPHRASE[DIGIT] = "a number from 0 to 9";
+    }
+}
+
+@rulecatch {
+    catch (RecognitionException re) {
+        throw re;
+    }
+}
+
+@lexer::members {
+
+}
+
+// Used to incrementally parse a series of semicolon-terminated SQL statement
+// Note than unlike the rule below an EOF is not expected at the end.
+nextStatement returns [BindableStatement ret]
+    :  s=oneStatement {$ret = s;} SEMICOLON
+    |  EOF
+    ;
+
+// Parses a single SQL statement (expects an EOF after the select statement).
+statement returns [BindableStatement ret]
+    :   s=oneStatement {$ret = s;} EOF
+    ;
+
+// Parses a select statement which must be the only statement (expects an EOF after the statement).
+query returns [SelectStatement ret]
+    :   SELECT s=hinted_select_node EOF {$ret=s;}
+    ;
+
+// Parses a single SQL statement (expects an EOF after the select statement).
+oneStatement returns [BindableStatement ret]
+    :   (SELECT s=hinted_select_node {$ret=s;} 
+    |    ns=non_select_node {$ret=ns;}
+        )
+    ;
+
+non_select_node returns [BindableStatement ret]
+@init{ contextStack.push(new ParseContext()); }
+    :  (s=upsert_node
+    |   s=delete_node
+    |   s=create_table_node
+    |   s=create_view_node
+    |   s=create_index_node
+    |   s=drop_table_node
+    |   s=drop_index_node
+    |   s=alter_index_node
+    |   s=alter_table_node
+    |	s=create_sequence_node
+    |	s=drop_sequence_node
+    |   s=explain_node) { contextStack.pop();  $ret = s; }
+    ;
+    
+explain_node returns [BindableStatement ret]
+    :   EXPLAIN q=oneStatement {$ret=factory.explain(q);}
+    ;
+
+// Parse a create table statement.
+create_table_node returns [CreateTableStatement ret]
+    :   CREATE TABLE (IF NOT ex=EXISTS)? t=from_table_name 
+        (LPAREN c=column_defs (pk=pk_constraint)? RPAREN)
+        (p=fam_properties)?
+        (SPLIT ON s=list_expressions)?
+        {ret = factory.createTable(t, p, c, pk, s, PTableType.TABLE, ex!=null, null, null, getBindCount()); }
+    ;
+
+// Parse a create view statement.
+create_view_node returns [CreateTableStatement ret]
+    :   CREATE VIEW (IF NOT ex=EXISTS)? t=from_table_name 
+        (LPAREN c=column_defs (pk=pk_constraint)? RPAREN)?
+        ( AS SELECT ASTERISK
+          FROM bt=from_table_name
+          (WHERE w=condition)? )?
+        (p=fam_properties)?
+        { ret = factory.createTable(t, p, c, pk, null, PTableType.VIEW, ex!=null, bt==null ? t : bt, w, getBindCount()); }
+    ;
+
+// Parse a create index statement.
+create_index_node returns [CreateIndexStatement ret]
+    :   CREATE INDEX (IF NOT ex=EXISTS)? i=index_name ON t=from_table_name
+        (LPAREN pk=index_pk_constraint RPAREN)
+        (INCLUDE (LPAREN icrefs=column_names RPAREN))?
+        (p=fam_properties)?
+        (SPLIT ON v=list_expressions)?
+        {ret = factory.createIndex(i, factory.namedTable(null,t), pk, icrefs, v, p, ex!=null, getBindCount()); }
+    ;
+
+// Parse a create sequence statement.
+create_sequence_node returns [CreateSequenceStatement ret]
+    :   CREATE SEQUENCE  (IF NOT ex=EXISTS)? t=from_table_name
+        (START WITH? s=int_literal_or_bind)?
+        (INCREMENT BY? i=int_literal_or_bind)?
+        (CACHE c=int_literal_or_bind)?
+    { $ret = factory.createSequence(t, s, i, c, ex!=null, getBindCount()); }
+    ;
+
+int_literal_or_bind returns [ParseNode ret]
+    : n=int_literal { $ret = n; }
+    | b=bind_expression { $ret = b; }
+    ;
+
+// Parse a drop sequence statement.
+drop_sequence_node returns [DropSequenceStatement ret]
+    :   DROP SEQUENCE  (IF ex=EXISTS)? t=from_table_name
+    { $ret = factory.dropSequence(t, ex!=null, getBindCount()); }
+    ;
+
+pk_constraint returns [PrimaryKeyConstraint ret]
+    :   COMMA? CONSTRAINT n=identifier PRIMARY KEY LPAREN cols=col_name_with_mod_list RPAREN { $ret = factory.primaryKey(n,cols); }
+    ;
+
+col_name_with_mod_list returns [List<Pair<ColumnName, ColumnModifier>> ret]
+@init{ret = new ArrayList<Pair<ColumnName, ColumnModifier>>(); }
+    :   p=col_name_with_mod {$ret.add(p);}  (COMMA p = col_name_with_mod {$ret.add(p);} )*
+;
+
+col_name_with_mod returns [Pair<ColumnName, ColumnModifier> ret]
+    :   f=identifier (order=ASC|order=DESC)? {$ret = Pair.newPair(factory.columnName(f), order == null ? null : ColumnModifier.fromDDLValue(order.getText()));}
+;
+
+index_pk_constraint returns [PrimaryKeyConstraint ret]
+    :   cols = col_def_name_with_mod_list {$ret = factory.primaryKey(null, cols); }
+    ;
+
+col_def_name_with_mod_list returns [List<Pair<ColumnName, ColumnModifier>> ret]
+@init{ret = new ArrayList<Pair<ColumnName, ColumnModifier>>(); }
+    :   p=col_def_name_with_mod {$ret.add(p);}  (COMMA p = col_def_name_with_mod {$ret.add(p);} )*
+;
+
+col_def_name_with_mod returns [Pair<ColumnName, ColumnModifier> ret]
+    :   c=column_name (order=ASC|order=DESC)? {$ret = Pair.newPair(c, order == null ? null : ColumnModifier.fromDDLValue(order.getText()));}
+;
+
+fam_properties returns [ListMultimap<String,Pair<String,Object>> ret]
+@init{ret = ArrayListMultimap.<String,Pair<String,Object>>create(); }
+    :  p=fam_prop_name EQ v=prop_value {$ret.put(p.getFamilyName(),new Pair<String,Object>(p.getPropertyName(),v));}  (COMMA p=fam_prop_name EQ v=prop_value {$ret.put(p.getFamilyName(),new Pair<String,Object>(p.getPropertyName(),v));} )*
+    ;
+
+fam_prop_name returns [PropertyName ret]
+    :   propName=identifier {$ret = factory.propertyName(propName); }
+    |   familyName=identifier DOT propName=identifier {$ret = factory.propertyName(familyName, propName); }
+    ;
+    
+prop_value returns [Object ret]
+    :   l=literal { $ret = l.getValue(); }
+    ;
+    
+column_name returns [ColumnName ret]
+    :   field=identifier {$ret = factory.columnName(field); }
+    |   family=identifier DOT field=identifier {$ret = factory.columnName(family, field); }
+    ;
+
+column_names returns [List<ColumnName> ret]
+@init{ret = new ArrayList<ColumnName>(); }
+    :  v = column_name {$ret.add(v);}  (COMMA v = column_name {$ret.add(v);} )*
+;
+
+	
+// Parse a drop table statement.
+drop_table_node returns [DropTableStatement ret]
+    :   DROP (v=VIEW | TABLE) (IF ex=EXISTS)? t=from_table_name
+        {ret = factory.dropTable(t, v==null ? PTableType.TABLE : PTableType.VIEW, ex!=null); }
+    ;
+
+// Parse a drop index statement
+drop_index_node returns [DropIndexStatement ret]
+    : DROP INDEX (IF ex=EXISTS)? i=index_name ON t=from_table_name
+      {ret = factory.dropIndex(i, t, ex!=null); }
+    ;
+
+// Parse a alter index statement
+alter_index_node returns [AlterIndexStatement ret]
+    : ALTER INDEX (IF ex=EXISTS)? i=index_name ON t=from_table_name s=(USABLE | UNUSABLE | REBUILD | DISABLE)
+      {ret = factory.alterIndex(factory.namedTable(null,factory.table(t.getSchemaName(),i.getName())), t.getTableName(), ex!=null, PIndexState.valueOf(SchemaUtil.normalizeIdentifier(s.getText()))); }
+    ;
+
+// Parse an alter table statement.
+alter_table_node returns [AlterTableStatement ret]
+    :   ALTER (TABLE | v=VIEW) t=from_table_name
+        ( (DROP COLUMN (IF ex=EXISTS)? c=column_names) | (ADD (IF NOT ex=EXISTS)? (d=column_defs) (p=properties)?) | (SET (p=properties)) )
+        { PTableType tt = v==null ? PTableType.TABLE : PTableType.VIEW; ret = ( c == null ? factory.addColumn(factory.namedTable(null,t), tt, d, ex!=null, p) : factory.dropColumn(factory.namedTable(null,t), tt, c, ex!=null) ); }
+    ;
+
+prop_name returns [String ret]
+    :   p=identifier {$ret = SchemaUtil.normalizeIdentifier(p); }
+    ;
+    
+properties returns [Map<String,Object> ret]
+@init{ret = new HashMap<String,Object>(); }
+    :  k=prop_name EQ v=prop_value {$ret.put(k,v);}  (COMMA k=prop_name EQ v=prop_value {$ret.put(k,v);} )*
+    ;
+
+column_defs returns [List<ColumnDef> ret]
+@init{ret = new ArrayList<ColumnDef>(); }
+    :  v = column_def {$ret.add(v);}  (COMMA v = column_def {$ret.add(v);} )*
+;
+
+column_def returns [ColumnDef ret]
+    :   c=column_name dt=identifier (LPAREN l=NUMBER (COMMA s=NUMBER)? RPAREN)? ar=ARRAY? (lsq=LSQUARE (a=NUMBER)? RSQUARE)? (n=NOT? NULL)? (pk=PRIMARY KEY (order=ASC|order=DESC)?)?
+        { $ret = factory.columnDef(c, dt, ar != null || lsq != null, a == null ? null :  Integer.parseInt( a.getText() ), n==null, 
+            l == null ? null : Integer.parseInt( l.getText() ),
+            s == null ? null : Integer.parseInt( s.getText() ),
+            pk != null, 
+            order == null ? null : ColumnModifier.fromDDLValue(order.getText()) ); }
+    ;
+
+dyn_column_defs returns [List<ColumnDef> ret]
+@init{ret = new ArrayList<ColumnDef>(); }
+    :  v = dyn_column_def {$ret.add(v);}  (COMMA v = dyn_column_def {$ret.add(v);} )*
+;
+
+dyn_column_def returns [ColumnDef ret]
+    :   c=column_name dt=identifier (LPAREN l=NUMBER (COMMA s=NUMBER)? RPAREN)? (lsq=LSQUARE (a=NUMBER)? RSQUARE)?
+        {$ret = factory.columnDef(c, dt, true,
+            l == null ? null : Integer.parseInt( l.getText() ),
+            s == null ? null : Integer.parseInt( s.getText() ),
+            false, 
+            null); }
+    ;
+
+dyn_column_name_or_def returns [ColumnDef ret]
+    :   c=column_name (dt=identifier (LPAREN l=NUMBER (COMMA s=NUMBER)? RPAREN)? )? (lsq=LSQUARE (a=NUMBER)? RSQUARE)?
+        {$ret = factory.columnDef(c, dt, true,
+            l == null ? null : Integer.parseInt( l.getText() ),
+            s == null ? null : Integer.parseInt( s.getText() ),
+            false, 
+            null); }
+    ;
+
+select_expression returns [SelectStatement ret]
+    :  SELECT s=select_node {$ret = s;}
+    ;
+    
+subquery_expression returns [ParseNode ret]
+    :  s=select_expression {$ret = factory.subquery(s);}
+    ;
+    
+// Parse a full select expression structure.
+select_node returns [SelectStatement ret]
+@init{ contextStack.push(new ParseContext()); }
+    :   (d=DISTINCT | ALL)? sel=select_list
+        FROM from=parseFrom
+        (WHERE where=condition)?
+        (GROUP BY group=group_by)?
+        (HAVING having=condition)?
+        (ORDER BY order=order_by)?
+        (LIMIT l=limit)?
+        { ParseContext context = contextStack.pop(); $ret = factory.select(from, null, d!=null, sel, where, group, having, order, l, getBindCount(), context.isAggregate()); }
+    ;
+
+// Parse a full select expression structure.
+hinted_select_node returns [SelectStatement ret]
+@init{ contextStack.push(new ParseContext()); }
+    :   (hint=hintClause)? 
+        s=select_node
+        { $ret = factory.select(s, hint); }
+    ;
+
+// Parse a full upsert expression structure.
+upsert_node returns [UpsertStatement ret]
+    :   UPSERT (hint=hintClause)? INTO t=from_table_name
+        (LPAREN p=upsert_column_refs RPAREN)?
+        ((VALUES LPAREN v=expression_terms RPAREN) | s=select_expression)
+        {ret = factory.upsert(factory.namedTable(null,t,p == null ? null : p.getFirst()), hint, p == null ? null : p.getSecond(), v, s, getBindCount()); }
+    ;
+
+upsert_column_refs returns [Pair<List<ColumnDef>,List<ColumnName>> ret]
+@init{ret = new Pair<List<ColumnDef>,List<ColumnName>>(new ArrayList<ColumnDef>(), new ArrayList<ColumnName>()); }
+    :  d=dyn_column_name_or_def { if (d.getDataType()!=null) { $ret.getFirst().add(d); } $ret.getSecond().add(d.getColumnDefName()); } 
+       (COMMA d=dyn_column_name_or_def { if (d.getDataType()!=null) { $ret.getFirst().add(d); } $ret.getSecond().add(d.getColumnDefName()); } )*
+;
+	
+// Parse a full delete expression structure.
+delete_node returns [DeleteStatement ret]
+    :   DELETE (hint=hintClause)? FROM t=from_table_name
+        (WHERE v=condition)?
+        (ORDER BY order=order_by)?
+        (LIMIT l=limit)?
+        {ret = factory.delete(factory.namedTable(null,t), hint, v, order, l, getBindCount()); }
+    ;
+
+limit returns [LimitNode ret]
+    : b=bind_expression { $ret = factory.limit(b); }
+    | l=int_literal { $ret = factory.limit(l); }
+    ;
+    
+hintClause returns [HintNode ret]
+    :  c=ML_HINT { $ret = factory.hint(c.getText()); }
+    ;
+
+// Parse the column/expression select list part of a select.
+select_list returns [List<AliasedNode> ret]
+@init{ret = new ArrayList<AliasedNode>();}
+    :   n=selectable {ret.add(n);} (COMMA n=selectable {ret.add(n);})*
+    |	ASTERISK { $ret = Collections.<AliasedNode>singletonList(factory.aliasedNode(null, factory.wildcard()));} // i.e. the '*' in 'select * from'    
+    ;
+
+// Parse either a select field or a sub select.
+selectable returns [AliasedNode ret]
+    :   field=expression (a=parseAlias)? { $ret = factory.aliasedNode(a, field); }
+    | 	familyName=identifier DOT ASTERISK { $ret = factory.aliasedNode(null, factory.family(familyName));} // i.e. the 'cf.*' in 'select cf.* from' cf being column family of an hbase table    
+    ;
+
+
+// Parse a group by statement
+group_by returns [List<ParseNode> ret]
+@init{ret = new ArrayList<ParseNode>();}
+    :   expr=expression { ret.add(expr); }
+        (COMMA expr = expression {ret.add(expr); })*
+    ;
+
+// Parse an order by statement
+order_by returns [List<OrderByNode> ret]
+@init{ret = new ArrayList<OrderByNode>();}
+    :   field=parseOrderByField { ret.add(field); }
+        (COMMA field = parseOrderByField {ret.add(field); })*
+    ;
+
+//parse the individual field for an order by clause
+parseOrderByField returns [OrderByNode ret]
+@init{boolean isAscending = true; boolean nullsLast = false;}
+    :   (expr = expression)
+        (ASC {isAscending = true;} | DESC {isAscending = false;})?
+        (NULLS (FIRST {nullsLast = false;} | LAST {nullsLast = true;}))?
+        { $ret = factory.orderBy(expr, nullsLast, isAscending); }
+    ;
+
+parseFrom returns [List<TableNode> ret]
+@init{ret = new ArrayList<TableNode>(4); }
+    :   t=table_ref {$ret.add(t);} (s=sub_table_ref { $ret.add(s); })*
+    ;
+    
+sub_table_ref returns [TableNode ret]
+    :   COMMA t=table_ref { $ret = t; }
+    |   t=join_spec { $ret = t; }
+    ;
+
+table_ref returns [TableNode ret]
+    :   n=bind_name ((AS)? alias=identifier)? { $ret = factory.bindTable(alias, factory.table(null,n)); } // TODO: review
+    |   t=from_table_name ((AS)? alias=identifier)? (LPAREN cdefs=dyn_column_defs RPAREN)? { $ret = factory.namedTable(alias,t,cdefs); }
+    |   LPAREN SELECT s=hinted_select_node RPAREN ((AS)? alias=identifier)? { $ret = factory.derivedTable(alias, s); }
+    ;
+
+join_spec returns [TableNode ret]
+    :   j=join_type JOIN t=table_ref ON e=condition { $ret = factory.join(j, e, t); }
+    ;
+
+join_type returns [JoinTableNode.JoinType ret]
+    :   INNER?   { $ret = JoinTableNode.JoinType.Inner; }
+    |   LEFT OUTER?   { $ret = JoinTableNode.JoinType.Left; }
+    |   RIGHT OUTER?  { $ret = JoinTableNode.JoinType.Right; }
+    |   FULL  OUTER?  { $ret = JoinTableNode.JoinType.Full; }
+    ;
+    
+parseAlias returns [String ret]
+    :   AS? alias=parseNoReserved { $ret = alias; }
+    ;
+
+// Parse a condition, such as used in a where clause - either a basic one, or an OR of (Single or AND) expressions
+condition returns [ParseNode ret]
+    :   e=condition_or { $ret = e; }
+    ;
+
+// A set of OR'd simple expressions
+condition_or returns [ParseNode ret]
+@init{List<ParseNode> l = new ArrayList<ParseNode>(4); }
+    :   i=condition_and {l.add(i);} (OR i=condition_and {l.add(i);})* { $ret = l.size() == 1 ? l.get(0) : factory.or(l); }
+    ;
+
+// A set of AND'd simple expressions
+condition_and returns [ParseNode ret]
+@init{List<ParseNode> l = new ArrayList<ParseNode>(4); }
+    :   i=condition_not {l.add(i);} (AND i=condition_not {l.add(i);})* { $ret = l.size() == 1 ? l.get(0) : factory.and(l); }
+    ;
+
+// NOT or parenthesis 
+condition_not returns [ParseNode ret]
+    :   (NOT? boolean_expr ) => n=NOT? e=boolean_expr { $ret = n == null ? e : factory.not(e); }
+    |   n=NOT? LPAREN e=condition RPAREN { $ret = n == null ? e : factory.not(e); }
+    ;
+
+boolean_expr returns [ParseNode ret]
+    :   l=expression ((EQ r=expression {$ret = factory.equal(l,r); } )
+                  |  ((NOEQ1 | NOEQ2) r=expression {$ret = factory.notEqual(l,r); } )
+                  |  (LT r=expression {$ret = factory.lt(l,r); } )
+                  |  (GT r=expression {$ret = factory.gt(l,r); } )
+                  |  (LT EQ r=expression {$ret = factory.lte(l,r); } )
+                  |  (GT EQ r=expression {$ret = factory.gte(l,r); } )
+                  |  (IS n=NOT? NULL {$ret = factory.isNull(l,n!=null); } )
+                  |  ( n=NOT? ((LIKE r=expression {$ret = factory.like(l,r,n!=null); } )
+                      |        (EXISTS LPAREN r=subquery_expression RPAREN {$ret = factory.exists(l,r,n!=null);} )
+                      |        (BETWEEN r1=expression AND r2=expression {$ret = factory.between(l,r1,r2,n!=null); } )
+                      |        ((IN ((r=bind_expression {$ret = factory.inList(Arrays.asList(l,r),n!=null);} )
+                                | (LPAREN r=subquery_expression RPAREN {$ret = factory.in(l,r,n!=null);} )
+                                | (v=list_expressions {List<ParseNode> il = new ArrayList<ParseNode>(v.size() + 1); il.add(l); il.addAll(v); $ret = factory.inList(il,n!=null);})
+                                )))
+                      ))
+                   |  { $ret = l; } )
+    ;
+
+bind_expression  returns [BindParseNode ret]
+    :   b=bind_name { $ret = factory.bind(b); }
+    ;
+    
+expression returns [ParseNode ret]
+    :   i=expression_add { $ret = i; }
+    ;
+
+expression_add returns [ParseNode ret]
+@init{List<ParseNode> l = new ArrayList<ParseNode>(4); }
+    :   i=expression_sub {l.add(i);} (PLUS i=expression_sub {l.add(i);})* { $ret = l.size() == 1 ? l.get(0) : factory.add(l); }
+    ;
+
+expression_sub returns [ParseNode ret]
+@init{List<ParseNode> l = new ArrayList<ParseNode>(4); }
+    :   i=expression_concat {l.add(i);} (MINUS i=expression_concat {l.add(i);})* { $ret = l.size() == 1 ? l.get(0) : factory.subtract(l); }
+    ;
+
+expression_concat returns [ParseNode ret]
+@init{List<ParseNode> l = new ArrayList<ParseNode>(4); }
+    :   i=expression_mult {l.add(i);} (CONCAT i=expression_mult {l.add(i);})* { $ret = l.size() == 1 ? l.get(0) : factory.concat(l); }
+    ;
+
+expression_mult returns [ParseNode ret]
+@init{List<ParseNode> l = new ArrayList<ParseNode>(4); }
+    :   i=expression_div {l.add(i);} (ASTERISK i=expression_div {l.add(i);})* { $ret = l.size() == 1 ? l.get(0) : factory.multiply(l); }
+    ;
+
+expression_div returns [ParseNode ret]
+@init{List<ParseNode> l = new ArrayList<ParseNode>(4); }
+    :   i=expression_negate {l.add(i);} (DIVIDE i=expression_negate {l.add(i);})* { $ret = l.size() == 1 ? l.get(0) : factory.divide(l); }
+    ;
+
+expression_negate returns [ParseNode ret]
+    :   m=MINUS? e=expression_term { $ret = m==null ? e : factory.negate(e); }
+    ;
+
+// The lowest level function, which includes literals, binds, but also parenthesized expressions, functions, and case statements.
+expression_term returns [ParseNode ret]
+    :   e=literal_or_bind_value { $ret = e; }
+    |   e=arrayable_expression_term (LSQUARE s=expression RSQUARE)?  { if (s == null) { $ret = e; } else { $ret = factory.arrayElemRef(Arrays.<ParseNode>asList(e,s)); } } 
+	;
+	    
+arrayable_expression_term returns [ParseNode ret]
+    :   field=identifier { $ret = factory.column(null,field,field); }
+    |   ex=ARRAY LSQUARE v=expression_terms RSQUARE {$ret = factory.upsertStmtArrayNode(v);}
+    |   tableName=table_name DOT field=identifier { $ret = factory.column(tableName, field, field); }
+    |   field=identifier LPAREN l=expression_list RPAREN wg=(WITHIN GROUP LPAREN ORDER BY l2=expression_terms (a=ASC | DESC) RPAREN)?
+        {
+            FunctionParseNode f = wg==null ? factory.function(field, l) : factory.function(field,l,l2,a!=null);
+            contextStack.peek().setAggregate(f.isAggregate());
+            $ret = f;
+        } 
+    |   field=identifier LPAREN t=ASTERISK RPAREN 
+        {
+            if (!isCountFunction(field)) {
+                throwRecognitionException(t); 
+            }
+            FunctionParseNode f = factory.function(field, LiteralParseNode.STAR);
+            contextStack.peek().setAggregate(f.isAggregate()); 
+            $ret = f;
+        } 
+    |   field=identifier LPAREN t=DISTINCT l=expression_list RPAREN 
+        {
+            FunctionParseNode f = factory.functionDistinct(field, l);
+            contextStack.peek().setAggregate(f.isAggregate());
+            $ret = f;
+        }
+    |   e=case_statement { $ret = e; }
+    |   LPAREN l=expression_terms RPAREN 
+    	{ 
+    		if(l.size() == 1) {
+    			$ret = l.get(0);
+    		}	
+    		else {
+    			$ret = factory.rowValueConstructor(l);
+    		}	 
+    	}
+    |   CAST e=expression AS dt=identifier { $ret = factory.cast(e, dt); }
+    |   (n=NEXT | CURRENT) VALUE FOR s=from_table_name { $ret = n==null ? factory.currentValueFor(s) : factory.nextValueFor(s);}    
+    ;
+
+expression_terms returns [List<ParseNode> ret]
+@init{ret = new ArrayList<ParseNode>(); }
+    :  e = expression {$ret.add(e);}  (COMMA e = expression {$ret.add(e);} )*
+;
+
+expression_list returns [List<ParseNode> ret]
+@init{ret = new ArrayList<ParseNode>(); }
+    :  (v = expression {$ret.add(v);})?  (COMMA v = expression {$ret.add(v);} )*
+;
+
+index_name returns [NamedNode ret]
+    :   name=identifier {$ret = factory.indexName(name); }
+    ;
+
+// TODO: figure out how not repeat this two times
+table_name returns [TableName ret]
+    :   t=identifier {$ret = factory.table(null, t); }
+    |   s=identifier DOT t=identifier {$ret = factory.table(s, t); }
+    ;
+
+// TODO: figure out how not repeat this two times
+from_table_name returns [TableName ret]
+    :   t=identifier {$ret = factory.table(null, t); }
+    |   s=identifier DOT t=identifier {$ret = factory.table(s, t); }
+    ;
+    
+// The lowest level function, which includes literals, binds, but also parenthesized expressions, functions, and case statements.
+literal_or_bind_value returns [ParseNode ret]
+    :   e=literal { $ret = e; }
+    |   b=bind_name { $ret = factory.bind(b); }    
+    ;
+
+// Get a string, integer, double, date, boolean, or NULL value.
+literal returns [LiteralParseNode ret]
+    :   t=STRING_LITERAL { ret = factory.literal(t.getText()); }
+    |   l=int_literal { ret = l; }
+    |   l=long_literal { ret = l; }
+    |   l=double_literal { ret = l; }
+    |   t=DECIMAL {
+            try {
+                ret = factory.literal(new BigDecimal(t.getText()));
+            } catch (NumberFormatException e) { // Shouldn't happen since we just parsed a decimal
+                throwRecognitionException(t);
+            }
+        }
+    |   NULL {ret = factory.literal(null);}
+    |   TRUE {ret = factory.literal(Boolean.TRUE);} 
+    |   FALSE {ret = factory.literal(Boolean.FALSE);}
+    ;
+    
+int_literal returns [LiteralParseNode ret]
+    :   n=NUMBER {
+            try {
+                Long v = Long.valueOf(n.getText());
+                if (v >= Integer.MIN_VALUE && v <= Integer.MAX_VALUE) {
+                    ret = factory.literal(v.intValue());
+                } else {
+                    ret = factory.literal(v);
+                }
+            } catch (NumberFormatException e) { // Shouldn't happen since we just parsed a number
+                throwRecognitionException(n);
+            }
+        }
+    ;
+
+long_literal returns [LiteralParseNode ret]
+    :   l=LONG {
+            try {
+                String lt = l.getText();
+                Long v = Long.valueOf(lt.substring(0, lt.length() - 1));
+                ret = factory.literal(v);
+            } catch (NumberFormatException e) { // Shouldn't happen since we just parsed a number
+                throwRecognitionException(l);
+            }
+        }
+    ;
+
+double_literal returns [LiteralParseNode ret]
+    :   d=DOUBLE {
+            try {
+                String dt = d.getText();
+                Double v = Double.valueOf(dt.substring(0, dt.length() - 1));
+                ret = factory.literal(v);
+            } catch (NumberFormatException e) { // Shouldn't happen since we just parsed a number
+                throwRecognitionException(d);
+            }
+        }
+    ;
+
+list_expressions returns [List<ParseNode> ret]
+@init{ret = new ArrayList<ParseNode>(); }
+    :   LPAREN  v = expression {$ret.add(v);}  (COMMA v = expression {$ret.add(v);} )* RPAREN
+;
+
+// parse a field, if it might be a bind name.
+table returns [String ret]
+    :   b=bind_name { $ret = b; }
+    |   n=parseNoReserved { $ret = n; }
+    ;
+
+// Bind names are a colon followed by 1+ letter/digits/underscores, or '?' (unclear how Oracle acutally deals with this, but we'll just treat it as a special bind)
+bind_name returns [String ret]
+    :   bname=BIND_NAME { String bnameStr = bname.getText().substring(1); updateBind(bnameStr); $ret = bnameStr; } 
+    |   QUESTION { $ret = nextBind(); } // TODO: only support this?
+    ;
+
+// Parse a field, includes line and column information.
+identifier returns [String ret]
+    :   c=parseNoReserved { $ret = c; }
+    ;
+
+parseNoReserved returns [String ret]
+    :   n=NAME { $ret = n.getText(); }
+    ;
+
+case_statement returns [ParseNode ret]
+@init{List<ParseNode> w = new ArrayList<ParseNode>(4);}
+    : CASE e1=expression (WHEN e2=expression THEN t=expression {w.add(t);w.add(factory.equal(e1,e2));})+ (ELSE el=expression {w.add(el);})? END {$ret = factory.caseWhen(w);}
+    | CASE (WHEN c=condition THEN t=expression {w.add(t);w.add(c);})+ (ELSE el=expression {w.add(el);})? END {$ret = factory.caseWhen(w);}
+    ;
+
+// --------------------------------------
+// The Lexer
+
+HINT_START: '/*+' ;
+COMMENT_START: '/*';
+COMMENT_AND_HINT_END: '*/' ;
+SL_COMMENT1: '//';
+SL_COMMENT2: '--';
+
+// Bind names start with a colon and followed by 1 or more letter/digit/underscores
+BIND_NAME
+    : COLON (LETTER|DIGIT|'_')+
+    ;
+
+// Valid names can have a single underscore, but not multiple
+// Turn back on literal testing, all names are literals.
+NAME
+    :    LETTER (FIELDCHAR)* ('\"' (DBL_QUOTE_CHAR)* '\"')?
+    |    '\"' (DBL_QUOTE_CHAR)* '\"'
+    ;
+
+// An integer number, positive or negative
+NUMBER
+    :   POSINTEGER
+    ;
+
+LONG
+    :   POSINTEGER ('L'|'l')
+    ;
+
+// Exponential format is not supported.
+DECIMAL
+    :   POSINTEGER? '.' POSINTEGER
+    ;
+
+DOUBLE
+    :   DECIMAL ('D'|'d')
+    ;
+
+DOUBLE_QUOTE
+    :   '"'
+    ;
+
+EQ
+    :   '='
+    ;
+
+LT
+    :   '<'
+    ;
+
+GT
+    :   '>'
+    ;
+
+DOUBLE_EQ
+    :   '=''='
+    ;
+
+NOEQ1
+    :   '!''='
+    ;
+
+NOEQ2
+    :   '<''>'
+    ;
+
+CONCAT
+    :   '|''|'
+    ;
+
+COMMA
+    :   ','
+    ;
+
+LPAREN
+    :   '('
+    ;
+
+RPAREN
+    :   ')'
+    ;
+
+SEMICOLON
+    :   ';'
+    ;
+
+COLON
+    :   ':'
+    ;
+
+QUESTION
+    :   '?'
+    ;
+
+LSQUARE
+    :   '['
+    ;
+
+RSQUARE
+    :   ']'
+    ;
+
+LCURLY
+    :   '{'
+    ;
+
+RCURLY
+    :   '}'
+    ;
+
+AT
+    :   '@'
+    ;
+
+TILDE
+    :   '~'
+    ;
+
+PLUS
+    :   '+'
+    ;
+
+MINUS
+    :   '-'
+    ;
+
+ASTERISK
+    :   '*'
+    ;
+
+DIVIDE
+    :   '/'
+    ;
+
+OUTER_JOIN
+    : '(' '+' ')'
+    ;
+// A FieldCharacter is a letter, digit, underscore, or a certain unicode section.
+fragment
+FIELDCHAR
+    :    LETTER
+    |    DIGIT
+    |    '_'
+    |    '\u0080'..'\ufffe'
+    ;
+
+// A Letter is a lower or upper case ascii character.
+fragment
+LETTER
+    :    'a'..'z'
+    |    'A'..'Z'
+    ;
+
+fragment
+POSINTEGER
+    :   DIGIT+
+    ;
+
+fragment
+DIGIT
+    :    '0'..'9'
+    ;
+
+// string literals
+STRING_LITERAL
+@init{ StringBuilder sb = new StringBuilder(); }
+    :   '\''
+    ( t=CHAR { sb.append(t.getText()); }
+    | t=CHAR_ESC { sb.append(getText()); }
+    )* '\'' { setText(sb.toString()); }
+    ;
+
+fragment
+CHAR
+    :   ( ~('\'' | '\\') )+
+    ;
+
+fragment
+DBL_QUOTE_CHAR
+    :   ( ~('\"') )+
+    ;
+
+// escape sequence inside a string literal
+fragment
+CHAR_ESC
+    :   '\\'
+        ( 'n'   { setText("\n"); }
+        | 'r'   { setText("\r"); }
+        | 't'   { setText("\t"); }
+        | 'b'   { setText("\b"); }
+        | 'f'   { setText("\f"); }
+        | '\"'  { setText("\""); }
+        | '\''  { setText("\'"); }
+        | '\\'  { setText("\\"); }
+        | '_'   { setText("\\_"); }
+        | '%'   { setText("\\\%"); }
+        )
+    |   '\'\''  { setText("\'"); }
+    ;
+
+// whitespace (skip)
+WS
+    :   ( ' ' | '\t' ) { $channel=HIDDEN; }
+    ;
+    
+EOL
+    :  ('\r' | '\n')
+    { skip(); }
+    ;
+
+// Keep everything in comment in a case sensitive manner
+ML_HINT
+@init{ StringBuilder sb = new StringBuilder(); }
+    : h=HINT_START ( options {greedy=false;} : t=.)*  { sb.append($text); }  COMMENT_AND_HINT_END
+    { setText(sb.substring(h.getText().length())); } // Get rid of the HINT_START text
+    ;
+
+ML_COMMENT
+    : COMMENT_START (~PLUS) ( options {greedy=false;} : . )* COMMENT_AND_HINT_END
+    { skip(); }
+    ;
+
+SL_COMMENT
+    : (SL_COMMENT1 | SL_COMMENT2) ( options {greedy=false;} : . )* EOL
+    { skip(); }
+    ;
+
+DOT
+    : '.'
+    ;
+

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/CapturingAbortable.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/CapturingAbortable.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/CapturingAbortable.java
new file mode 100644
index 0000000..5de5428
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/CapturingAbortable.java
@@ -0,0 +1,68 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index;
+
+import org.apache.hadoop.hbase.Abortable;
+
+/**
+ * {@link Abortable} that can rethrow the cause of the abort.
+ */
+public class CapturingAbortable implements Abortable {
+
+  private Abortable delegate;
+  private Throwable cause;
+  private String why;
+
+  public CapturingAbortable(Abortable delegate) {
+    this.delegate = delegate;
+  }
+
+  @Override
+  public void abort(String why, Throwable e) {
+    if (delegate.isAborted()) {
+      return;
+    }
+    this.why = why;
+    this.cause = e;
+    delegate.abort(why, e);
+
+  }
+
+  @Override
+  public boolean isAborted() {
+    return delegate.isAborted();
+  }
+
+  /**
+   * Throw the cause of the abort, if <tt>this</tt> was aborted. If there was an exception causing
+   * the abort, re-throws that. Otherwise, just throws a generic {@link Exception} with the reason
+   * why the abort was caused.
+   * @throws Throwable the cause of the abort.
+   */
+  public void throwCauseIfAborted() throws Throwable {
+    if (!this.isAborted()) {
+      return;
+    }
+    if (cause == null) {
+      throw new Exception(why);
+    }
+    throw cause;
+  }
+}
\ No newline at end of file


[34/51] [partial] Initial commit of master branch from github

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..677328e
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
@@ -0,0 +1,340 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.exception;
+
+import java.sql.SQLException;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.index.util.IndexManagementUtil;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.schema.AmbiguousColumnException;
+import org.apache.phoenix.schema.AmbiguousTableException;
+import org.apache.phoenix.schema.ColumnAlreadyExistsException;
+import org.apache.phoenix.schema.ColumnFamilyNotFoundException;
+import org.apache.phoenix.schema.ColumnNotFoundException;
+import org.apache.phoenix.schema.ConcurrentTableMutationException;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.ReadOnlyTableException;
+import org.apache.phoenix.schema.SequenceAlreadyExistsException;
+import org.apache.phoenix.schema.SequenceNotFoundException;
+import org.apache.phoenix.schema.TableAlreadyExistsException;
+import org.apache.phoenix.schema.TableNotFoundException;
+import org.apache.phoenix.schema.TypeMismatchException;
+import org.apache.phoenix.util.MetaDataUtil;
+
+import com.google.common.collect.Maps;
+
+
+/**
+ * Various SQLException Information. Including a vender-specific errorcode and a standard SQLState.
+ * 
+ * @author zhuang
+ * @since 1.0
+ */
+public enum SQLExceptionCode {
+
+    /**
+     * Connection Exception (errorcode 01, sqlstate 08)
+     */
+    IO_EXCEPTION(101, "08000", "Unexpected IO exception."),
+    MALFORMED_CONNECTION_URL(102, "08001", "Malformed connection url."),
+    CANNOT_ESTABLISH_CONNECTION(103, "08004", "Unable to establish connection."),
+    
+    /**
+     * Data Exception (errorcode 02, sqlstate 22)
+     */
+    ILLEGAL_DATA(201, "22000", "Illegal data."),
+    DIVIDE_BY_ZERO(202, "22012", "Divide by zero."),
+    TYPE_MISMATCH(203, "22005", "Type mismatch.", new Factory() {
+        @Override
+        public SQLException newException(SQLExceptionInfo info) {
+            return new TypeMismatchException(info.getMessage());
+        }
+    }),
+    VALUE_IN_UPSERT_NOT_CONSTANT(204, "22008", "Values in UPSERT must evaluate to a constant."),
+    MALFORMED_URL(205, "22009", "Malformed URL."),
+    DATA_INCOMPATIBLE_WITH_TYPE(206, "22003", "The value is outside the range for the data type."),
+    MISSING_CHAR_LENGTH(207, "22003", "Missing length for CHAR."),
+    NONPOSITIVE_CHAR_LENGTH(208, "22003", "CHAR or VARCHAR must have a positive length."),
+    DECIMAL_PRECISION_OUT_OF_RANGE(209, "22003", "Decimal precision outside of range. Should be within 1 and " + PDataType.MAX_PRECISION + "."),
+    MISSING_BINARY_LENGTH(210, "22003", "Missing length for BINARY."),
+    NONPOSITIVE_BINARY_LENGTH(211, "22003", "BINARY must have a positive length."),
+    SERVER_ARITHMETIC_ERROR(212, "22012", "Arithmetic error on server."),
+    VALUE_OUTSIDE_RANGE(213,"22003","Value outside range."),
+    VALUE_IN_LIST_NOT_CONSTANT(214, "22008", "Values in IN must evaluate to a constant."),
+    
+    /**
+     * Constraint Violation (errorcode 03, sqlstate 23)
+     */
+    CONCURRENT_TABLE_MUTATION(301, "23000", "Concurrent modification to table.", new Factory() {
+        @Override
+        public SQLException newException(SQLExceptionInfo info) {
+            return new ConcurrentTableMutationException(info.getSchemaName(), info.getTableName());
+        }
+    }),
+    CANNOT_INDEX_COLUMN_ON_TYPE(302, "23100", "The column cannot be index due to its type."),
+    
+    /**
+     * Invalid Cursor State (errorcode 04, sqlstate 24)
+     */
+    CURSOR_BEFORE_FIRST_ROW(401, "24015","Cursor before first row."),
+    CURSOR_PAST_LAST_ROW(402, "24016", "Cursor past last row."),
+    
+    /**
+     * Syntax Error or Access Rule Violation (errorcode 05, sqlstate 42)
+     */
+    AMBIGUOUS_TABLE(501, "42000", "Table name exists in more than one table schema and is used without being qualified.", new Factory() {
+        @Override
+        public SQLException newException(SQLExceptionInfo info) {
+            return new AmbiguousTableException(info.getTableName(), info.getRootCause());
+        }
+    }),
+    AMBIGUOUS_COLUMN(502, "42702", "Column reference ambiguous or duplicate names.", new Factory() {
+        @Override
+        public SQLException newException(SQLExceptionInfo info) {
+            return new AmbiguousColumnException(info.getColumnName(), info.getRootCause());
+        }
+    }),
+    INDEX_MISSING_PK_COLUMNS(503, "42602", "Index table missing PK Columns."),
+     COLUMN_NOT_FOUND(504, "42703", "Undefined column.", new Factory() {
+        @Override
+        public SQLException newException(SQLExceptionInfo info) {
+            return new ColumnNotFoundException(info.getSchemaName(), info.getTableName(), info.getFamilyName(), info.getColumnName());
+        }
+    }),
+    READ_ONLY_TABLE(505, "42000", "Table is read only.", new Factory() {
+        @Override
+        public SQLException newException(SQLExceptionInfo info) {
+            return new ReadOnlyTableException(info.getMessage(), info.getSchemaName(), info.getTableName(), info.getFamilyName());
+        }
+    }),
+    CANNOT_DROP_PK(506, "42817", "Primary key column may not be dropped."),
+    CANNOT_CONVERT_TYPE(507, "42846", "Cannot convert type."),
+    PRIMARY_KEY_MISSING(509, "42888", "The table does not have a primary key."),
+    PRIMARY_KEY_ALREADY_EXISTS(510, "42889", "The table already has a primary key."),
+    ORDER_BY_NOT_IN_SELECT_DISTINCT(511, "42890", "All ORDER BY expressions must appear in SELECT DISTINCT:"),
+    INVALID_PRIMARY_KEY_CONSTRAINT(512, "42891", "Invalid column reference in primary key constraint"),
+    ARRAY_NOT_ALLOWED_IN_PRIMARY_KEY(513, "42892", "Array type not allowed as primary key constraint"),
+    COLUMN_EXIST_IN_DEF(514, "42892", "A duplicate column name was detected in the object definition or ALTER TABLE statement.", new Factory() {
+        @Override
+        public SQLException newException(SQLExceptionInfo info) {
+            return new ColumnAlreadyExistsException(info.getSchemaName(), info.getTableName(), info.getColumnName());
+        }
+    }),
+    ORDER_BY_ARRAY_NOT_SUPPORTED(515, "42893", "ORDER BY of an array type is not allowed"),
+    NON_EQUALITY_ARRAY_COMPARISON(516, "42894", "Array types may only be compared using = or !="),
+    
+    /** 
+     * HBase and Phoenix specific implementation defined sub-classes.
+     * Column family related exceptions.
+     * 
+     * For the following exceptions, use errorcode 10.
+     */
+    COLUMN_FAMILY_NOT_FOUND(1001, "42I01", "Undefined column family.", new Factory() {
+        @Override
+        public SQLException newException(SQLExceptionInfo info) {
+            return new ColumnFamilyNotFoundException(info.getFamilyName());
+        }
+    }),
+    PROPERTIES_FOR_FAMILY(1002, "42I02","Properties may not be defined for an unused family name."),
+    // Primary/row key related exceptions.
+    PRIMARY_KEY_WITH_FAMILY_NAME(1003, "42J01", "Primary key columns must not have a family name."),
+    PRIMARY_KEY_OUT_OF_ORDER(1004, "42J02", "Order of columns in primary key constraint must match the order in which they're declared."),
+    VARBINARY_IN_ROW_KEY(1005, "42J03", "The VARBINARY type can only be used as the last part of a multi-part row key."),
+    NOT_NULLABLE_COLUMN_IN_ROW_KEY(1006, "42J04", "Only nullable columns may be added to a multi-part row key."),
+    VARBINARY_LAST_PK(1015, "42J04", "Cannot add column to table when the last PK column is of type VARBINARY."),
+    NULLABLE_FIXED_WIDTH_LAST_PK(1023, "42J04", "Cannot add column to table when the last PK column is nullable and fixed width."),
+    CANNOT_MODIFY_VIEW_PK(1036, "42J04", "Cannot modify the primary key of a VIEW."),
+    BASE_TABLE_COLUMN(1037, "42J04", "Cannot modify columns of base table used by tenant-specific tables."),
+    // Key/value column related errors
+    KEY_VALUE_NOT_NULL(1007, "42K01", "A key/value column may not be declared as not null."),
+    // View related errors.
+    VIEW_WITH_TABLE_CONFIG(1008, "42L01", "A view may not contain table configuration properties."),
+    VIEW_WITH_PROPERTIES(1009, "42L02", "Properties may not be defined for a view."),
+    // Table related errors that are not in standard code.
+    CANNOT_MUTATE_TABLE(1010, "42M01", "Not allowed to mutate table."),
+    UNEXPECTED_MUTATION_CODE(1011, "42M02", "Unexpected mutation code."),
+    TABLE_UNDEFINED(1012, "42M03", "Table undefined.", new Factory() {
+        @Override
+        public SQLException newException(SQLExceptionInfo info) {
+            return new TableNotFoundException(info.getSchemaName(), info.getTableName());
+        }
+    }),
+    TABLE_ALREADY_EXIST(1013, "42M04", "Table already exists.", new Factory() {
+        @Override
+        public SQLException newException(SQLExceptionInfo info) {
+            return new TableAlreadyExistsException(info.getSchemaName(), info.getTableName());
+        }
+    }),
+
+    // Syntax error
+    TYPE_NOT_SUPPORTED_FOR_OPERATOR(1014, "42Y01", "The operator does not support the operand type."),
+    AGGREGATE_IN_GROUP_BY(1016, "42Y26", "Aggregate expressions may not be used in GROUP BY."),
+    AGGREGATE_IN_WHERE(1017, "42Y26", "Aggregate may not be used in WHERE."),
+    AGGREGATE_WITH_NOT_GROUP_BY_COLUMN(1018, "42Y27", "Aggregate may not contain columns not in GROUP BY."),
+    ONLY_AGGREGATE_IN_HAVING_CLAUSE(1019, "42Y26", "Only aggregate maybe used in the HAVING clause."),
+    UPSERT_COLUMN_NUMBERS_MISMATCH(1020, "42Y60", "Number of columns upserting must match number of values."),
+    // Table properties exception.
+    INVALID_BUCKET_NUM(1021, "42Y80", "Salt bucket numbers should be with 1 and 256."),
+    NO_SPLITS_ON_SALTED_TABLE(1022, "42Y81", "Should not specify split points on salted table with default row key order."),
+    SALT_ONLY_ON_CREATE_TABLE(1024, "42Y83", "Salt bucket number may only be specified when creating a table."),
+    SET_UNSUPPORTED_PROP_ON_ALTER_TABLE(1025, "42Y84", "Unsupported property set in ALTER TABLE command."),
+    CANNOT_ADD_NOT_NULLABLE_COLUMN(1038, "42Y84", "Only nullable columns may be added for a pre-existing table."),
+    NO_MUTABLE_INDEXES(1026, "42Y85", "Mutable secondary indexes are only supported for HBase version " + MetaDataUtil.decodeHBaseVersionAsString(PhoenixDatabaseMetaData.MUTABLE_SI_VERSION_THRESHOLD) + " and above."),
+    NO_DELETE_IF_IMMUTABLE_INDEX(1027, "42Y86", "Delete not allowed on a table with IMMUTABLE_ROW with non PK column in index."),
+    INVALID_INDEX_STATE_TRANSITION(1028, "42Y87", "Invalid index state transition."),
+    INVALID_MUTABLE_INDEX_CONFIG(1029, "42Y88", "Mutable secondary indexes must have the " 
+            + IndexManagementUtil.WAL_EDIT_CODEC_CLASS_KEY + " property set to " 
+            +  IndexManagementUtil.INDEX_WAL_EDIT_CODEC_CLASS_NAME + " in the hbase-sites.xml of every region server"),
+            
+            
+    CANNOT_CREATE_TENANT_SPECIFIC_TABLE(1030, "42Y89", "Cannot create table for tenant-specific connection"),
+    CANNOT_DEFINE_PK_FOR_VIEW(1031, "42Y90", "Defining PK columns for a VIEW is not allowed."),
+    DEFAULT_COLUMN_FAMILY_ONLY_ON_CREATE_TABLE(1034, "42Y93", "Default column family may only be specified when creating a table."),
+    INSUFFICIENT_MULTI_TENANT_COLUMNS(1040, "42Y96", "A MULTI_TENANT table must have 2 or more PK columns with the first column being VARCHAR or CHAR."),
+    VIEW_WHERE_IS_CONSTANT(1045, "43A02", "WHERE clause in VIEW should not evaluate to a constant."),
+    CANNOT_UPDATE_VIEW_COLUMN(1046, "43A03", "Column used in WHERE clause of VIEW may not be updated."),
+        
+    /** Sequence related */
+    SEQUENCE_ALREADY_EXIST(1200, "42Z00", "Sequence already exists.", new Factory() {
+        @Override
+        public SQLException newException(SQLExceptionInfo info) {
+            return new SequenceAlreadyExistsException(info.getSchemaName(), info.getTableName());
+        }
+    }),
+    SEQUENCE_UNDEFINED(1201, "42Z01", "Sequence undefined.", new Factory() {
+        @Override
+        public SQLException newException(SQLExceptionInfo info) {
+            return new SequenceNotFoundException(info.getSchemaName(), info.getTableName());
+        }
+    }),
+    STARTS_WITH_MUST_BE_CONSTANT(1202, "42Z02", "Sequence STARTS WITH value must be an integer or long constant."),
+    INCREMENT_BY_MUST_BE_CONSTANT(1203, "42Z03", "Sequence INCREMENT BY value must be an integer or long constant."),
+    CACHE_MUST_BE_NON_NEGATIVE_CONSTANT(1204, "42Z04", "Sequence CACHE value must be a non negative integer constant."),
+    INVALID_USE_OF_NEXT_VALUE_FOR(1205, "42Z05", "NEXT VALUE FOR may only be used as in a SELECT or an UPSERT VALUES expression."),
+    CANNOT_CALL_CURRENT_BEFORE_NEXT_VALUE(1206, "42Z06", "NEXT VALUE FOR must be called before CURRENT VALUE FOR is called."),
+    EMPTY_SEQUENCE_CACHE(1207, "42Z07", "No more cached sequence values"),
+
+    /** Parser error. (errorcode 06, sqlState 42P) */
+    PARSER_ERROR(601, "42P00", "Syntax error.", Factory.SYTAX_ERROR),
+    MISSING_TOKEN(602, "42P00", "Syntax error.", Factory.SYTAX_ERROR),
+    UNWANTED_TOKEN(603, "42P00", "Syntax error.", Factory.SYTAX_ERROR),
+    MISMATCHED_TOKEN(604, "42P00", "Syntax error.", Factory.SYTAX_ERROR),
+    UNKNOWN_FUNCTION(605, "42P00", "Syntax error.", Factory.SYTAX_ERROR),
+    
+    /**
+     * Implementation defined class. Execution exceptions (errorcode 11, sqlstate XCL). 
+     */
+    RESULTSET_CLOSED(1101, "XCL01", "ResultSet is closed."),
+    GET_TABLE_REGIONS_FAIL(1102, "XCL02", "Cannot get all table regions"),
+    EXECUTE_QUERY_NOT_APPLICABLE(1103, "XCL03", "executeQuery may not be used."),
+    EXECUTE_UPDATE_NOT_APPLICABLE(1104, "XCL03", "executeUpdate may not be used."),
+    SPLIT_POINT_NOT_CONSTANT(1105, "XCL04", "Split points must be constants."),
+    
+    /**
+     * Implementation defined class. Phoenix internal error. (errorcode 20, sqlstate INT).
+     */
+    CANNOT_CALL_METHOD_ON_TYPE(2001, "INT01", "Cannot call method on the argument type."),
+    CLASS_NOT_UNWRAPPABLE(2002, "INT03", "Class not unwrappable"),
+    PARAM_INDEX_OUT_OF_BOUND(2003, "INT04", "Parameter position is out of range."),
+    PARAM_VALUE_UNBOUND(2004, "INT05", "Parameter value unbound"),
+    INTERRUPTED_EXCEPTION(2005, "INT07", "Interrupted exception."),
+    INCOMPATIBLE_CLIENT_SERVER_JAR(2006, "INT08", "Incompatible jars detected between client and server."),
+    OUTDATED_JARS(2007, "INT09", "Outdated jars."),
+    INDEX_METADATA_NOT_FOUND(2008, "INT10", "Unable to find cached index metadata. "),
+    UNKNOWN_ERROR_CODE(2009, "INT11", "Unknown error code"),
+    ;
+
+    private final int errorCode;
+    private final String sqlState;
+    private final String message;
+    private final Factory factory;
+
+    private SQLExceptionCode(int errorCode, String sqlState, String message) {
+        this(errorCode, sqlState, message, Factory.DEFAULTY);
+    }
+
+    private SQLExceptionCode(int errorCode, String sqlState, String message, Factory factory) {
+        this.errorCode = errorCode;
+        this.sqlState = sqlState;
+        this.message = message;
+        this.factory = factory;
+    }
+
+    public String getSQLState() {
+        return sqlState;
+    }
+
+    public String getMessage() {
+        return message;
+    }
+
+    public int getErrorCode() {
+        return errorCode;
+    }
+
+    @Override
+    public String toString() {
+        return "ERROR " + errorCode + " (" + sqlState + "): " + message;
+    }
+
+    public Factory getExceptionFactory() {
+        return factory;
+    }
+
+    public static interface Factory {
+        public static final Factory DEFAULTY = new Factory() {
+
+            @Override
+            public SQLException newException(SQLExceptionInfo info) {
+                return new SQLException(info.toString(), info.getCode().getSQLState(), info.getCode().getErrorCode(), info.getRootCause());
+            }
+            
+        };
+        public static final Factory SYTAX_ERROR = new Factory() {
+
+            @Override
+            public SQLException newException(SQLExceptionInfo info) {
+                return new PhoenixParserException(info.getMessage(), info.getRootCause());
+            }
+            
+        };
+        public SQLException newException(SQLExceptionInfo info);
+    }
+    
+    private static final Map<Integer,SQLExceptionCode> errorCodeMap = Maps.newHashMapWithExpectedSize(SQLExceptionCode.values().length);
+    static {
+        for (SQLExceptionCode code : SQLExceptionCode.values()) {
+            SQLExceptionCode otherCode = errorCodeMap.put(code.getErrorCode(), code);
+            if (otherCode != null) {
+                throw new IllegalStateException("Duplicate error code for " + code + " and " + otherCode);
+            }
+        }
+    }
+    
+    public static SQLExceptionCode fromErrorCode(int errorCode) throws SQLException {
+        SQLExceptionCode code = errorCodeMap.get(errorCode);
+        if (code == null) {
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode.UNKNOWN_ERROR_CODE)
+            .setMessage(Integer.toString(errorCode)).build().buildException();
+        }
+        return code;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionInfo.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionInfo.java b/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionInfo.java
new file mode 100644
index 0000000..e025054
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionInfo.java
@@ -0,0 +1,166 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.exception;
+
+import java.sql.SQLException;
+
+import org.apache.phoenix.util.SchemaUtil;
+
+
+/**
+ * Object serves as a closure of all coordinate information for SQLException messages.
+ * 
+ * @author zhuang
+ * @since 1.0
+ */
+public class SQLExceptionInfo {
+
+    /**
+     * Constants used in naming exception location.
+     */
+    public static final String SCHEMA_NAME = "schemaName";
+    public static final String TABLE_NAME = "tableName";
+    public static final String FAMILY_NAME = "familyName";
+    public static final String COLUMN_NAME = "columnName";
+
+    private final Throwable rootCause;
+    private final SQLExceptionCode code; // Should always have one.
+    private final String message;
+    private final String schemaName;
+    private final String tableName;
+    private final String familyName;
+    private final String columnName;
+
+    public static class Builder {
+
+        private Throwable rootCause;
+        private SQLExceptionCode code; // Should always have one.
+        private String message;
+        private String schemaName;
+        private String tableName;
+        private String familyName;
+        private String columnName;
+
+        public Builder(SQLExceptionCode code) {
+            this.code = code;
+        }
+
+        public Builder setRootCause(Throwable t) {
+            this.rootCause = t;
+            return this;
+        }
+
+        public Builder setMessage(String message) {
+            this.message = message;
+            return this;
+        }
+
+        public Builder setSchemaName(String schemaName) {
+            this.schemaName = schemaName;
+            return this;
+        }
+
+        public Builder setTableName(String tableName) {
+            this.tableName = tableName;
+            return this;
+        }
+
+        public Builder setFamilyName(String familyName) {
+            this.familyName = familyName;
+            return this;
+        }
+
+        public Builder setColumnName(String columnName) {
+            this.columnName = columnName;
+            return this;
+        }
+
+        public SQLExceptionInfo build() {
+            return new SQLExceptionInfo(this);
+        }
+
+        @Override
+        public String toString() {
+            return code.toString();
+        }
+    }
+
+    private SQLExceptionInfo(Builder builder) {
+        code = builder.code;
+        rootCause = builder.rootCause;
+        message = builder.message;
+        schemaName = builder.schemaName;
+        tableName = builder.tableName;
+        familyName = builder.familyName;
+        columnName = builder.columnName;
+    }
+
+    @Override
+    public String toString() {
+        StringBuilder builder = new StringBuilder(code.toString());
+        if (message != null) {
+            builder.append(" ").append(message);
+        }
+        String columnDisplayName = SchemaUtil.getMetaDataEntityName(schemaName, tableName, familyName, columnName);
+        if (columnName != null) {
+            builder.append(" ").append(COLUMN_NAME).append("=").append(columnDisplayName);
+        } else if (familyName != null) {
+            builder.append(" ").append(FAMILY_NAME).append("=").append(columnDisplayName);
+        } else if (tableName != null) {
+            builder.append(" ").append(TABLE_NAME).append("=").append(columnDisplayName);
+        } else if (schemaName != null) {
+            builder.append(" ").append(SCHEMA_NAME).append("=").append(columnDisplayName);
+        }
+        return builder.toString();
+    }
+
+    public SQLException buildException() {
+        return code.getExceptionFactory().newException(this);
+    }
+
+    public Throwable getRootCause() {
+        return rootCause;
+    }
+
+    public String getSchemaName() {
+        return schemaName;
+    }
+
+    public String getTableName() {
+        return tableName;
+    }
+
+    public String getFamilyName() {
+        return familyName;
+    }
+
+    public String getColumnName() {
+        return columnName;
+    }
+
+    public SQLExceptionCode getCode() {
+        return code;
+    }
+
+    public String getMessage() {
+        return message;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/exception/UnknownFunctionException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/exception/UnknownFunctionException.java b/phoenix-core/src/main/java/org/apache/phoenix/exception/UnknownFunctionException.java
new file mode 100644
index 0000000..1f66992
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/exception/UnknownFunctionException.java
@@ -0,0 +1,37 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.exception;
+
+/**
+ * Thrown by ParseNodeFactory when it could not identify a node as a valid function.
+ */
+public class UnknownFunctionException extends RuntimeException {
+    private static final long serialVersionUID = 1L;
+    private final String funcName;
+
+    public UnknownFunctionException(String funcName) {
+        super();
+        this.funcName = funcName;
+    }
+
+    public String getFuncName() {
+        return funcName;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/exception/ValueTypeIncompatibleException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/exception/ValueTypeIncompatibleException.java b/phoenix-core/src/main/java/org/apache/phoenix/exception/ValueTypeIncompatibleException.java
new file mode 100644
index 0000000..403c009
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/exception/ValueTypeIncompatibleException.java
@@ -0,0 +1,38 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.exception;
+
+import org.apache.phoenix.schema.IllegalDataException;
+import org.apache.phoenix.schema.PDataType;
+
+
+public class ValueTypeIncompatibleException extends IllegalDataException {
+    private static final long serialVersionUID = 1L;
+    private static SQLExceptionCode code = SQLExceptionCode.DATA_INCOMPATIBLE_WITH_TYPE;
+
+    public ValueTypeIncompatibleException(PDataType type, Integer precision, Integer scale) {
+        super(new SQLExceptionInfo.Builder(code).setMessage(getTypeDisplayString(type, precision, scale))
+                .build().toString());
+    }
+
+    private static String getTypeDisplayString(PDataType type, Integer precision, Integer scale) {
+        return type.toString() + "(" + precision + "," + scale + ")";
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/execute/AggregatePlan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/AggregatePlan.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/AggregatePlan.java
new file mode 100644
index 0000000..b16a4e1
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/AggregatePlan.java
@@ -0,0 +1,171 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.execute;
+
+
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.phoenix.compile.GroupByCompiler.GroupBy;
+import org.apache.phoenix.compile.OrderByCompiler.OrderBy;
+import org.apache.phoenix.compile.RowProjector;
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.coprocessor.UngroupedAggregateRegionObserver;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.OrderByExpression;
+import org.apache.phoenix.expression.RowKeyExpression;
+import org.apache.phoenix.expression.aggregator.Aggregators;
+import org.apache.phoenix.iterate.AggregatingResultIterator;
+import org.apache.phoenix.iterate.ConcatResultIterator;
+import org.apache.phoenix.iterate.DistinctAggregatingResultIterator;
+import org.apache.phoenix.iterate.FilterAggregatingResultIterator;
+import org.apache.phoenix.iterate.GroupedAggregatingResultIterator;
+import org.apache.phoenix.iterate.LimitingResultIterator;
+import org.apache.phoenix.iterate.MergeSortRowKeyResultIterator;
+import org.apache.phoenix.iterate.OrderedAggregatingResultIterator;
+import org.apache.phoenix.iterate.OrderedResultIterator;
+import org.apache.phoenix.iterate.ParallelIterators;
+import org.apache.phoenix.iterate.ParallelIterators.ParallelIteratorFactory;
+import org.apache.phoenix.iterate.PeekingResultIterator;
+import org.apache.phoenix.iterate.ResultIterator;
+import org.apache.phoenix.iterate.SequenceResultIterator;
+import org.apache.phoenix.iterate.SpoolingResultIterator;
+import org.apache.phoenix.iterate.UngroupedAggregatingResultIterator;
+import org.apache.phoenix.parse.FilterableStatement;
+import org.apache.phoenix.query.KeyRange;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.TableRef;
+
+
+
+/**
+ *
+ * Query plan for aggregating queries
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class AggregatePlan extends BasicQueryPlan {
+    private final Aggregators aggregators;
+    private final Expression having;
+    private List<KeyRange> splits;
+
+    public AggregatePlan(
+            StatementContext context, FilterableStatement statement, TableRef table, RowProjector projector,
+            Integer limit, OrderBy orderBy, ParallelIteratorFactory parallelIteratorFactory, GroupBy groupBy,
+            Expression having) {
+        super(context, statement, table, projector, context.getBindManager().getParameterMetaData(), limit, orderBy, groupBy, parallelIteratorFactory);
+        this.having = having;
+        this.aggregators = context.getAggregationManager().getAggregators();
+    }
+
+    @Override
+    public List<KeyRange> getSplits() {
+        return splits;
+    }
+
+    private static class OrderingResultIteratorFactory implements ParallelIteratorFactory {
+        private final QueryServices services;
+        
+        public OrderingResultIteratorFactory(QueryServices services) {
+            this.services = services;
+        }
+        @Override
+        public PeekingResultIterator newIterator(ResultIterator scanner) throws SQLException {
+            Expression expression = RowKeyExpression.INSTANCE;
+            OrderByExpression orderByExpression = new OrderByExpression(expression, false, true);
+            int threshold = services.getProps().getInt(QueryServices.SPOOL_THRESHOLD_BYTES_ATTRIB, QueryServicesOptions.DEFAULT_SPOOL_THRESHOLD_BYTES);
+            return new OrderedResultIterator(scanner, Collections.<OrderByExpression>singletonList(orderByExpression), threshold);
+        }
+    }
+
+    private static class WrappingResultIteratorFactory implements ParallelIteratorFactory {
+        private final ParallelIteratorFactory innerFactory;
+        private final ParallelIteratorFactory outerFactory;
+        
+        public WrappingResultIteratorFactory(ParallelIteratorFactory innerFactory, ParallelIteratorFactory outerFactory) {
+            this.innerFactory = innerFactory;
+            this.outerFactory = outerFactory;
+        }
+        @Override
+        public PeekingResultIterator newIterator(ResultIterator scanner) throws SQLException {
+            PeekingResultIterator iterator = innerFactory.newIterator(scanner);
+            return outerFactory.newIterator(iterator);
+        }
+    }
+
+    private ParallelIteratorFactory wrapParallelIteratorFactory () {
+        ParallelIteratorFactory innerFactory;
+        QueryServices services = context.getConnection().getQueryServices();
+        if (groupBy.isEmpty() || groupBy.isOrderPreserving()) {
+            innerFactory = new SpoolingResultIterator.SpoolingResultIteratorFactory(services);
+        } else {
+            innerFactory = new OrderingResultIteratorFactory(services);
+        }
+        if (parallelIteratorFactory == null) {
+            return innerFactory;
+        }
+        // wrap any existing parallelIteratorFactory
+        return new WrappingResultIteratorFactory(innerFactory, parallelIteratorFactory);
+    }
+    
+    @Override
+    protected ResultIterator newIterator() throws SQLException {
+        if (groupBy.isEmpty()) {
+            UngroupedAggregateRegionObserver.serializeIntoScan(context.getScan());
+        }
+        ParallelIterators parallelIterators = new ParallelIterators(context, tableRef, statement, projection, groupBy, null, wrapParallelIteratorFactory());
+        splits = parallelIterators.getSplits();
+
+        AggregatingResultIterator aggResultIterator;
+        // No need to merge sort for ungrouped aggregation
+        if (groupBy.isEmpty()) {
+            aggResultIterator = new UngroupedAggregatingResultIterator(new ConcatResultIterator(parallelIterators), aggregators);
+        } else {
+            aggResultIterator = new GroupedAggregatingResultIterator(new MergeSortRowKeyResultIterator(parallelIterators), aggregators);
+        }
+
+        if (having != null) {
+            aggResultIterator = new FilterAggregatingResultIterator(aggResultIterator, having);
+        }
+        
+        if (statement.isDistinct() && statement.isAggregate()) { // Dedup on client if select distinct and aggregation
+            aggResultIterator = new DistinctAggregatingResultIterator(aggResultIterator, getProjector());
+        }
+
+        ResultIterator resultScanner = aggResultIterator;
+        if (orderBy.getOrderByExpressions().isEmpty()) {
+            if (limit != null) {
+                resultScanner = new LimitingResultIterator(aggResultIterator, limit);
+            }
+        } else {
+            int thresholdBytes = getConnectionQueryServices(context.getConnection().getQueryServices()).getProps().getInt(
+                    QueryServices.SPOOL_THRESHOLD_BYTES_ATTRIB, QueryServicesOptions.DEFAULT_SPOOL_THRESHOLD_BYTES);
+            resultScanner = new OrderedAggregatingResultIterator(aggResultIterator, orderBy.getOrderByExpressions(), thresholdBytes, limit);
+        }
+        
+        if (context.getSequenceManager().getSequenceCount() > 0) {
+            resultScanner = new SequenceResultIterator(resultScanner, context.getSequenceManager());
+        }
+        return resultScanner;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/execute/BasicQueryPlan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/BasicQueryPlan.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/BasicQueryPlan.java
new file mode 100644
index 0000000..2656aaf
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/BasicQueryPlan.java
@@ -0,0 +1,197 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.execute;
+
+import java.sql.ParameterMetaData;
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import com.google.common.collect.Lists;
+import org.apache.phoenix.compile.ExplainPlan;
+import org.apache.phoenix.compile.GroupByCompiler.GroupBy;
+import org.apache.phoenix.compile.OrderByCompiler.OrderBy;
+import org.apache.phoenix.compile.QueryPlan;
+import org.apache.phoenix.compile.RowProjector;
+import org.apache.phoenix.compile.ScanRanges;
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.iterate.DelegateResultIterator;
+import org.apache.phoenix.iterate.ParallelIterators.ParallelIteratorFactory;
+import org.apache.phoenix.iterate.ResultIterator;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.parse.FilterableStatement;
+import org.apache.phoenix.query.ConnectionQueryServices;
+import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.util.SQLCloseable;
+import org.apache.phoenix.util.SQLCloseables;
+import org.apache.phoenix.util.ScanUtil;
+
+
+
+/**
+ *
+ * Query plan that has no child plans
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public abstract class BasicQueryPlan implements QueryPlan {
+    protected static final long DEFAULT_ESTIMATED_SIZE = 10 * 1024; // 10 K
+    
+    protected final TableRef tableRef;
+    protected final StatementContext context;
+    protected final FilterableStatement statement;
+    protected final RowProjector projection;
+    protected final ParameterMetaData paramMetaData;
+    protected final Integer limit;
+    protected final OrderBy orderBy;
+    protected final GroupBy groupBy;
+    protected final ParallelIteratorFactory parallelIteratorFactory;
+
+    protected BasicQueryPlan(
+            StatementContext context, FilterableStatement statement, TableRef table,
+            RowProjector projection, ParameterMetaData paramMetaData, Integer limit, OrderBy orderBy,
+            GroupBy groupBy, ParallelIteratorFactory parallelIteratorFactory) {
+        this.context = context;
+        this.statement = statement;
+        this.tableRef = table;
+        this.projection = projection;
+        this.paramMetaData = paramMetaData;
+        this.limit = limit;
+        this.orderBy = orderBy;
+        this.groupBy = groupBy;
+        this.parallelIteratorFactory = parallelIteratorFactory;
+    }
+
+    @Override
+    public GroupBy getGroupBy() {
+        return groupBy;
+    }
+
+    
+    @Override
+    public OrderBy getOrderBy() {
+        return orderBy;
+    }
+
+    @Override
+    public TableRef getTableRef() {
+        return tableRef;
+    }
+
+    @Override
+    public Integer getLimit() {
+        return limit;
+    }
+
+    @Override
+    public RowProjector getProjector() {
+        return projection;
+    }
+
+    protected ConnectionQueryServices getConnectionQueryServices(ConnectionQueryServices services) {
+        // Get child services associated with tenantId of query.
+        ConnectionQueryServices childServices = context.getConnection().getTenantId() == null ? 
+                services : 
+                services.getChildQueryServices(new ImmutableBytesWritable(context.getConnection().getTenantId().getBytes()));
+        return childServices;
+    }
+
+//    /**
+//     * Sets up an id used to do round robin queue processing on the server
+//     * @param scan
+//     */
+//    private void setProducer(Scan scan) {
+//        byte[] producer = Bytes.toBytes(UUID.randomUUID().toString());
+//        scan.setAttribute(HBaseServer.CALL_QUEUE_PRODUCER_ATTRIB_NAME, producer);
+//    }
+    
+    @Override
+    public final ResultIterator iterator() throws SQLException {
+        return iterator(Collections.<SQLCloseable>emptyList());
+    }
+
+    public final ResultIterator iterator(final List<SQLCloseable> dependencies) throws SQLException {
+        if (context.getScanRanges() == ScanRanges.NOTHING) {
+            return ResultIterator.EMPTY_ITERATOR;
+        }
+        
+        Scan scan = context.getScan();
+        // Set producer on scan so HBase server does round robin processing
+        //setProducer(scan);
+        // Set the time range on the scan so we don't get back rows newer than when the statement was compiled
+        // The time stamp comes from the server at compile time when the meta data
+        // is resolved.
+        // TODO: include time range in explain plan?
+        PhoenixConnection connection = context.getConnection();
+        Long scn = connection.getSCN();
+        ScanUtil.setTimeRange(scan, scn == null ? context.getCurrentTime() : scn);
+        ScanUtil.setTenantId(scan, connection.getTenantId() == null ? null : connection.getTenantId().getBytes());
+        ResultIterator iterator = newIterator();
+        return dependencies.isEmpty() ? 
+                iterator : new DelegateResultIterator(iterator) {
+            @Override
+            public void close() throws SQLException {
+                try {
+                    super.close();
+                } finally {
+                    SQLCloseables.closeAll(dependencies);
+                }
+            }
+        };
+    }
+
+    abstract protected ResultIterator newIterator() throws SQLException;
+    
+    @Override
+    public long getEstimatedSize() {
+        return DEFAULT_ESTIMATED_SIZE;
+    }
+
+    @Override
+    public ParameterMetaData getParameterMetaData() {
+        return paramMetaData;
+    }
+
+    @Override
+    public FilterableStatement getStatement() {
+        return statement;
+    }
+
+    @Override
+    public StatementContext getContext() {
+        return context;
+    }
+
+    @Override
+    public ExplainPlan getExplainPlan() throws SQLException {
+        if (context.getScanRanges() == ScanRanges.NOTHING) {
+            return new ExplainPlan(Collections.singletonList("DEGENERATE SCAN OVER " + tableRef.getTable().getName().getString()));
+        }
+        
+        ResultIterator iterator = iterator();
+        List<String> planSteps = Lists.newArrayListWithExpectedSize(5);
+        iterator.explain(planSteps);
+        return new ExplainPlan(planSteps);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/execute/CommitException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/CommitException.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/CommitException.java
new file mode 100644
index 0000000..4ca47ec
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/CommitException.java
@@ -0,0 +1,43 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.execute;
+
+import java.sql.SQLException;
+
+public class CommitException extends SQLException {
+    private static final long serialVersionUID = 1L;
+    private final MutationState uncommittedState;
+    private final MutationState committedState;
+
+    public CommitException(Exception e, MutationState uncommittedState, MutationState committedState) {
+        super(e);
+        this.uncommittedState = uncommittedState;
+        this.committedState = committedState;
+    }
+
+    public MutationState getUncommittedState() {
+        return uncommittedState;
+    }
+
+    public MutationState getCommittedState() {
+        return committedState;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/execute/DegenerateQueryPlan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/DegenerateQueryPlan.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/DegenerateQueryPlan.java
new file mode 100644
index 0000000..ba38fae
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/DegenerateQueryPlan.java
@@ -0,0 +1,52 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.execute;
+
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.phoenix.compile.GroupByCompiler.GroupBy;
+import org.apache.phoenix.compile.OrderByCompiler.OrderBy;
+import org.apache.phoenix.compile.*;
+import org.apache.phoenix.iterate.ResultIterator;
+import org.apache.phoenix.jdbc.PhoenixParameterMetaData;
+import org.apache.phoenix.parse.FilterableStatement;
+import org.apache.phoenix.query.*;
+import org.apache.phoenix.schema.TableRef;
+
+public class DegenerateQueryPlan extends BasicQueryPlan {
+
+    public DegenerateQueryPlan(StatementContext context, FilterableStatement statement, TableRef table) {
+        super(context, statement, table, RowProjector.EMPTY_PROJECTOR, PhoenixParameterMetaData.EMPTY_PARAMETER_META_DATA, null, OrderBy.EMPTY_ORDER_BY, GroupBy.EMPTY_GROUP_BY, null);
+        context.setScanRanges(ScanRanges.NOTHING);
+    }
+
+    @Override
+    public List<KeyRange> getSplits() {
+        return Collections.emptyList();
+    }
+
+    @Override
+    protected ResultIterator newIterator() throws SQLException {
+        return null;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/execute/HashJoinPlan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/HashJoinPlan.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/HashJoinPlan.java
new file mode 100644
index 0000000..73b456a
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/HashJoinPlan.java
@@ -0,0 +1,186 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.execute;
+
+import java.sql.ParameterMetaData;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import org.apache.hadoop.hbase.client.Scan;
+
+import com.google.common.collect.Lists;
+import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.cache.ServerCacheClient.ServerCache;
+import org.apache.phoenix.compile.ExplainPlan;
+import org.apache.phoenix.compile.QueryPlan;
+import org.apache.phoenix.compile.RowProjector;
+import org.apache.phoenix.compile.ScanRanges;
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.compile.GroupByCompiler.GroupBy;
+import org.apache.phoenix.compile.OrderByCompiler.OrderBy;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.iterate.ResultIterator;
+import org.apache.phoenix.job.JobManager.JobCallable;
+import org.apache.phoenix.join.HashCacheClient;
+import org.apache.phoenix.join.HashJoinInfo;
+import org.apache.phoenix.parse.FilterableStatement;
+import org.apache.phoenix.query.ConnectionQueryServices;
+import org.apache.phoenix.query.KeyRange;
+import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.util.SQLCloseable;
+
+public class HashJoinPlan implements QueryPlan {
+    
+    private BasicQueryPlan plan;
+    private HashJoinInfo joinInfo;
+    private List<Expression>[] hashExpressions;
+    private QueryPlan[] hashPlans;
+    
+    public HashJoinPlan(BasicQueryPlan plan, HashJoinInfo joinInfo,
+            List<Expression>[] hashExpressions, QueryPlan[] hashPlans) {
+        this.plan = plan;
+        this.joinInfo = joinInfo;
+        this.hashExpressions = hashExpressions;
+        this.hashPlans = hashPlans;
+    }
+
+    @Override
+    public Integer getLimit() {
+        return plan.getLimit();
+    }
+
+    @Override
+    public OrderBy getOrderBy() {
+        return plan.getOrderBy();
+    }
+
+    @Override
+    public RowProjector getProjector() {
+        return plan.getProjector();
+    }
+
+    @Override
+    public ResultIterator iterator() throws SQLException {
+        ImmutableBytesPtr[] joinIds = joinInfo.getJoinIds();
+        assert (joinIds.length == hashExpressions.length && joinIds.length == hashPlans.length);
+        
+        final HashCacheClient hashClient = new HashCacheClient(plan.getContext().getConnection());
+        Scan scan = plan.getContext().getScan();
+        final ScanRanges ranges = plan.getContext().getScanRanges();
+        
+        int count = joinIds.length;
+        ConnectionQueryServices services = getContext().getConnection().getQueryServices();
+        ExecutorService executor = services.getExecutor();
+        List<Future<ServerCache>> futures = new ArrayList<Future<ServerCache>>(count);
+        List<SQLCloseable> dependencies = new ArrayList<SQLCloseable>(count);
+        for (int i = 0; i < count; i++) {
+            final int index = i;
+            futures.add(executor.submit(new JobCallable<ServerCache>() {
+
+                @Override
+                public ServerCache call() throws Exception {
+                    QueryPlan hashPlan = hashPlans[index];
+                    return hashClient.addHashCache(ranges, hashPlan.iterator(), 
+                            hashPlan.getEstimatedSize(), hashExpressions[index], plan.getTableRef());
+                }
+
+                @Override
+                public Object getJobId() {
+                    return HashJoinPlan.this;
+                }
+            }));
+        }
+        for (int i = 0; i < count; i++) {
+            try {
+                ServerCache cache = futures.get(i).get();
+                joinIds[i].set(cache.getId());
+                dependencies.add(cache);
+            } catch (InterruptedException e) {
+                throw new SQLException("Hash join execution interrupted.", e);
+            } catch (ExecutionException e) {
+                throw new SQLException("Encountered exception in hash plan execution.", 
+                        e.getCause());
+            }
+        }
+        HashJoinInfo.serializeHashJoinIntoScan(scan, joinInfo);
+        
+        return plan.iterator(dependencies);
+    }
+    
+    @Override
+    public long getEstimatedSize() {
+        return plan.getEstimatedSize();
+    }
+
+    @Override
+    public List<KeyRange> getSplits() {
+        return plan.getSplits();
+    }
+
+    @Override
+    public ExplainPlan getExplainPlan() throws SQLException {
+        List<String> mainQuerySteps = plan.getExplainPlan().getPlanSteps();
+        List<String> planSteps = Lists.newArrayList(mainQuerySteps);
+        int count = hashPlans.length;
+        planSteps.add("    PARALLEL EQUI-JOIN " + count + " HASH TABLES:");
+        for (int i = 0; i < count; i++) {
+            boolean skipMerge = joinInfo.getSchemas()[i].getFieldCount() == 0;
+        	planSteps.add("    BUILD HASH TABLE " + i + (skipMerge ? " (SKIP MERGE)" : ""));
+        	List<String> steps = hashPlans[i].getExplainPlan().getPlanSteps();
+        	for (String step : steps) {
+        		planSteps.add("        " + step);
+        	}
+        }
+        if (joinInfo.getPostJoinFilterExpression() != null) {
+        	planSteps.add("    AFTER-JOIN SERVER FILTER BY " + joinInfo.getPostJoinFilterExpression().toString());
+        }
+        
+        return new ExplainPlan(planSteps);
+    }
+
+    @Override
+    public ParameterMetaData getParameterMetaData() {
+        return plan.getParameterMetaData();
+    }
+
+    @Override
+    public StatementContext getContext() {
+        return plan.getContext();
+    }
+
+    @Override
+    public GroupBy getGroupBy() {
+        return plan.getGroupBy();
+    }
+
+    @Override
+    public TableRef getTableRef() {
+        return plan.getTableRef();
+    }
+
+    @Override
+    public FilterableStatement getStatement() {
+        return plan.getStatement();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..298c7c1
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
@@ -0,0 +1,444 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.execute;
+
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.cache.ServerCacheClient;
+import org.apache.phoenix.cache.ServerCacheClient.ServerCache;
+import org.apache.phoenix.coprocessor.MetaDataProtocol.MetaDataMutationResult;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.index.IndexMaintainer;
+import org.apache.phoenix.index.IndexMetaDataCacheClient;
+import org.apache.phoenix.index.PhoenixIndexCodec;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.IllegalDataException;
+import org.apache.phoenix.schema.MetaDataClient;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PRow;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.IndexUtil;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.SQLCloseable;
+import org.apache.phoenix.util.ServerUtil;
+
+/**
+ * 
+ * Tracks the uncommitted state
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class MutationState implements SQLCloseable {
+    private static final Logger logger = LoggerFactory.getLogger(MutationState.class);
+
+    private PhoenixConnection connection;
+    private final long maxSize;
+    private final ImmutableBytesPtr tempPtr = new ImmutableBytesPtr();
+    private final Map<TableRef, Map<ImmutableBytesPtr,Map<PColumn,byte[]>>> mutations = Maps.newHashMapWithExpectedSize(3); // TODO: Sizing?
+    private final long sizeOffset;
+    private int numRows = 0;
+
+    public MutationState(int maxSize, PhoenixConnection connection) {
+        this(maxSize,connection,0);
+    }
+    
+    public MutationState(int maxSize, PhoenixConnection connection, long sizeOffset) {
+        this.maxSize = maxSize;
+        this.connection = connection;
+        this.sizeOffset = sizeOffset;
+    }
+    
+    public MutationState(TableRef table, Map<ImmutableBytesPtr,Map<PColumn,byte[]>> mutations, long sizeOffset, long maxSize, PhoenixConnection connection) {
+        this.maxSize = maxSize;
+        this.connection = connection;
+        this.mutations.put(table, mutations);
+        this.sizeOffset = sizeOffset;
+        this.numRows = mutations.size();
+        throwIfTooBig();
+    }
+    
+    private MutationState(List<Map.Entry<TableRef, Map<ImmutableBytesPtr,Map<PColumn,byte[]>>>> entries, long sizeOffset, long maxSize, PhoenixConnection connection) {
+        this.maxSize = maxSize;
+        this.connection = connection;
+        this.sizeOffset = sizeOffset;
+        for (Map.Entry<TableRef, Map<ImmutableBytesPtr,Map<PColumn,byte[]>>> entry : entries) {
+            numRows += entry.getValue().size();
+            this.mutations.put(entry.getKey(), entry.getValue());
+        }
+        throwIfTooBig();
+    }
+    
+    private void throwIfTooBig() {
+        if (numRows > maxSize) {
+            // TODO: throw SQLException ?
+            throw new IllegalArgumentException("MutationState size of " + numRows + " is bigger than max allowed size of " + maxSize);
+        }
+    }
+    
+    public long getUpdateCount() {
+        return sizeOffset + numRows;
+    }
+    /**
+     * Combine a newer mutation with this one, where in the event of overlaps,
+     * the newer one will take precedence.
+     * @param newMutation the newer mutation
+     */
+    public void join(MutationState newMutation) {
+        if (this == newMutation) { // Doesn't make sense
+            return;
+        }
+        // Merge newMutation with this one, keeping state from newMutation for any overlaps
+        for (Map.Entry<TableRef, Map<ImmutableBytesPtr,Map<PColumn,byte[]>>> entry : newMutation.mutations.entrySet()) {
+            // Replace existing entries for the table with new entries
+            Map<ImmutableBytesPtr,Map<PColumn,byte[]>> existingRows = this.mutations.put(entry.getKey(), entry.getValue());
+            if (existingRows != null) { // Rows for that table already exist
+                // Loop through new rows and replace existing with new
+                for (Map.Entry<ImmutableBytesPtr,Map<PColumn,byte[]>> rowEntry : entry.getValue().entrySet()) {
+                    // Replace existing row with new row
+                    Map<PColumn,byte[]> existingValues = existingRows.put(rowEntry.getKey(), rowEntry.getValue());
+                    if (existingValues != null) {
+                        if (existingValues != PRow.DELETE_MARKER) {
+                            Map<PColumn,byte[]> newRow = rowEntry.getValue();
+                            // if new row is PRow.DELETE_MARKER, it means delete, and we don't need to merge it with existing row. 
+                            if (newRow != PRow.DELETE_MARKER) {
+                                // Replace existing column values with new column values
+                                for (Map.Entry<PColumn,byte[]> valueEntry : newRow.entrySet()) {
+                                    existingValues.put(valueEntry.getKey(), valueEntry.getValue());
+                                }
+                                // Now that the existing row has been merged with the new row, replace it back
+                                // again (since it was replaced with the new one above).
+                                existingRows.put(rowEntry.getKey(), existingValues);
+                            }
+                        }
+                    } else {
+                        numRows++;
+                    }
+                }
+                // Put the existing one back now that it's merged
+                this.mutations.put(entry.getKey(), existingRows);
+            } else {
+                numRows += entry.getValue().size();
+            }
+        }
+        throwIfTooBig();
+    }
+    
+    private Iterator<Pair<byte[],List<Mutation>>> addRowMutations(final TableRef tableRef, final Map<ImmutableBytesPtr, Map<PColumn, byte[]>> values, long timestamp, boolean includeMutableIndexes) {
+        final List<Mutation> mutations = Lists.newArrayListWithExpectedSize(values.size());
+        Iterator<Map.Entry<ImmutableBytesPtr,Map<PColumn,byte[]>>> iterator = values.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<ImmutableBytesPtr,Map<PColumn,byte[]>> rowEntry = iterator.next();
+            ImmutableBytesPtr key = rowEntry.getKey();
+            PRow row = tableRef.getTable().newRow(connection.getKeyValueBuilder(), timestamp, key);
+            if (rowEntry.getValue() == PRow.DELETE_MARKER) { // means delete
+                row.delete();
+            } else {
+                for (Map.Entry<PColumn,byte[]> valueEntry : rowEntry.getValue().entrySet()) {
+                    row.setValue(valueEntry.getKey(), valueEntry.getValue());
+                }
+            }
+            mutations.addAll(row.toRowMutations());
+        }
+        final Iterator<PTable> indexes = // Only maintain tables with immutable rows through this client-side mechanism
+                (tableRef.getTable().isImmutableRows() || includeMutableIndexes) ? 
+                        IndexMaintainer.nonDisabledIndexIterator(tableRef.getTable().getIndexes().iterator()) : 
+                        Iterators.<PTable>emptyIterator();
+        return new Iterator<Pair<byte[],List<Mutation>>>() {
+            boolean isFirst = true;
+
+            @Override
+            public boolean hasNext() {
+                return isFirst || indexes.hasNext();
+            }
+
+            @Override
+            public Pair<byte[], List<Mutation>> next() {
+                if (isFirst) {
+                    isFirst = false;
+                    return new Pair<byte[],List<Mutation>>(tableRef.getTable().getPhysicalName().getBytes(),mutations);
+                }
+                PTable index = indexes.next();
+                List<Mutation> indexMutations;
+                try {
+                    indexMutations =
+                            IndexUtil.generateIndexData(tableRef.getTable(), index, mutations,
+                                tempPtr, connection.getKeyValueBuilder());
+                } catch (SQLException e) {
+                    throw new IllegalDataException(e);
+                }
+                return new Pair<byte[],List<Mutation>>(index.getPhysicalName().getBytes(),indexMutations);
+            }
+
+            @Override
+            public void remove() {
+                throw new UnsupportedOperationException();
+            }
+            
+        };
+    }
+    
+    /**
+     * Get the unsorted list of HBase mutations for the tables with uncommitted data.
+     * @return list of HBase mutations for uncommitted data.
+     */
+    public Iterator<Pair<byte[],List<Mutation>>> toMutations() {
+        return toMutations(false);
+    }
+    
+    public Iterator<Pair<byte[],List<Mutation>>> toMutations(final boolean includeMutableIndexes) {
+        final Iterator<Map.Entry<TableRef, Map<ImmutableBytesPtr,Map<PColumn,byte[]>>>> iterator = this.mutations.entrySet().iterator();
+        if (!iterator.hasNext()) {
+            return Iterators.emptyIterator();
+        }
+        Long scn = connection.getSCN();
+        final long timestamp = scn == null ? HConstants.LATEST_TIMESTAMP : scn;
+        return new Iterator<Pair<byte[],List<Mutation>>>() {
+            private Map.Entry<TableRef, Map<ImmutableBytesPtr,Map<PColumn,byte[]>>> current = iterator.next();
+            private Iterator<Pair<byte[],List<Mutation>>> innerIterator = init();
+                    
+            private Iterator<Pair<byte[],List<Mutation>>> init() {
+                return addRowMutations(current.getKey(), current.getValue(), timestamp, includeMutableIndexes);
+            }
+            
+            @Override
+            public boolean hasNext() {
+                return innerIterator.hasNext() || iterator.hasNext();
+            }
+
+            @Override
+            public Pair<byte[], List<Mutation>> next() {
+                if (!innerIterator.hasNext()) {
+                    current = iterator.next();
+                }
+                return innerIterator.next();
+            }
+
+            @Override
+            public void remove() {
+                throw new UnsupportedOperationException();
+            }
+            
+        };
+    }
+        
+    /**
+     * Validates that the meta data is still valid based on the current server time
+     * and returns the server time to use for the upsert for each table.
+     * @param connection
+     * @return the server time to use for the upsert
+     * @throws SQLException if the table or any columns no longer exist
+     */
+    private long[] validate() throws SQLException {
+        int i = 0;
+        Long scn = connection.getSCN();
+        MetaDataClient client = new MetaDataClient(connection);
+        long[] timeStamps = new long[this.mutations.size()];
+        for (Map.Entry<TableRef, Map<ImmutableBytesPtr,Map<PColumn,byte[]>>> entry : mutations.entrySet()) {
+            TableRef tableRef = entry.getKey();
+            long serverTimeStamp = tableRef.getTimeStamp();
+            PTable table = tableRef.getTable();
+            if (!connection.getAutoCommit()) {
+                MetaDataMutationResult result = client.updateCache(table.getSchemaName().getString(), table.getTableName().getString());
+                long timestamp = result.getMutationTime();
+                if (timestamp != QueryConstants.UNSET_TIMESTAMP) {
+                    serverTimeStamp = timestamp;
+                    if (result.wasUpdated()) {
+                        // TODO: use bitset?
+                        PColumn[] columns = new PColumn[table.getColumns().size()];
+                        for (Map.Entry<ImmutableBytesPtr,Map<PColumn,byte[]>> rowEntry : entry.getValue().entrySet()) {
+                            Map<PColumn,byte[]> valueEntry = rowEntry.getValue();
+                            if (valueEntry != PRow.DELETE_MARKER) {
+                                for (PColumn column : valueEntry.keySet()) {
+                                    columns[column.getPosition()] = column;
+                                }
+                            }
+                        }
+                        table = connection.getPMetaData().getTable(tableRef.getTable().getName().getString());
+                        for (PColumn column : columns) {
+                            if (column != null) {
+                                table.getColumnFamily(column.getFamilyName().getString()).getColumn(column.getName().getString());
+                            }
+                        }
+                    }
+                }
+            }
+            timeStamps[i++] = scn == null ? serverTimeStamp : scn;
+        }
+        return timeStamps;
+    }
+    
+    private static void logMutationSize(HTableInterface htable, List<Mutation> mutations) {
+        long byteSize = 0;
+        int keyValueCount = 0;
+        for (Mutation mutation : mutations) {
+            if (mutation.getFamilyMap() != null) { // Not a Delete of the row
+                for (Entry<byte[], List<KeyValue>> entry : mutation.getFamilyMap().entrySet()) {
+                    if (entry.getValue() != null) {
+                        for (KeyValue kv : entry.getValue()) {
+                            byteSize += kv.getBuffer().length;
+                            keyValueCount++;
+                        }
+                    }
+                }
+            }
+        }
+        logger.debug("Sending " + mutations.size() + " mutations for " + Bytes.toString(htable.getTableName()) + " with " + keyValueCount + " key values of total size " + byteSize + " bytes");
+    }
+    
+    public void commit() throws SQLException {
+        int i = 0;
+        byte[] tenantId = connection.getTenantId() == null ? null : connection.getTenantId().getBytes();
+        long[] serverTimeStamps = validate();
+        Iterator<Map.Entry<TableRef, Map<ImmutableBytesPtr,Map<PColumn,byte[]>>>> iterator = this.mutations.entrySet().iterator();
+        List<Map.Entry<TableRef, Map<ImmutableBytesPtr,Map<PColumn,byte[]>>>> committedList = Lists.newArrayListWithCapacity(this.mutations.size());
+        while (iterator.hasNext()) {
+            Map.Entry<TableRef, Map<ImmutableBytesPtr,Map<PColumn,byte[]>>> entry = iterator.next();
+            Map<ImmutableBytesPtr,Map<PColumn,byte[]>> valuesMap = entry.getValue();
+            TableRef tableRef = entry.getKey();
+            PTable table = tableRef.getTable();
+            table.getIndexMaintainers(tempPtr);
+            boolean hasIndexMaintainers = tempPtr.getLength() > 0;
+            boolean isDataTable = true;
+            long serverTimestamp = serverTimeStamps[i++];
+            Iterator<Pair<byte[],List<Mutation>>> mutationsIterator = addRowMutations(tableRef, valuesMap, serverTimestamp, false);
+            while (mutationsIterator.hasNext()) {
+                Pair<byte[],List<Mutation>> pair = mutationsIterator.next();
+                byte[] htableName = pair.getFirst();
+                List<Mutation> mutations = pair.getSecond();
+                
+                int retryCount = 0;
+                boolean shouldRetry = false;
+                do {
+                    ServerCache cache = null;
+                    if (hasIndexMaintainers && isDataTable) {
+                        byte[] attribValue = null;
+                        byte[] uuidValue;
+                        if (IndexMetaDataCacheClient.useIndexMetadataCache(connection, mutations, tempPtr.getLength())) {
+                            IndexMetaDataCacheClient client = new IndexMetaDataCacheClient(connection, tableRef);
+                            cache = client.addIndexMetadataCache(mutations, tempPtr);
+                            uuidValue = cache.getId();
+                            // If we haven't retried yet, retry for this case only, as it's possible that
+                            // a split will occur after we send the index metadata cache to all known
+                            // region servers.
+                            shouldRetry = true;
+                        } else {
+                            attribValue = ByteUtil.copyKeyBytesIfNecessary(tempPtr);
+                            uuidValue = ServerCacheClient.generateId();
+                        }
+                        // Either set the UUID to be able to access the index metadata from the cache
+                        // or set the index metadata directly on the Mutation
+                        for (Mutation mutation : mutations) {
+                            if (tenantId != null) {
+                                mutation.setAttribute(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId);
+                            }
+                            mutation.setAttribute(PhoenixIndexCodec.INDEX_UUID, uuidValue);
+                            if (attribValue != null) {
+                                mutation.setAttribute(PhoenixIndexCodec.INDEX_MD, attribValue);
+                            }
+                        }
+                    }
+                    
+                    SQLException sqlE = null;
+                    HTableInterface hTable = connection.getQueryServices().getTable(htableName);
+                    try {
+                        if (logger.isDebugEnabled()) logMutationSize(hTable, mutations);
+                        long startTime = System.currentTimeMillis();
+                        hTable.batch(mutations);
+                        shouldRetry = false;
+                        if (logger.isDebugEnabled()) logger.debug("Total time for batch call of  " + mutations.size() + " mutations into " + table.getName().getString() + ": " + (System.currentTimeMillis() - startTime) + " ms");
+                        committedList.add(entry);
+                    } catch (Exception e) {
+                        SQLException inferredE = ServerUtil.parseServerExceptionOrNull(e);
+                        if (inferredE != null) {
+                            if (shouldRetry && retryCount == 0 && inferredE.getErrorCode() == SQLExceptionCode.INDEX_METADATA_NOT_FOUND.getErrorCode()) {
+                                // Swallow this exception once, as it's possible that we split after sending the index metadata
+                                // and one of the region servers doesn't have it. This will cause it to have it the next go around.
+                                // If it fails again, we don't retry.
+                                logger.warn("Swallowing exception and retrying after clearing meta cache on connection. " + inferredE);
+                                connection.getQueryServices().clearTableRegionCache(htableName);
+                                continue;
+                            }
+                            e = inferredE;
+                        }
+                        // Throw to client with both what was committed so far and what is left to be committed.
+                        // That way, client can either undo what was done or try again with what was not done.
+                        sqlE = new CommitException(e, this, new MutationState(committedList, this.sizeOffset, this.maxSize, this.connection));
+                    } finally {
+                        try {
+                            hTable.close();
+                        } catch (IOException e) {
+                            if (sqlE != null) {
+                                sqlE.setNextException(ServerUtil.parseServerException(e));
+                            } else {
+                                sqlE = ServerUtil.parseServerException(e);
+                            }
+                        } finally {
+                            try {
+                                if (cache != null) {
+                                    cache.close();
+                                }
+                            } finally {
+                                if (sqlE != null) {
+                                    throw sqlE;
+                                }
+                            }
+                        }
+                    }
+                } while (shouldRetry && retryCount++ < 1);
+                isDataTable = false;
+            }
+            numRows -= entry.getValue().size();
+            iterator.remove(); // Remove batches as we process them
+        }
+        assert(numRows==0);
+        assert(this.mutations.isEmpty());
+    }
+    
+    public void rollback(PhoenixConnection connection) throws SQLException {
+        this.mutations.clear();
+        numRows = 0;
+    }
+    
+    @Override
+    public void close() throws SQLException {
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/execute/ScanPlan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/ScanPlan.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/ScanPlan.java
new file mode 100644
index 0000000..01199c9
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/ScanPlan.java
@@ -0,0 +1,116 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.execute;
+
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.phoenix.compile.GroupByCompiler.GroupBy;
+import org.apache.phoenix.compile.OrderByCompiler.OrderBy;
+import org.apache.phoenix.compile.RowProjector;
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.coprocessor.ScanRegionObserver;
+import org.apache.phoenix.iterate.ConcatResultIterator;
+import org.apache.phoenix.iterate.LimitingResultIterator;
+import org.apache.phoenix.iterate.MergeSortRowKeyResultIterator;
+import org.apache.phoenix.iterate.MergeSortTopNResultIterator;
+import org.apache.phoenix.iterate.ParallelIterators;
+import org.apache.phoenix.iterate.ParallelIterators.ParallelIteratorFactory;
+import org.apache.phoenix.iterate.ResultIterator;
+import org.apache.phoenix.iterate.SequenceResultIterator;
+import org.apache.phoenix.iterate.SpoolingResultIterator;
+import org.apache.phoenix.parse.FilterableStatement;
+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.PTable;
+import org.apache.phoenix.schema.SaltingUtil;
+import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.util.ScanUtil;
+
+
+
+/**
+ * 
+ * Query plan for a basic table scan
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class ScanPlan extends BasicQueryPlan {
+    private List<KeyRange> splits;
+    
+    public ScanPlan(StatementContext context, FilterableStatement statement, TableRef table, RowProjector projector, Integer limit, OrderBy orderBy, ParallelIteratorFactory parallelIteratorFactory) {
+        super(context, statement, table, projector, context.getBindManager().getParameterMetaData(), limit, orderBy, null, parallelIteratorFactory == null ? new SpoolingResultIterator.SpoolingResultIteratorFactory(context.getConnection().getQueryServices()) : parallelIteratorFactory);
+        if (!orderBy.getOrderByExpressions().isEmpty()) { // TopN
+            int thresholdBytes = context.getConnection().getQueryServices().getProps().getInt(
+                    QueryServices.SPOOL_THRESHOLD_BYTES_ATTRIB, QueryServicesOptions.DEFAULT_SPOOL_THRESHOLD_BYTES);
+            ScanRegionObserver.serializeIntoScan(context.getScan(), thresholdBytes, limit == null ? -1 : limit, orderBy.getOrderByExpressions(), projector.getEstimatedRowByteSize());
+        }
+    }
+    
+    @Override
+    public List<KeyRange> getSplits() {
+        return splits;
+    }
+    
+    @Override
+    protected ResultIterator newIterator() throws SQLException {
+        // Set any scan attributes before creating the scanner, as it will be too late afterwards
+        context.getScan().setAttribute(ScanRegionObserver.NON_AGGREGATE_QUERY, QueryConstants.TRUE);
+        if (OrderBy.REV_ROW_KEY_ORDER_BY.equals(orderBy)) {
+            ScanUtil.setReversed(context.getScan());
+        }
+        ResultIterator scanner;
+        TableRef tableRef = this.getTableRef();
+        PTable table = tableRef.getTable();
+        boolean isSalted = table.getBucketNum() != null;
+        /* If no limit or topN, use parallel iterator so that we get results faster. Otherwise, if
+         * limit is provided, run query serially.
+         */
+        boolean isOrdered = !orderBy.getOrderByExpressions().isEmpty();
+        ParallelIterators iterators = new ParallelIterators(context, tableRef, statement, projection, GroupBy.EMPTY_GROUP_BY, isOrdered ? null : limit, parallelIteratorFactory);
+        splits = iterators.getSplits();
+        if (isOrdered) {
+            scanner = new MergeSortTopNResultIterator(iterators, limit, orderBy.getOrderByExpressions());
+        } else {
+            if (isSalted && 
+                    (getConnectionQueryServices(context.getConnection().getQueryServices()).getProps().getBoolean(
+                            QueryServices.ROW_KEY_ORDER_SALTED_TABLE_ATTRIB, 
+                            QueryServicesOptions.DEFAULT_ROW_KEY_ORDER_SALTED_TABLE) ||
+                     orderBy == OrderBy.FWD_ROW_KEY_ORDER_BY ||
+                     orderBy == OrderBy.REV_ROW_KEY_ORDER_BY)) { // ORDER BY was optimized out b/c query is in row key order
+                scanner = new MergeSortRowKeyResultIterator(iterators, SaltingUtil.NUM_SALTING_BYTES, orderBy == OrderBy.REV_ROW_KEY_ORDER_BY);
+            } else {
+                scanner = new ConcatResultIterator(iterators);
+            }
+            if (limit != null) {
+                scanner = new LimitingResultIterator(scanner, limit);
+            }
+        }
+
+        if (context.getSequenceManager().getSequenceCount() > 0) {
+            scanner = new SequenceResultIterator(scanner, context.getSequenceManager());
+        }
+        return scanner;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/AddExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/AddExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/AddExpression.java
new file mode 100644
index 0000000..fd906c5
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/AddExpression.java
@@ -0,0 +1,56 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.util.List;
+
+import org.apache.phoenix.expression.visitor.ExpressionVisitor;
+
+
+/**
+ * 
+ * Subtract expression implementation
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public abstract class AddExpression extends BaseAddSubtractExpression {
+    public AddExpression() {
+    }
+
+    public AddExpression(List<Expression> children) {
+        super(children);
+    }
+
+    @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;
+    }
+
+    @Override
+    public String getOperatorString() {
+        return " + ";
+    }
+}


[24/51] [partial] Initial commit of master branch from github

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/iterate/MergeSortTopNResultIterator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/MergeSortTopNResultIterator.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/MergeSortTopNResultIterator.java
new file mode 100644
index 0000000..77d1c62
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/MergeSortTopNResultIterator.java
@@ -0,0 +1,100 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.iterate;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.OrderByExpression;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+/**
+ * 
+ * ResultIterator that does a merge sort on the list of iterators provided,
+ * returning the rows ordered by the OrderByExpression. The input
+ * iterators must be ordered by the OrderByExpression.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class MergeSortTopNResultIterator extends MergeSortResultIterator {
+
+    private final int limit;
+    private int count = 0;
+    private final List<OrderByExpression> orderByColumns;
+    private final ImmutableBytesWritable ptr1 = new ImmutableBytesWritable();
+    private final ImmutableBytesWritable ptr2 = new ImmutableBytesWritable();
+    
+    public MergeSortTopNResultIterator(ResultIterators iterators, Integer limit, List<OrderByExpression> orderByColumns) {
+        super(iterators);
+        this.limit = limit == null ? -1 : limit;
+        this.orderByColumns = orderByColumns;
+    }
+
+    @Override
+    protected int compare(Tuple t1, Tuple t2) {
+        for (int i = 0; i < orderByColumns.size(); i++) {
+            OrderByExpression order = orderByColumns.get(i);
+            Expression orderExpr = order.getExpression();
+            boolean isNull1 = !orderExpr.evaluate(t1, ptr1) || ptr1.getLength() == 0;
+            boolean isNull2 = !orderExpr.evaluate(t2, ptr2) || ptr2.getLength() == 0;
+            if (isNull1 && isNull2) {
+                continue;
+            } else if (isNull1) {
+                return order.isNullsLast() ? 1 : -1;
+            } else if (isNull2) {
+                return order.isNullsLast() ? -1 : 1;
+            }
+            int cmp = ptr1.compareTo(ptr2);
+            if (cmp == 0) {
+                continue;
+            }
+            return order.isAscending() ? cmp : -cmp;
+        }
+        return 0;
+    }
+
+    @Override
+    public Tuple peek() throws SQLException {
+        if (limit >= 0 && count >= limit) {
+            return null;
+        }
+        return super.peek();
+    }
+
+    @Override
+    public Tuple next() throws SQLException {
+        if (limit >= 0 && count++ >= limit) {
+            return null;
+        }
+        return super.next();
+    }
+
+
+    @Override
+    public void explain(List<String> planSteps) {
+        resultIterators.explain(planSteps);
+        planSteps.add("    SERVER TOP " + limit + " ROW" + (limit == 1 ? "" : "S") + " SORTED BY " + orderByColumns.toString());
+        planSteps.add("CLIENT MERGE SORT");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/iterate/OrderedAggregatingResultIterator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/OrderedAggregatingResultIterator.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/OrderedAggregatingResultIterator.java
new file mode 100644
index 0000000..2b7011a
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/OrderedAggregatingResultIterator.java
@@ -0,0 +1,63 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.iterate;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.phoenix.expression.OrderByExpression;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+
+/**
+ * Result scanner that sorts aggregated rows by columns specified in the ORDER BY clause.
+ * <p>
+ * Note that currently the sort is entirely done in memory. 
+ *  
+ * @author syyang
+ * @since 0.1
+ */
+public class OrderedAggregatingResultIterator extends OrderedResultIterator implements AggregatingResultIterator {
+
+    public OrderedAggregatingResultIterator(AggregatingResultIterator delegate,
+                                List<OrderByExpression> orderByExpressions,
+                                int thresholdBytes, Integer limit) throws SQLException {
+        super (delegate, orderByExpressions, thresholdBytes, limit);
+    }
+
+    @Override
+    protected AggregatingResultIterator getDelegate() {
+        return (AggregatingResultIterator)super.getDelegate();
+    }
+    
+    @Override
+    public Tuple next() throws SQLException {
+        Tuple tuple = super.next();
+        if (tuple != null) {
+            aggregate(tuple);
+        }
+        return tuple;
+    }
+    
+    @Override
+    public void aggregate(Tuple result) {
+        getDelegate().aggregate(result);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..288c465
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/OrderedResultIterator.java
@@ -0,0 +1,254 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.iterate;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkPositionIndex;
+
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.Comparator;
+import java.util.List;
+
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import com.google.common.base.Function;
+import com.google.common.collect.Collections2;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Ordering;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.OrderByExpression;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.SizedUtil;
+
+/**
+ * Result scanner that sorts aggregated rows by columns specified in the ORDER BY clause.
+ * <p>
+ * Note that currently the sort is entirely done in memory. 
+ *  
+ * @author syyang, jtaylor
+ * @since 0.1
+ */
+public class OrderedResultIterator implements PeekingResultIterator {
+
+    /** A container that holds pointers to a {@link Result} and its sort keys. */
+    protected static class ResultEntry {
+        protected final ImmutableBytesWritable[] sortKeys;
+        protected final Tuple result;
+
+        ResultEntry(ImmutableBytesWritable[] sortKeys, Tuple result) {
+            this.sortKeys = sortKeys;
+            this.result = result;
+        }
+        
+        ImmutableBytesWritable getSortKey(int index) {
+            checkPositionIndex(index, sortKeys.length);
+            return sortKeys[index];
+        }
+        
+        Tuple getResult() {
+            return result;
+        }
+    }
+    
+    /** A function that returns Nth key for a given {@link ResultEntry}. */
+    private static class NthKey implements Function<ResultEntry, ImmutableBytesWritable> {
+        private final int index;
+
+        NthKey(int index) {
+            this.index = index;
+        }
+        @Override
+        public ImmutableBytesWritable apply(ResultEntry entry) {
+            return entry.getSortKey(index);
+        }
+    }
+
+    /** Returns the expression of a given {@link OrderByExpression}. */
+    private static final Function<OrderByExpression, Expression> TO_EXPRESSION = new Function<OrderByExpression, Expression>() {
+        @Override
+        public Expression apply(OrderByExpression column) {
+            return column.getExpression();
+        }
+    };
+
+    private final int thresholdBytes;
+    private final Integer limit;
+    private final ResultIterator delegate;
+    private final List<OrderByExpression> orderByExpressions;
+    private final long estimatedByteSize;
+    
+    private PeekingResultIterator resultIterator;
+    private long byteSize;
+
+    protected ResultIterator getDelegate() {
+        return delegate;
+    }
+    
+    public OrderedResultIterator(ResultIterator delegate,
+                                 List<OrderByExpression> orderByExpressions,
+                                 int thresholdBytes, Integer limit) {
+        this(delegate, orderByExpressions, thresholdBytes, limit, 0);
+    }
+
+    public OrderedResultIterator(ResultIterator delegate,
+            List<OrderByExpression> orderByExpressions, int thresholdBytes) throws SQLException {
+        this(delegate, orderByExpressions, thresholdBytes, null);
+    }
+
+    public OrderedResultIterator(ResultIterator delegate, List<OrderByExpression> orderByExpressions, 
+            int thresholdBytes, Integer limit, int estimatedRowSize) {
+        checkArgument(!orderByExpressions.isEmpty());
+        this.delegate = delegate;
+        this.orderByExpressions = orderByExpressions;
+        this.thresholdBytes = thresholdBytes;
+        this.limit = limit;
+        long estimatedEntrySize =
+            // ResultEntry
+            SizedUtil.OBJECT_SIZE + 
+            // ImmutableBytesWritable[]
+            SizedUtil.ARRAY_SIZE + orderByExpressions.size() * SizedUtil.IMMUTABLE_BYTES_WRITABLE_SIZE +
+            // Tuple
+            SizedUtil.OBJECT_SIZE + estimatedRowSize;
+
+        // Make sure we don't overflow Long, though this is really unlikely to happen.
+        assert(limit == null || Long.MAX_VALUE / estimatedEntrySize >= limit);
+
+        this.estimatedByteSize = limit == null ? 0 : limit * estimatedEntrySize;
+    }
+
+    public Integer getLimit() {
+        return limit;
+    }
+
+    public long getEstimatedByteSize() {
+        return estimatedByteSize;
+    }
+
+    public long getByteSize() {
+        return byteSize;
+    }
+    /**
+     * Builds a comparator from the list of columns in ORDER BY clause.
+     * @param orderByExpressions the columns in ORDER BY clause.
+     * @return the comparator built from the list of columns in ORDER BY clause.
+     */
+    // ImmutableBytesWritable.Comparator doesn't implement generics
+    @SuppressWarnings("unchecked")
+    private static Comparator<ResultEntry> buildComparator(List<OrderByExpression> orderByExpressions) {
+        Ordering<ResultEntry> ordering = null;
+        int pos = 0;
+        for (OrderByExpression col : orderByExpressions) {
+            Ordering<ImmutableBytesWritable> o = Ordering.from(new ImmutableBytesWritable.Comparator());
+            if(!col.isAscending()) o = o.reverse();
+            o = col.isNullsLast() ? o.nullsLast() : o.nullsFirst();
+            Ordering<ResultEntry> entryOrdering = o.onResultOf(new NthKey(pos++));
+            ordering = ordering == null ? entryOrdering : ordering.compound(entryOrdering);
+        }
+        return ordering;
+    }
+
+    @Override
+    public Tuple next() throws SQLException {
+        return getResultIterator().next();
+    }
+    
+    private PeekingResultIterator getResultIterator() throws SQLException {
+        if (resultIterator != null) {
+            return resultIterator;
+        }
+        
+        final int numSortKeys = orderByExpressions.size();
+        List<Expression> expressions = Lists.newArrayList(Collections2.transform(orderByExpressions, TO_EXPRESSION));
+        final Comparator<ResultEntry> comparator = buildComparator(orderByExpressions);
+        try{
+            final MappedByteBufferSortedQueue queueEntries = new MappedByteBufferSortedQueue(comparator, limit, thresholdBytes);
+            resultIterator = new PeekingResultIterator() {
+                int count = 0;
+                @Override
+                public Tuple next() throws SQLException {
+                    ResultEntry entry = queueEntries.poll();
+                    if (entry == null || (limit != null && ++count > limit)) {
+                        resultIterator = PeekingResultIterator.EMPTY_ITERATOR;
+                        return null;
+                    }
+                    return entry.getResult();
+                }
+                
+                @Override
+                public Tuple peek() throws SQLException {
+                    if (limit != null && count > limit) {
+                        return null;
+                    }
+                    ResultEntry entry =  queueEntries.peek();
+                    if (entry == null) {
+                        return null;
+                    }
+                    return entry.getResult();
+                }
+
+                @Override
+                public void explain(List<String> planSteps) {
+                }
+                
+                @Override
+                public void close() throws SQLException {
+                    queueEntries.close();
+                }
+            };
+            for (Tuple result = delegate.next(); result != null; result = delegate.next()) {
+                int pos = 0;
+                ImmutableBytesWritable[] sortKeys = new ImmutableBytesWritable[numSortKeys];
+                for (Expression expression : expressions) {
+                    final ImmutableBytesWritable sortKey = new ImmutableBytesWritable();
+                    boolean evaluated = expression.evaluate(result, sortKey);
+                    // set the sort key that failed to get evaluated with null
+                    sortKeys[pos++] = evaluated && sortKey.getLength() > 0 ? sortKey : null;
+                }
+                queueEntries.add(new ResultEntry(sortKeys, result));
+            }
+            this.byteSize = queueEntries.getByteSize();
+        } catch (IOException e) {
+            throw new SQLException("", e);
+        } finally {
+            delegate.close();
+        }
+        
+        return resultIterator;
+    }
+
+    @Override
+    public Tuple peek() throws SQLException {
+        return getResultIterator().peek();
+    }
+
+    @Override
+    public void close()  {
+        resultIterator = PeekingResultIterator.EMPTY_ITERATOR;
+    }
+
+
+    @Override
+    public void explain(List<String> planSteps) {
+        delegate.explain(planSteps);
+        planSteps.add("CLIENT" + (limit == null ? "" : " TOP " + limit + " ROW"  + (limit == 1 ? "" : "S"))  + " SORTED BY " + orderByExpressions.toString());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/iterate/ParallelIteratorRegionSplitter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/ParallelIteratorRegionSplitter.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/ParallelIteratorRegionSplitter.java
new file mode 100644
index 0000000..a14b1e8
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/ParallelIteratorRegionSplitter.java
@@ -0,0 +1,36 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.iterate;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.phoenix.query.KeyRange;
+
+
+/**
+ * Interface for strategies determining how to split regions in ParallelIterators.
+ * 
+ * @author zhuang
+ */
+public interface ParallelIteratorRegionSplitter {
+
+    public List<KeyRange> getSplits() throws SQLException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/iterate/ParallelIteratorRegionSplitterFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/ParallelIteratorRegionSplitterFactory.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/ParallelIteratorRegionSplitterFactory.java
new file mode 100644
index 0000000..88efc44
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/ParallelIteratorRegionSplitterFactory.java
@@ -0,0 +1,40 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.iterate;
+
+import java.sql.SQLException;
+
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.parse.HintNode;
+import org.apache.phoenix.schema.TableRef;
+
+
+/**
+ * Factory class for the Region Splitter used by the project.
+ */
+public class ParallelIteratorRegionSplitterFactory {
+
+    public static ParallelIteratorRegionSplitter getSplitter(StatementContext context, TableRef table, HintNode hintNode) throws SQLException {
+        if (context.getScanRanges().useSkipScanFilter()) {
+            return SkipRangeParallelIteratorRegionSplitter.getInstance(context, table, hintNode);
+        }
+        return DefaultParallelIteratorRegionSplitter.getInstance(context, table, hintNode);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/iterate/ParallelIterators.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/ParallelIterators.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/ParallelIterators.java
new file mode 100644
index 0000000..8ea2d3d
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/ParallelIterators.java
@@ -0,0 +1,235 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.iterate;
+
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableSet;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
+import org.apache.hadoop.hbase.filter.PageFilter;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Function;
+import org.apache.phoenix.compile.GroupByCompiler.GroupBy;
+import org.apache.phoenix.compile.RowProjector;
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.job.JobManager.JobCallable;
+import org.apache.phoenix.parse.FilterableStatement;
+import org.apache.phoenix.parse.HintNode;
+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.schema.PTable;
+import org.apache.phoenix.schema.SaltingUtil;
+import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.SQLCloseables;
+import org.apache.phoenix.util.ScanUtil;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.ServerUtil;
+
+
+/**
+ *
+ * Class that parallelizes the scan over a table using the ExecutorService provided.  Each region of the table will be scanned in parallel with
+ * the results accessible through {@link #getIterators()}
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class ParallelIterators extends ExplainTable implements ResultIterators {
+	private static final Logger logger = LoggerFactory.getLogger(ParallelIterators.class);
+    private final List<KeyRange> splits;
+    private final ParallelIteratorFactory iteratorFactory;
+    
+    public static interface ParallelIteratorFactory {
+        PeekingResultIterator newIterator(ResultIterator scanner) throws SQLException;
+    }
+
+    private static final int DEFAULT_THREAD_TIMEOUT_MS = 60000; // 1min
+
+    static final Function<HRegionLocation, KeyRange> TO_KEY_RANGE = new Function<HRegionLocation, KeyRange>() {
+        @Override
+        public KeyRange apply(HRegionLocation region) {
+            return KeyRange.getKeyRange(region.getRegionInfo().getStartKey(), region.getRegionInfo().getEndKey());
+        }
+    };
+
+    public ParallelIterators(StatementContext context, TableRef tableRef, FilterableStatement statement, RowProjector projector, GroupBy groupBy, Integer limit, ParallelIteratorFactory iteratorFactory) throws SQLException {
+        super(context, tableRef, groupBy);
+        this.splits = getSplits(context, tableRef, statement.getHint());
+        this.iteratorFactory = iteratorFactory;
+        Scan scan = context.getScan();
+        PTable table = tableRef.getTable();
+        if (projector.isProjectEmptyKeyValue()) {
+            Map<byte [], NavigableSet<byte []>> familyMap = scan.getFamilyMap();
+            // If nothing projected into scan and we only have one column family, just allow everything
+            // to be projected and use a FirstKeyOnlyFilter to skip from row to row. This turns out to
+            // be quite a bit faster.
+            if (familyMap.isEmpty() && table.getColumnFamilies().size() == 1) {
+                // Project the one column family. We must project a column family since it's possible
+                // that there are other non declared column families that we need to ignore.
+                scan.addFamily(table.getColumnFamilies().get(0).getName().getBytes());
+                ScanUtil.andFilterAtBeginning(scan, new FirstKeyOnlyFilter());
+            } else {
+                byte[] ecf = SchemaUtil.getEmptyColumnFamily(table.getColumnFamilies());
+                // 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);
+                }
+            }
+        }
+        if (limit != null) {
+            ScanUtil.andFilterAtEnd(scan, new PageFilter(limit));
+        }
+    }
+
+    /**
+     * Splits the given scan's key range so that each split can be queried in parallel
+     * @param hintNode TODO
+     *
+     * @return the key ranges that should be scanned in parallel
+     */
+    // exposed for tests
+    public static List<KeyRange> getSplits(StatementContext context, TableRef table, HintNode hintNode) throws SQLException {
+        return ParallelIteratorRegionSplitterFactory.getSplitter(context, table, hintNode).getSplits();
+    }
+
+    public List<KeyRange> getSplits() {
+        return splits;
+    }
+
+    /**
+     * Executes the scan in parallel across all regions, blocking until all scans are complete.
+     * @return the result iterators for the scan of each region
+     */
+    @Override
+    public List<PeekingResultIterator> getIterators() throws SQLException {
+        boolean success = false;
+        final ConnectionQueryServices services = context.getConnection().getQueryServices();
+        ReadOnlyProps props = services.getProps();
+        int numSplits = splits.size();
+        List<PeekingResultIterator> iterators = new ArrayList<PeekingResultIterator>(numSplits);
+        List<Pair<byte[],Future<PeekingResultIterator>>> futures = new ArrayList<Pair<byte[],Future<PeekingResultIterator>>>(numSplits);
+        final UUID scanId = UUID.randomUUID();
+        try {
+            ExecutorService executor = services.getExecutor();
+            for (KeyRange split : splits) {
+                final Scan splitScan = new Scan(this.context.getScan());
+                // Intersect with existing start/stop key if the table is salted
+                // If not salted, we've already intersected it. If salted, we need
+                // to wait until now to intersect, as we're running parallel scans
+                // on all the possible regions here.
+                if (tableRef.getTable().getBucketNum() != null) {
+                    KeyRange minMaxRange = context.getMinMaxRange();
+                    if (minMaxRange != null) {
+                        // Add salt byte based on current split, as minMaxRange won't have it
+                        minMaxRange = SaltingUtil.addSaltByte(split.getLowerRange(), minMaxRange);
+                        split = split.intersect(minMaxRange);
+                    }
+                }
+                if (ScanUtil.intersectScanRange(splitScan, split.getLowerRange(), split.getUpperRange(), this.context.getScanRanges().useSkipScanFilter())) {
+                    // Delay the swapping of start/stop row until row so we don't muck with the intersect logic
+                    ScanUtil.swapStartStopRowIfReversed(splitScan);
+                    Future<PeekingResultIterator> future =
+                        executor.submit(new JobCallable<PeekingResultIterator>() {
+
+                        @Override
+                        public PeekingResultIterator call() throws Exception {
+                            // TODO: different HTableInterfaces for each thread or the same is better?
+                        	long startTime = System.currentTimeMillis();
+                            ResultIterator scanner = new TableResultIterator(context, tableRef, splitScan);
+                            if (logger.isDebugEnabled()) {
+                            	logger.debug("Id: " + scanId + ", Time: " + (System.currentTimeMillis() - startTime) + "ms, Scan: " + splitScan);
+                            }
+                            return iteratorFactory.newIterator(scanner);
+                        }
+
+                        /**
+                         * Defines the grouping for round robin behavior.  All threads spawned to process
+                         * this scan will be grouped together and time sliced with other simultaneously
+                         * executing parallel scans.
+                         */
+                        @Override
+                        public Object getJobId() {
+                            return ParallelIterators.this;
+                        }
+                    });
+                    futures.add(new Pair<byte[],Future<PeekingResultIterator>>(split.getLowerRange(),future));
+                }
+            }
+
+            int timeoutMs = props.getInt(QueryServices.THREAD_TIMEOUT_MS_ATTRIB, DEFAULT_THREAD_TIMEOUT_MS);
+            final int factor = ScanUtil.isReversed(this.context.getScan()) ? -1 : 1;
+            // Sort futures by row key so that we have a predicatble order we're getting rows back for scans.
+            // We're going to wait here until they're finished anyway and this makes testing much easier.
+            Collections.sort(futures, new Comparator<Pair<byte[],Future<PeekingResultIterator>>>() {
+                @Override
+                public int compare(Pair<byte[], Future<PeekingResultIterator>> o1, Pair<byte[], Future<PeekingResultIterator>> o2) {
+                    return factor * Bytes.compareTo(o1.getFirst(), o2.getFirst());
+                }
+            });
+            for (Pair<byte[],Future<PeekingResultIterator>> future : futures) {
+                iterators.add(future.getSecond().get(timeoutMs, TimeUnit.MILLISECONDS));
+            }
+
+            success = true;
+            return iterators;
+        } catch (Exception e) {
+            throw ServerUtil.parseServerException(e);
+        } finally {
+            if (!success) {
+                SQLCloseables.closeAllQuietly(iterators);
+                // Don't call cancel, as it causes the HConnection to get into a funk
+//                for (Pair<byte[],Future<PeekingResultIterator>> future : futures) {
+//                    future.getSecond().cancel(true);
+//                }
+            }
+        }
+    }
+
+    @Override
+    public int size() {
+        return this.splits.size();
+    }
+
+    @Override
+    public void explain(List<String> planSteps) {
+        StringBuilder buf = new StringBuilder();
+        buf.append("CLIENT PARALLEL " + size() + "-WAY ");
+        explain(buf.toString(),planSteps);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/iterate/PeekingResultIterator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/PeekingResultIterator.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/PeekingResultIterator.java
new file mode 100644
index 0000000..994f343
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/PeekingResultIterator.java
@@ -0,0 +1,63 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.iterate;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.phoenix.schema.tuple.Tuple;
+
+
+/**
+ * 
+ * Interface for iterating through results returned from a scan, adding the
+ * ability to peek at the next result.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public interface PeekingResultIterator extends ResultIterator {
+    public static final PeekingResultIterator EMPTY_ITERATOR = new PeekingResultIterator() {
+
+        @Override
+        public Tuple next() throws SQLException {
+            return null;
+        }
+
+        @Override
+        public Tuple peek() {
+            return null;
+        }
+
+        @Override
+        public void close() throws SQLException {
+        }
+
+        @Override
+        public void explain(List<String> planSteps) {
+        }
+    };
+
+    /**
+     * Returns the next result without advancing the iterator
+     * @throws SQLException
+     */
+    public Tuple peek() throws SQLException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..4160174
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/RegionScannerResultIterator.java
@@ -0,0 +1,65 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.iterate;
+
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.regionserver.MultiVersionConsistencyControl;
+import org.apache.hadoop.hbase.regionserver.RegionScanner;
+
+import org.apache.phoenix.schema.tuple.MultiKeyValueTuple;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.ServerUtil;
+
+
+public class RegionScannerResultIterator extends BaseResultIterator {
+    private final RegionScanner scanner;
+    
+    public RegionScannerResultIterator(RegionScanner scanner) {
+        this.scanner = scanner;
+        MultiVersionConsistencyControl.setThreadReadPoint(scanner.getMvccReadPoint());
+    }
+    
+    @Override
+    public Tuple next() throws SQLException {
+        try {
+            // TODO: size
+            List<KeyValue> results = new ArrayList<KeyValue>();
+            // 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
+            boolean hasMore = scanner.nextRaw(results, null);
+            if (!hasMore && results.isEmpty()) {
+                return null;
+            }
+            // 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.setKeyValues(results);
+            return tuple;
+        } catch (IOException e) {
+            throw ServerUtil.parseServerException(e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/iterate/ResultIterator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/ResultIterator.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/ResultIterator.java
new file mode 100644
index 0000000..43e4758
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/ResultIterator.java
@@ -0,0 +1,54 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.iterate;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.SQLCloseable;
+
+
+public interface ResultIterator extends SQLCloseable {
+    public static final ResultIterator EMPTY_ITERATOR = new ResultIterator() {
+        @Override
+        public void close() throws SQLException {
+        }
+
+        @Override
+        public Tuple next() throws SQLException {
+            return null;
+        }
+
+        @Override
+        public void explain(List<String> planSteps) {
+        }
+    };
+
+    /**
+     * Grab the next row's worth of values. The iterator will return a Tuple.
+     * @return Tuple object if there is another row, null if the scanner is
+     * exhausted.
+     * @throws SQLException e
+     */
+    public Tuple next() throws SQLException;
+    
+    public void explain(List<String> planSteps);
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/iterate/ResultIterators.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/ResultIterators.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/ResultIterators.java
new file mode 100644
index 0000000..0ee9562
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/ResultIterators.java
@@ -0,0 +1,29 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.iterate;
+
+import java.sql.SQLException;
+import java.util.List;
+
+public interface ResultIterators {
+    public List<PeekingResultIterator> getIterators() throws SQLException;
+    public int size();
+    public void explain(List<String> planSteps);
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/iterate/ScanningResultIterator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/ScanningResultIterator.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/ScanningResultIterator.java
new file mode 100644
index 0000000..0c98121
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/ScanningResultIterator.java
@@ -0,0 +1,61 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.iterate;
+
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+
+import org.apache.phoenix.schema.tuple.ResultTuple;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.ServerUtil;
+
+
+public class ScanningResultIterator implements ResultIterator {
+    private final ResultScanner scanner;
+    
+    public ScanningResultIterator(ResultScanner scanner) {
+        this.scanner = scanner;
+    }
+    
+    @Override
+    public void close() throws SQLException {
+        scanner.close();
+    }
+
+    @Override
+    public Tuple next() throws SQLException {
+        try {
+            Result result = scanner.next();
+            // TODO: use ResultTuple.setResult(result)
+            // Need to create a new one if holding on to it (i.e. OrderedResultIterator)
+            return result == null ? null : new ResultTuple(result);
+        } catch (IOException e) {
+            throw ServerUtil.parseServerException(e);
+        }
+    }
+
+    @Override
+    public void explain(List<String> planSteps) {
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/iterate/SequenceResultIterator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/SequenceResultIterator.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/SequenceResultIterator.java
new file mode 100644
index 0000000..e3aeb7f
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/SequenceResultIterator.java
@@ -0,0 +1,59 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.iterate;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.phoenix.compile.SequenceManager;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+/**
+ * 
+ * Iterates through tuples retrieving sequences from the server as needed
+ *
+ * @author jtaylor
+ * @since 3.0
+ */
+public class SequenceResultIterator extends DelegateResultIterator {
+    private final SequenceManager sequenceManager;
+    
+    public SequenceResultIterator(ResultIterator delegate, SequenceManager sequenceManager) throws SQLException {
+        super(delegate);
+        sequenceManager.initSequences();
+        this.sequenceManager = sequenceManager;
+    }
+    
+    @Override
+    public Tuple next() throws SQLException {
+        Tuple next = super.next();
+        if (next == null) {
+            return null;
+        }
+        sequenceManager.incrementSequenceValues();
+        return next;
+    }
+
+    @Override
+    public void explain(List<String> planSteps) {
+        super.explain(planSteps);
+        planSteps.add("CLIENT RESERVE " + sequenceManager.getSequenceCount() + " SEQUENCES");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/iterate/SkipRangeParallelIteratorRegionSplitter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/SkipRangeParallelIteratorRegionSplitter.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/SkipRangeParallelIteratorRegionSplitter.java
new file mode 100644
index 0000000..2f6e0fa
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/SkipRangeParallelIteratorRegionSplitter.java
@@ -0,0 +1,85 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.iterate;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.HRegionLocation;
+
+import com.google.common.base.Predicate;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import org.apache.phoenix.compile.ScanRanges;
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.parse.HintNode;
+import org.apache.phoenix.query.KeyRange;
+import org.apache.phoenix.schema.SaltingUtil;
+import org.apache.phoenix.schema.TableRef;
+
+
+/**
+ * Split the region according to the information contained in the scan's SkipScanFilter.
+ */
+public class SkipRangeParallelIteratorRegionSplitter extends DefaultParallelIteratorRegionSplitter {
+
+    public static SkipRangeParallelIteratorRegionSplitter getInstance(StatementContext context, TableRef table, HintNode hintNode) {
+        return new SkipRangeParallelIteratorRegionSplitter(context, table, hintNode);
+    }
+
+    protected SkipRangeParallelIteratorRegionSplitter(StatementContext context, TableRef table, HintNode hintNode) {
+        super(context, table, hintNode);
+    }
+
+    @Override
+    protected List<HRegionLocation> getAllRegions() throws SQLException {
+        List<HRegionLocation> allTableRegions = context.getConnection().getQueryServices().getAllTableRegions(tableRef.getTable().getPhysicalName().getBytes());
+        return filterRegions(allTableRegions, context.getScanRanges());
+    }
+
+    public List<HRegionLocation> filterRegions(List<HRegionLocation> allTableRegions, final ScanRanges ranges) {
+        Iterable<HRegionLocation> regions;
+        if (ranges == ScanRanges.EVERYTHING) {
+            return allTableRegions;
+        } else if (ranges == ScanRanges.NOTHING) { // TODO: why not emptyList?
+            return Lists.<HRegionLocation>newArrayList();
+        } else {
+            regions = Iterables.filter(allTableRegions,
+                    new Predicate<HRegionLocation>() {
+                    @Override
+                    public boolean apply(HRegionLocation region) {
+                        KeyRange minMaxRange = context.getMinMaxRange();
+                        if (minMaxRange != null) {
+                            KeyRange range = KeyRange.getKeyRange(region.getRegionInfo().getStartKey(), region.getRegionInfo().getEndKey());
+                            if (tableRef.getTable().getBucketNum() != null) {
+                                // Add salt byte, as minMaxRange won't have it
+                                minMaxRange = SaltingUtil.addSaltByte(region.getRegionInfo().getStartKey(), minMaxRange);
+                            }
+                            range = range.intersect(minMaxRange);
+                            return ranges.intersect(range.getLowerRange(), range.getUpperRange());
+                        }
+                        return ranges.intersect(region.getRegionInfo().getStartKey(), region.getRegionInfo().getEndKey());
+                    }
+            });
+        }
+        return Lists.newArrayList(regions);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/iterate/SpoolTooBigToDiskException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/SpoolTooBigToDiskException.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/SpoolTooBigToDiskException.java
new file mode 100644
index 0000000..4dffee0
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/SpoolTooBigToDiskException.java
@@ -0,0 +1,17 @@
+package org.apache.phoenix.iterate;
+
+/**
+ * Thrown by {@link org.apache.phoenix.iterate.SpoolingResultIterator } when
+ * result is too big to fit into memory and too big to spool to disk.
+ * 
+ * @author haitaoyao
+ * 
+ */
+public class SpoolTooBigToDiskException extends RuntimeException {
+
+	private static final long serialVersionUID = 1L;
+
+	public SpoolTooBigToDiskException(String msg) {
+		super(msg);
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/iterate/SpoolingResultIterator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/SpoolingResultIterator.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/SpoolingResultIterator.java
new file mode 100644
index 0000000..5acf71c
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/SpoolingResultIterator.java
@@ -0,0 +1,320 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.iterate;
+
+import java.io.*;
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.commons.io.output.DeferredFileOutputStream;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.io.WritableUtils;
+
+import org.apache.phoenix.iterate.ParallelIterators.ParallelIteratorFactory;
+import org.apache.phoenix.memory.MemoryManager;
+import org.apache.phoenix.memory.MemoryManager.MemoryChunk;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.tuple.ResultTuple;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.*;
+
+
+
+/**
+ * 
+ * Result iterator that spools the results of a scan to disk once an in-memory threshold has been reached.
+ * If the in-memory threshold is not reached, the results are held in memory with no disk writing perfomed.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class SpoolingResultIterator implements PeekingResultIterator {
+    private final PeekingResultIterator spoolFrom;
+    
+    public static class SpoolingResultIteratorFactory implements ParallelIteratorFactory {
+        private final QueryServices services;
+        
+        public SpoolingResultIteratorFactory(QueryServices services) {
+            this.services = services;
+        }
+        @Override
+        public PeekingResultIterator newIterator(ResultIterator scanner) throws SQLException {
+            return new SpoolingResultIterator(scanner, services);
+        }
+        
+    }
+
+    public SpoolingResultIterator(ResultIterator scanner, QueryServices services) throws SQLException {
+        this (scanner, services.getMemoryManager(), 
+        		services.getProps().getInt(QueryServices.SPOOL_THRESHOLD_BYTES_ATTRIB, QueryServicesOptions.DEFAULT_SPOOL_THRESHOLD_BYTES),
+        		services.getProps().getLong(QueryServices.MAX_SPOOL_TO_DISK_BYTES_ATTRIB, QueryServicesOptions.DEFAULT_MAX_SPOOL_TO_DISK_BYTES));
+    }
+    
+    /**
+    * Create a result iterator by iterating through the results of a scan, spooling them to disk once
+    * a threshold has been reached. The scanner passed in is closed prior to returning.
+    * @param scanner the results of a table scan
+    * @param mm memory manager tracking memory usage across threads.
+    * @param thresholdBytes the requested threshold.  Will be dialed down if memory usage (as determined by
+    *  the memory manager) is exceeded.
+    * @throws SQLException
+    */
+    SpoolingResultIterator(ResultIterator scanner, MemoryManager mm, final int thresholdBytes, final long maxSpoolToDisk) throws SQLException {
+        boolean success = false;
+        boolean usedOnDiskIterator = false;
+        final MemoryChunk chunk = mm.allocate(0, thresholdBytes);
+        File tempFile = null;
+        try {
+            // Can't be bigger than int, since it's the max of the above allocation
+            int size = (int)chunk.getSize();
+            tempFile = File.createTempFile("ResultSpooler",".bin");
+            DeferredFileOutputStream spoolTo = new DeferredFileOutputStream(size, tempFile) {
+                @Override
+                protected void thresholdReached() throws IOException {
+                    super.thresholdReached();
+                    chunk.close();
+                }
+            };
+            DataOutputStream out = new DataOutputStream(spoolTo);
+            final long maxBytesAllowed = maxSpoolToDisk == -1 ? 
+            		Long.MAX_VALUE : thresholdBytes + maxSpoolToDisk;
+            long bytesWritten = 0L;
+            int maxSize = 0;
+            for (Tuple result = scanner.next(); result != null; result = scanner.next()) {
+                int length = TupleUtil.write(result, out);
+                bytesWritten += length;
+                if(bytesWritten > maxBytesAllowed){
+                		throw new SpoolTooBigToDiskException("result too big, max allowed(bytes): " + maxBytesAllowed);
+                }
+                maxSize = Math.max(length, maxSize);
+            }
+            spoolTo.close();
+            if (spoolTo.isInMemory()) {
+                byte[] data = spoolTo.getData();
+                chunk.resize(data.length);
+                spoolFrom = new InMemoryResultIterator(data, chunk);
+            } else {
+                spoolFrom = new OnDiskResultIterator(maxSize, spoolTo.getFile());
+                usedOnDiskIterator = true;
+            }
+            success = true;
+        } catch (IOException e) {
+            throw ServerUtil.parseServerException(e);
+        } finally {
+            try {
+                scanner.close();
+            } finally {
+                try {
+                    if (!usedOnDiskIterator) {
+                        tempFile.delete();
+                    }
+                } finally {
+                    if (!success) {
+                        chunk.close();
+                    }
+                }
+            }
+        }
+    }
+
+    @Override
+    public Tuple peek() throws SQLException {
+        return spoolFrom.peek();
+    }
+
+    @Override
+    public Tuple next() throws SQLException {
+        return spoolFrom.next();
+    }
+    
+    @Override
+    public void close() throws SQLException {
+        spoolFrom.close();
+    }
+
+    /**
+     * 
+     * Backing result iterator if it was not necessary to spool results to disk.
+     *
+     * @author jtaylor
+     * @since 0.1
+     */
+    private static class InMemoryResultIterator implements PeekingResultIterator {
+        private final MemoryChunk memoryChunk;
+        private final byte[] bytes;
+        private Tuple next;
+        private int offset;
+        
+        private InMemoryResultIterator(byte[] bytes, MemoryChunk memoryChunk) throws SQLException {
+            this.bytes = bytes;
+            this.memoryChunk = memoryChunk;
+            advance();
+        }
+
+        private Tuple advance() throws SQLException {
+            if (offset >= bytes.length) {
+                return next = null;
+            }
+            int resultSize = ByteUtil.vintFromBytes(bytes, offset);
+            offset += WritableUtils.getVIntSize(resultSize);
+            ImmutableBytesWritable value = new ImmutableBytesWritable(bytes,offset,resultSize);
+            offset += resultSize;
+            Tuple result = new ResultTuple(new Result(value));
+            return next = result;
+        }
+        
+        @Override
+        public Tuple peek() throws SQLException {
+            return next;
+        }
+
+        @Override
+        public Tuple next() throws SQLException {
+            Tuple current = next;
+            advance();
+            return current;
+        }
+        
+        @Override
+        public void close() {
+            memoryChunk.close();
+        }
+
+        @Override
+        public void explain(List<String> planSteps) {
+        }
+    }
+    
+    /**
+     * 
+     * Backing result iterator if results were spooled to disk
+     *
+     * @author jtaylor
+     * @since 0.1
+     */
+    private static class OnDiskResultIterator implements PeekingResultIterator {
+        private final File file;
+        private DataInputStream spoolFrom;
+        private Tuple next;
+        private int maxSize;
+        private int bufferIndex;
+        private byte[][] buffers = new byte[2][];
+        private boolean isClosed;
+        
+        private OnDiskResultIterator (int maxSize, File file) {
+            this.file = file;
+            this.maxSize = maxSize;
+        }
+        
+        private synchronized void init() throws IOException {
+            if (spoolFrom == null) {
+                spoolFrom = new DataInputStream(new BufferedInputStream(new FileInputStream(file)));
+                // We need two so that we can have a current and a next without them stomping on each other
+                buffers[0] = new byte[maxSize];
+                buffers[1] = new byte[maxSize];
+                advance();
+            }
+        }
+    
+        private synchronized void reachedEnd() throws IOException {
+            next = null;
+            isClosed = true;
+            try {
+                if (spoolFrom != null) {
+                    spoolFrom.close();
+                }
+            } finally {
+                file.delete();
+            }
+        }
+        
+        private synchronized Tuple advance() throws IOException {
+            if (isClosed) {
+                return next;
+            }
+            int length;
+            try {
+                length = WritableUtils.readVInt(spoolFrom);
+            } catch (EOFException e) {
+                reachedEnd();
+                return next;
+            }
+            int totalBytesRead = 0;
+            int offset = 0;
+            // Alternate between buffers so that the current one is not affected by advancing
+            bufferIndex = (bufferIndex + 1) % 2;
+            byte[] buffer = buffers [bufferIndex];
+            while(totalBytesRead < length) {
+                int bytesRead = spoolFrom.read(buffer, offset, length);
+                if (bytesRead == -1) {
+                    reachedEnd();
+                    return next;
+                }
+                offset += bytesRead;
+                totalBytesRead += bytesRead;
+            }
+            next = new ResultTuple(new Result(new ImmutableBytesWritable(buffer,0,length)));
+            return next;
+        }
+        
+        @Override
+        public synchronized Tuple peek() throws SQLException {
+            try {
+                init();
+                return next;
+            } catch (IOException e) {
+                throw ServerUtil.parseServerException(e);
+            }
+        }
+    
+        @Override
+        public synchronized Tuple next() throws SQLException {
+            try {
+                init();
+                Tuple current = next;
+                advance();
+                return current;
+            } catch (IOException e) {
+                throw ServerUtil.parseServerException(e);
+            }
+        }
+        
+        @Override
+        public synchronized void close() throws SQLException {
+            try {
+                if (!isClosed) {
+                    reachedEnd();
+                }
+            } catch (IOException e) {
+                throw ServerUtil.parseServerException(e);
+            }
+        }
+
+        @Override
+        public void explain(List<String> planSteps) {
+        }
+    }
+
+    @Override
+    public void explain(List<String> planSteps) {
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/iterate/TableResultIterator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/TableResultIterator.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/TableResultIterator.java
new file mode 100644
index 0000000..c9354a8
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/TableResultIterator.java
@@ -0,0 +1,86 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.iterate;
+
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Scan;
+
+import com.google.common.io.Closeables;
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.ServerUtil;
+
+
+/**
+ *
+ * Wrapper for ResultScanner creation that closes HTableInterface
+ * when ResultScanner is closed.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class TableResultIterator extends ExplainTable implements ResultIterator {
+    private final HTableInterface htable;
+    private final ResultIterator delegate;
+
+    public TableResultIterator(StatementContext context, TableRef tableRef) throws SQLException {
+        this(context, tableRef, context.getScan());
+    }
+
+    public TableResultIterator(StatementContext context, TableRef tableRef, Scan scan) throws SQLException {
+        super(context, tableRef);
+        htable = context.getConnection().getQueryServices().getTable(tableRef.getTable().getPhysicalName().getBytes());
+        try {
+            delegate = new ScanningResultIterator(htable.getScanner(scan));
+        } catch (IOException e) {
+            Closeables.closeQuietly(htable);
+            throw ServerUtil.parseServerException(e);
+        }
+    }
+
+    @Override
+    public void close() throws SQLException {
+        try {
+            delegate.close();
+        } finally {
+            try {
+                htable.close();
+            } catch (IOException e) {
+                throw ServerUtil.parseServerException(e);
+            }
+        }
+    }
+
+    @Override
+    public Tuple next() throws SQLException {
+        return delegate.next();
+    }
+
+    @Override
+    public void explain(List<String> planSteps) {
+        StringBuilder buf = new StringBuilder();
+        explain(buf.toString(),planSteps);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/iterate/UngroupedAggregatingResultIterator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/UngroupedAggregatingResultIterator.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/UngroupedAggregatingResultIterator.java
new file mode 100644
index 0000000..347e8eb
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/UngroupedAggregatingResultIterator.java
@@ -0,0 +1,56 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.iterate;
+
+import static org.apache.phoenix.query.QueryConstants.*;
+
+import java.sql.SQLException;
+
+import org.apache.phoenix.expression.aggregator.Aggregators;
+import org.apache.phoenix.schema.tuple.SingleKeyValueTuple;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.KeyValueUtil;
+
+
+public class UngroupedAggregatingResultIterator extends GroupedAggregatingResultIterator {
+    private boolean hasRows = false;
+
+    public UngroupedAggregatingResultIterator( PeekingResultIterator resultIterator, Aggregators aggregators) {
+        super(resultIterator, aggregators);
+    }
+    
+    @Override
+    public Tuple next() throws SQLException {
+        Tuple result = super.next();
+        // Ensure ungrouped aggregregation always returns a row, even if the underlying iterator doesn't.
+        if (result == null && !hasRows) {
+            // Generate value using unused ClientAggregators
+            byte[] value = aggregators.toBytes(aggregators.getAggregators());
+            result = new SingleKeyValueTuple(
+                    KeyValueUtil.newKeyValue(UNGROUPED_AGG_ROW_KEY, 
+                            SINGLE_COLUMN_FAMILY, 
+                            SINGLE_COLUMN, 
+                            AGG_TIMESTAMP, 
+                            value));
+        }
+        hasRows = true;
+        return result;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/jdbc/Jdbc7Shim.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/Jdbc7Shim.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/Jdbc7Shim.java
new file mode 100644
index 0000000..7a73a4e
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/Jdbc7Shim.java
@@ -0,0 +1,66 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.jdbc;
+
+import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
+import java.util.concurrent.Executor;
+import java.util.logging.Logger;
+
+/**
+ * Interfaces to be implemented by classes that need to be "JDK7" compliant,
+ * but also run in JDK6
+ */
+public final class Jdbc7Shim {
+
+    public interface Statement {  // Note: do not extend "regular" statement or else eclipse 3.7 complains
+        void closeOnCompletion() throws SQLException;
+        boolean isCloseOnCompletion() throws SQLException;
+    }
+
+    public interface CallableStatement extends Statement {
+        public <T> T getObject(int columnIndex, Class<T> type) throws SQLException;
+        public <T> T getObject(String columnLabel, Class<T> type) throws SQLException;
+    }
+
+    public interface Connection {
+         void setSchema(String schema) throws SQLException;
+         String getSchema() throws SQLException;
+         void abort(Executor executor) throws SQLException;
+         void setNetworkTimeout(Executor executor, int milliseconds) throws SQLException;
+         int getNetworkTimeout() throws SQLException;
+    }
+
+    public interface ResultSet {
+         public <T> T getObject(int columnIndex, Class<T> type) throws SQLException;
+         public <T> T getObject(String columnLabel, Class<T> type) throws SQLException;
+    }
+
+    public interface DatabaseMetaData {
+        java.sql.ResultSet getPseudoColumns(String catalog, String schemaPattern,
+                             String tableNamePattern, String columnNamePattern)
+            throws SQLException;
+        boolean  generatedKeyAlwaysReturned() throws SQLException;
+    }
+
+    public interface Driver {
+        public Logger getParentLogger() throws SQLFeatureNotSupportedException;
+    }
+}
\ No newline at end of file


[17/51] [partial] Initial commit of master branch from github

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseContext.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseContext.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseContext.java
new file mode 100644
index 0000000..275ceb2
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseContext.java
@@ -0,0 +1,60 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.util.List;
+
+import com.google.common.collect.Lists;
+
+public class ParseContext {
+    private boolean isAggregate;
+    
+    public ParseContext() {
+    }
+
+    public boolean isAggregate() {
+        return isAggregate;
+    }
+
+    public void setAggregate(boolean isAggregate) {
+        this.isAggregate |= isAggregate;
+    }
+
+    public static class Stack {
+        private final List<ParseContext> stack = Lists.newArrayListWithExpectedSize(5);
+        
+        public void push(ParseContext context) {
+            stack.add(context);
+        }
+        
+        public ParseContext pop() {
+            return stack.remove(stack.size()-1);
+        }
+        
+        public ParseContext peek() {
+            return stack.get(stack.size()-1);
+        }
+        
+        public boolean isEmpty() {
+            return stack.isEmpty();
+        }
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseException.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseException.java
new file mode 100644
index 0000000..010dd88
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseException.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+/**
+ * 
+ * RuntimeException for exceptions occurring during parsing,
+ * since ANTLR doesn't handle typed exceptions well.
+ *
+ * @author jtaylor
+ * @since 2.0
+ */
+public class ParseException extends RuntimeException {
+
+    public ParseException() {
+    }
+
+    public ParseException(String msg) {
+        super(msg);
+    }
+
+    public ParseException(Throwable t) {
+        super(t);
+    }
+
+    public ParseException(String msg, Throwable t) {
+        super(msg, t);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNode.java
new file mode 100644
index 0000000..46a385c
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNode.java
@@ -0,0 +1,52 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+import java.util.List;
+
+
+
+
+/**
+ * 
+ * Abstract base class for a parse node in SQL
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public abstract class ParseNode {
+    public abstract List<ParseNode> getChildren();
+    public abstract <T> T accept(ParseNodeVisitor<T> visitor) throws SQLException;
+    
+    public boolean isStateless() {
+        return false;
+    }
+    
+    /**
+     * Allows node to override what the alias is for a given node.
+     * Useful for a column reference, as JDBC says that the alias
+     * name for "a.b" should be "b"
+     * @return the alias to use for this node or null for no alias
+     */
+    public String getAlias() {
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
new file mode 100644
index 0000000..20082a2
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
@@ -0,0 +1,564 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.lang.reflect.Constructor;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+import org.apache.hadoop.hbase.util.Pair;
+
+import com.google.common.collect.ListMultimap;
+import com.google.common.collect.Maps;
+import org.apache.phoenix.exception.UnknownFunctionException;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.ExpressionType;
+import org.apache.phoenix.expression.function.AvgAggregateFunction;
+import org.apache.phoenix.expression.function.CountAggregateFunction;
+import org.apache.phoenix.expression.function.CurrentDateFunction;
+import org.apache.phoenix.expression.function.CurrentTimeFunction;
+import org.apache.phoenix.expression.function.DistinctCountAggregateFunction;
+import org.apache.phoenix.expression.function.FunctionExpression;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunctionInfo;
+import org.apache.phoenix.parse.JoinTableNode.JoinType;
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.PIndexState;
+import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.schema.TypeMismatchException;
+import org.apache.phoenix.util.SchemaUtil;
+
+
+/**
+ *
+ * Factory used by parser to construct object model while parsing a SQL statement
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class ParseNodeFactory {
+    private static final String ARRAY_ELEM = "ARRAY_ELEM";
+	// TODO: Use Google's Reflection library instead to find aggregate functions
+    @SuppressWarnings("unchecked")
+    private static final List<Class<? extends FunctionExpression>> CLIENT_SIDE_BUILT_IN_FUNCTIONS = Arrays.<Class<? extends FunctionExpression>>asList(
+        CurrentDateFunction.class,
+        CurrentTimeFunction.class,
+        AvgAggregateFunction.class
+        );
+    private static final Map<BuiltInFunctionKey, BuiltInFunctionInfo> BUILT_IN_FUNCTION_MAP = Maps.newHashMap();
+
+    /**
+     *
+     * Key used to look up a built-in function using the combination of
+     * the lowercase name and the number of arguments. This disambiguates
+     * the aggregate MAX(<col>) from the non aggregate MAX(<col1>,<col2>).
+     *
+     * @author jtaylor
+     * @since 0.1
+     */
+    private static class BuiltInFunctionKey {
+        private final String upperName;
+        private final int argCount;
+
+        private BuiltInFunctionKey(String lowerName, int argCount) {
+            this.upperName = lowerName;
+            this.argCount = argCount;
+        }
+        
+        @Override
+        public String toString() {
+            return upperName;
+        }
+        
+        @Override
+        public int hashCode() {
+            final int prime = 31;
+            int result = 1;
+            result = prime * result + argCount;
+            result = prime * result + ((upperName == null) ? 0 : upperName.hashCode());
+            return result;
+        }
+
+        @Override
+        public boolean equals(Object obj) {
+            if (this == obj) return true;
+            if (obj == null) return false;
+            if (getClass() != obj.getClass()) return false;
+            BuiltInFunctionKey other = (BuiltInFunctionKey)obj;
+            if (argCount != other.argCount) return false;
+            if (!upperName.equals(other.upperName)) return false;
+            return true;
+        }
+    }
+
+    private static void addBuiltInFunction(Class<? extends FunctionExpression> f) throws Exception {
+        BuiltInFunction d = f.getAnnotation(BuiltInFunction.class);
+        if (d == null) {
+            return;
+        }
+        int nArgs = d.args().length;
+        BuiltInFunctionInfo value = new BuiltInFunctionInfo(f, d);
+        do {
+            // Add function to function map, throwing if conflicts found
+            // Add entry for each possible version of function based on arguments that are not required to be present (i.e. arg with default value)
+            BuiltInFunctionKey key = new BuiltInFunctionKey(value.getName(), nArgs);
+            if (BUILT_IN_FUNCTION_MAP.put(key, value) != null) {
+                throw new IllegalStateException("Multiple " + value.getName() + " functions with " + nArgs + " arguments");
+            }
+        } while (--nArgs >= 0 && d.args()[nArgs].defaultValue().length() > 0);
+
+        // Look for default values that aren't at the end and throw
+        while (--nArgs >= 0) {
+            if (d.args()[nArgs].defaultValue().length() > 0) {
+                throw new IllegalStateException("Function " + value.getName() + " has non trailing default value of '" + d.args()[nArgs].defaultValue() + "'. Only trailing arguments may have default values");
+            }
+        }
+    }
+    /**
+     * Reflect this class and populate static structures from it.
+     * Don't initialize in static block because we have a circular dependency
+     */
+    private synchronized static void initBuiltInFunctionMap() {
+        if (!BUILT_IN_FUNCTION_MAP.isEmpty()) {
+            return;
+        }
+        Class<? extends FunctionExpression> f = null;
+        try {
+            // Reflection based parsing which yields direct explicit function evaluation at runtime
+            for (int i = 0; i < CLIENT_SIDE_BUILT_IN_FUNCTIONS.size(); i++) {
+                f = CLIENT_SIDE_BUILT_IN_FUNCTIONS.get(i);
+                addBuiltInFunction(f);
+            }
+            for (ExpressionType et : ExpressionType.values()) {
+                Class<? extends Expression> ec = et.getExpressionClass();
+                if (FunctionExpression.class.isAssignableFrom(ec)) {
+                    @SuppressWarnings("unchecked")
+                    Class<? extends FunctionExpression> c = (Class<? extends FunctionExpression>)ec;
+                    addBuiltInFunction(f = c);
+                }
+            }
+        } catch (Exception e) {
+            throw new RuntimeException("Failed initialization of built-in functions at class '" + f + "'", e);
+        }
+    }
+
+    private static BuiltInFunctionInfo getInfo(String name, List<ParseNode> children) {
+        return get(SchemaUtil.normalizeIdentifier(name), children);
+    }
+
+    public static BuiltInFunctionInfo get(String normalizedName, List<ParseNode> children) {
+        initBuiltInFunctionMap();
+        BuiltInFunctionInfo info = BUILT_IN_FUNCTION_MAP.get(new BuiltInFunctionKey(normalizedName,children.size()));
+        if (info == null) {
+            throw new UnknownFunctionException(normalizedName);
+        }
+        return info;
+    }
+
+    public ParseNodeFactory() {
+    }
+
+    public ExplainStatement explain(BindableStatement statement) {
+        return new ExplainStatement(statement);
+    }
+
+    public AliasedNode aliasedNode(String alias, ParseNode expression) {
+    	return new AliasedNode(alias, expression);
+    }
+    
+    public AddParseNode add(List<ParseNode> children) {
+        return new AddParseNode(children);
+    }
+
+    public SubtractParseNode subtract(List<ParseNode> children) {
+        return new SubtractParseNode(children);
+    }
+
+    public MultiplyParseNode multiply(List<ParseNode> children) {
+        return new MultiplyParseNode(children);
+    }
+
+    public AndParseNode and(List<ParseNode> children) {
+        return new AndParseNode(children);
+    }
+    
+    public FamilyWildcardParseNode family(String familyName){
+    	    return new FamilyWildcardParseNode(familyName, false);
+    }
+
+    public WildcardParseNode wildcard() {
+        return WildcardParseNode.INSTANCE;
+    }
+
+    public BetweenParseNode between(ParseNode l, ParseNode r1, ParseNode r2, boolean negate) {
+        return new BetweenParseNode(l, r1, r2, negate);
+    }
+
+    public BindParseNode bind(String bind) {
+        return new BindParseNode(bind);
+    }
+
+    public StringConcatParseNode concat(List<ParseNode> children) {
+        return new StringConcatParseNode(children);
+    }
+
+    public ColumnParseNode column(TableName tableName, String name, String alias) {
+        return new ColumnParseNode(tableName,name,alias);
+    }
+    
+    public ColumnName columnName(String columnName) {
+        return new ColumnName(columnName);
+    }
+
+    public ColumnName columnName(String familyName, String columnName) {
+        return new ColumnName(familyName, columnName);
+    }
+
+    public PropertyName propertyName(String propertyName) {
+        return new PropertyName(propertyName);
+    }
+
+    public PropertyName propertyName(String familyName, String propertyName) {
+        return new PropertyName(familyName, propertyName);
+    }
+
+    public ColumnDef columnDef(ColumnName columnDefName, String sqlTypeName, boolean isNull, Integer maxLength, Integer scale, boolean isPK, ColumnModifier columnModifier) {
+        return new ColumnDef(columnDefName, sqlTypeName, isNull, maxLength, scale, isPK, columnModifier);
+    }
+    
+    public ColumnDef columnDef(ColumnName columnDefName, String sqlTypeName, boolean isArray, Integer arrSize, boolean isNull, Integer maxLength, Integer scale, boolean isPK, 
+        	ColumnModifier columnModifier) {
+        return new ColumnDef(columnDefName, sqlTypeName, isArray, arrSize, isNull, maxLength, scale, isPK, columnModifier);
+    }
+
+    public PrimaryKeyConstraint primaryKey(String name, List<Pair<ColumnName, ColumnModifier>> columnNameAndModifier) {
+        return new PrimaryKeyConstraint(name, columnNameAndModifier);
+    }
+    
+    public CreateTableStatement createTable(TableName tableName, ListMultimap<String,Pair<String,Object>> props, List<ColumnDef> columns, PrimaryKeyConstraint pkConstraint, List<ParseNode> splits, PTableType tableType, boolean ifNotExists, TableName baseTableName, ParseNode tableTypeIdNode, int bindCount) {
+        return new CreateTableStatement(tableName, props, columns, pkConstraint, splits, tableType, ifNotExists, baseTableName, tableTypeIdNode, bindCount);
+    }
+    
+    public CreateIndexStatement createIndex(NamedNode indexName, NamedTableNode dataTable, PrimaryKeyConstraint pkConstraint, List<ColumnName> includeColumns, List<ParseNode> splits, ListMultimap<String,Pair<String,Object>> props, boolean ifNotExists, int bindCount) {
+        return new CreateIndexStatement(indexName, dataTable, pkConstraint, includeColumns, splits, props, ifNotExists, bindCount);
+    }
+    
+    public CreateSequenceStatement createSequence(TableName tableName, ParseNode startsWith, ParseNode incrementBy, ParseNode cacheSize, boolean ifNotExits, int bindCount){
+    	return new CreateSequenceStatement(tableName, startsWith, incrementBy, cacheSize, ifNotExits, bindCount);
+    } 
+    
+    public DropSequenceStatement dropSequence(TableName tableName, boolean ifExits, int bindCount){
+        return new DropSequenceStatement(tableName, ifExits, bindCount);
+    }
+    
+	public SequenceValueParseNode currentValueFor(TableName tableName) {
+		return new SequenceValueParseNode(tableName, SequenceValueParseNode.Op.CURRENT_VALUE);
+	}
+    
+    public SequenceValueParseNode nextValueFor(TableName tableName) {
+        return new SequenceValueParseNode(tableName, SequenceValueParseNode.Op.NEXT_VALUE);
+    }
+    
+    public AddColumnStatement addColumn(NamedTableNode table,  PTableType tableType, List<ColumnDef> columnDefs, boolean ifNotExists, Map<String,Object> props) {
+        return new AddColumnStatement(table, tableType, columnDefs, ifNotExists, props);
+    }
+    
+    public DropColumnStatement dropColumn(NamedTableNode table,  PTableType tableType, List<ColumnName> columnNodes, boolean ifExists) {
+        return new DropColumnStatement(table, tableType, columnNodes, ifExists);
+    }
+    
+    public DropTableStatement dropTable(TableName tableName, PTableType tableType, boolean ifExists) {
+        return new DropTableStatement(tableName, tableType, ifExists);
+    }
+    
+    public DropIndexStatement dropIndex(NamedNode indexName, TableName tableName, boolean ifExists) {
+        return new DropIndexStatement(indexName, tableName, ifExists);
+    }
+    
+    public AlterIndexStatement alterIndex(NamedTableNode indexTableNode, String dataTableName, boolean ifExists, PIndexState state) {
+        return new AlterIndexStatement(indexTableNode, dataTableName, ifExists, state);
+    }
+    
+    public TableName table(String schemaName, String tableName) {
+        return TableName.createNormalized(schemaName,tableName);
+    }
+
+    public NamedNode indexName(String name) {
+        return new NamedNode(name);
+    }
+
+    public NamedTableNode namedTable(String alias, TableName name) {
+        return new NamedTableNode(alias, name);
+    }
+
+    public NamedTableNode namedTable(String alias, TableName name ,List<ColumnDef> dyn_columns) {
+        return new NamedTableNode(alias, name,dyn_columns);
+    }
+
+    public BindTableNode bindTable(String alias, TableName name) {
+        return new BindTableNode(alias, name);
+    }
+
+    public CaseParseNode caseWhen(List<ParseNode> children) {
+        return new CaseParseNode(children);
+    }
+
+    public DivideParseNode divide(List<ParseNode> children) {
+        return new DivideParseNode(children);
+    }
+
+
+    public FunctionParseNode functionDistinct(String name, List<ParseNode> args) {
+        if (CountAggregateFunction.NAME.equals(SchemaUtil.normalizeIdentifier(name))) {
+            BuiltInFunctionInfo info = getInfo(
+                    SchemaUtil.normalizeIdentifier(DistinctCountAggregateFunction.NAME), args);
+            return new DistinctCountParseNode(name, args, info);
+        } else {
+            throw new UnsupportedOperationException("DISTINCT not supported with " + name);
+        }
+    }
+    
+    public FunctionParseNode arrayElemRef(List<ParseNode> args) {
+    	return function(ARRAY_ELEM, args);
+    }
+
+    public FunctionParseNode function(String name, List<ParseNode> args) {
+        BuiltInFunctionInfo info = getInfo(name, args);
+        Constructor<? extends FunctionParseNode> ctor = info.getNodeCtor();
+        if (ctor == null) {
+            return info.isAggregate()
+            ? new AggregateFunctionParseNode(name, args, info)
+            : new FunctionParseNode(name, args, info);
+        } else {
+            try {
+                return ctor.newInstance(name, args, info);
+            } catch (Exception e) {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+
+    public FunctionParseNode function(String name, List<ParseNode> valueNodes,
+            List<ParseNode> columnNodes, boolean isAscending) {
+        // Right now we support PERCENT functions on only one column
+        if (valueNodes.size() != 1 || columnNodes.size() != 1) {
+            throw new UnsupportedOperationException(name + " not supported on multiple columns");
+        }
+        List<ParseNode> children = new ArrayList<ParseNode>(3);
+        children.add(columnNodes.get(0));
+        children.add(new LiteralParseNode(Boolean.valueOf(isAscending)));
+        children.add(valueNodes.get(0));
+        return function(name, children);
+    }
+    
+
+    public HintNode hint(String hint) {
+        return new HintNode(hint);
+    }
+
+    public InListParseNode inList(List<ParseNode> children, boolean negate) {
+        return new InListParseNode(children, negate);
+    }
+
+    public ExistsParseNode exists(ParseNode l, ParseNode r, boolean negate) {
+        return new ExistsParseNode(l, r, negate);
+    }
+
+    public InParseNode in(ParseNode l, ParseNode r, boolean negate) {
+        return new InParseNode(l, r, negate);
+    }
+
+    public IsNullParseNode isNull(ParseNode child, boolean negate) {
+        return new IsNullParseNode(child, negate);
+    }
+
+    public JoinTableNode join (JoinType type, ParseNode on, TableNode table) {
+        return new JoinTableNode(type, on, table);
+    }
+
+    public DerivedTableNode derivedTable (String alias, SelectStatement select) {
+        return new DerivedTableNode(alias, select);
+    }
+
+    public LikeParseNode like(ParseNode lhs, ParseNode rhs, boolean negate) {
+        return new LikeParseNode(lhs, rhs, negate);
+    }
+
+
+    public LiteralParseNode literal(Object value) {
+        return new LiteralParseNode(value);
+    }
+    
+    public CastParseNode cast(ParseNode expression, String dataType) {
+    	return new CastParseNode(expression, dataType);
+    }
+    
+    public CastParseNode cast(ParseNode expression, PDataType dataType) {
+    	return new CastParseNode(expression, dataType);
+    }
+    
+    public ParseNode rowValueConstructor(List<ParseNode> l) {
+        return new RowValueConstructorParseNode(l);
+    }
+    
+    private void checkTypeMatch (PDataType expectedType, PDataType actualType) throws SQLException {
+        if (!expectedType.isCoercibleTo(actualType)) {
+            throw TypeMismatchException.newException(expectedType, actualType);
+        }
+    }
+
+    public LiteralParseNode literal(Object value, PDataType expectedType) throws SQLException {
+        PDataType actualType = PDataType.fromLiteral(value);
+        if (actualType != null && actualType != expectedType) {
+            checkTypeMatch(expectedType, actualType);
+            value = expectedType.toObject(value, actualType);
+        }
+        return new LiteralParseNode(value);
+    }
+
+    public LiteralParseNode coerce(LiteralParseNode literalNode, PDataType expectedType) throws SQLException {
+        PDataType actualType = literalNode.getType();
+        if (actualType != null) {
+            Object before = literalNode.getValue();
+            checkTypeMatch(expectedType, actualType);
+            Object after = expectedType.toObject(before, actualType);
+            if (before != after) {
+                literalNode = literal(after);
+            }
+        }
+        return literalNode;
+    }
+
+    public ComparisonParseNode comparison(CompareOp op, ParseNode lhs, ParseNode rhs) {
+        switch (op){
+        case LESS:
+            return lt(lhs,rhs);
+        case LESS_OR_EQUAL:
+            return lte(lhs,rhs);
+        case EQUAL:
+            return equal(lhs,rhs);
+        case NOT_EQUAL:
+            return notEqual(lhs,rhs);
+        case GREATER_OR_EQUAL:
+            return gte(lhs,rhs);
+        case GREATER:
+            return gt(lhs,rhs);
+        default:
+            throw new IllegalArgumentException("Unexpcted CompareOp of " + op);
+        }
+    }
+
+    public GreaterThanParseNode gt(ParseNode lhs, ParseNode rhs) {
+        return new GreaterThanParseNode(lhs, rhs);
+    }
+
+
+    public GreaterThanOrEqualParseNode gte(ParseNode lhs, ParseNode rhs) {
+        return new GreaterThanOrEqualParseNode(lhs, rhs);
+    }
+
+    public LessThanParseNode lt(ParseNode lhs, ParseNode rhs) {
+        return new LessThanParseNode(lhs, rhs);
+    }
+
+
+    public LessThanOrEqualParseNode lte(ParseNode lhs, ParseNode rhs) {
+        return new LessThanOrEqualParseNode(lhs, rhs);
+    }
+
+    public EqualParseNode equal(ParseNode lhs, ParseNode rhs) {
+        return new EqualParseNode(lhs, rhs);
+    }
+
+    public ArrayConstructorNode upsertStmtArrayNode(List<ParseNode> upsertStmtArray) {
+    	return new ArrayConstructorNode(upsertStmtArray);
+    }
+
+    public MultiplyParseNode negate(ParseNode child) {
+        return new MultiplyParseNode(Arrays.asList(child,this.literal(-1)));
+    }
+
+    public NotEqualParseNode notEqual(ParseNode lhs, ParseNode rhs) {
+        return new NotEqualParseNode(lhs, rhs);
+    }
+
+    public NotParseNode not(ParseNode child) {
+        return new NotParseNode(child);
+    }
+
+
+    public OrParseNode or(List<ParseNode> children) {
+        return new OrParseNode(children);
+    }
+
+
+    public OrderByNode orderBy(ParseNode expression, boolean nullsLast, boolean orderAscending) {
+        return new OrderByNode(expression, nullsLast, orderAscending);
+    }
+
+
+    public OuterJoinParseNode outer(ParseNode node) {
+        return new OuterJoinParseNode(node);
+    }
+    
+    public SelectStatement select(List<? extends TableNode> from, HintNode hint, boolean isDistinct, List<AliasedNode> select, ParseNode where,
+            List<ParseNode> groupBy, ParseNode having, List<OrderByNode> orderBy, LimitNode limit, int bindCount, boolean isAggregate) {
+
+        return new SelectStatement(from, hint, isDistinct, select, where, groupBy == null ? Collections.<ParseNode>emptyList() : groupBy, having, orderBy == null ? Collections.<OrderByNode>emptyList() : orderBy, limit, bindCount, isAggregate);
+    }
+    
+    public UpsertStatement upsert(NamedTableNode table, HintNode hint, List<ColumnName> columns, List<ParseNode> values, SelectStatement select, int bindCount) {
+        return new UpsertStatement(table, hint, columns, values, select, bindCount);
+    }
+    
+    public DeleteStatement delete(NamedTableNode table, HintNode hint, ParseNode node, List<OrderByNode> orderBy, LimitNode limit, int bindCount) {
+        return new DeleteStatement(table, hint, node, orderBy, limit, bindCount);
+    }
+
+    public SelectStatement select(SelectStatement statement, ParseNode where, ParseNode having) {
+        return select(statement.getFrom(), statement.getHint(), statement.isDistinct(), statement.getSelect(), where, statement.getGroupBy(), having, statement.getOrderBy(), statement.getLimit(), statement.getBindCount(), statement.isAggregate());
+    }
+
+    public SelectStatement select(SelectStatement statement, List<? extends TableNode> tables) {
+        return select(tables, statement.getHint(), statement.isDistinct(), statement.getSelect(), statement.getWhere(), statement.getGroupBy(), statement.getHaving(), statement.getOrderBy(), statement.getLimit(), statement.getBindCount(), statement.isAggregate());
+    }
+
+    public SelectStatement select(SelectStatement statement, HintNode hint) {
+        return hint == null || hint.isEmpty() ? statement : select(statement.getFrom(), hint, statement.isDistinct(), statement.getSelect(), statement.getWhere(), statement.getGroupBy(), statement.getHaving(), statement.getOrderBy(), statement.getLimit(), statement.getBindCount(), statement.isAggregate());
+    }
+
+    public SubqueryParseNode subquery(SelectStatement select) {
+        return new SubqueryParseNode(select);
+    }
+
+    public LimitNode limit(BindParseNode b) {
+        return new LimitNode(b);
+    }
+
+    public LimitNode limit(LiteralParseNode l) {
+        return new LimitNode(l);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeRewriter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeRewriter.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeRewriter.java
new file mode 100644
index 0000000..504b496
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeRewriter.java
@@ -0,0 +1,553 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import org.apache.phoenix.compile.ColumnResolver;
+import org.apache.phoenix.schema.AmbiguousColumnException;
+import org.apache.phoenix.schema.ColumnNotFoundException;
+
+/**
+ * 
+ * Base class for visitors that rewrite the expression node hierarchy
+ *
+ * @author jtaylor 
+ * @since 0.1
+ */
+public class ParseNodeRewriter extends TraverseAllParseNodeVisitor<ParseNode> {
+    
+    protected static final ParseNodeFactory NODE_FACTORY = new ParseNodeFactory();
+
+    public static ParseNode rewrite(ParseNode where, ParseNodeRewriter rewriter) throws SQLException {
+        if (where == null) {
+            return null;
+        }
+        rewriter.reset();
+        return where.accept(rewriter);
+    }
+    
+    /**
+     * Rewrite the select statement by switching any constants to the right hand side
+     * of the expression.
+     * @param statement the select statement
+     * @return new select statement
+     * @throws SQLException 
+     */
+    public static SelectStatement rewrite(SelectStatement statement, ParseNodeRewriter rewriter) throws SQLException {
+        Map<String,ParseNode> aliasMap = rewriter.getAliasMap();
+        List<TableNode> from = statement.getFrom();
+        List<TableNode> normFrom = from;
+        if (from.size() > 1) {
+        	for (int i = 1; i < from.size(); i++) {
+        		TableNode tableNode = from.get(i);
+        		if (tableNode instanceof JoinTableNode) {
+        			JoinTableNode joinTableNode = (JoinTableNode) tableNode;
+        			ParseNode onNode = joinTableNode.getOnNode();
+        			rewriter.reset();
+        			ParseNode normOnNode = onNode.accept(rewriter);
+        			if (onNode == normOnNode) {
+        				if (from != normFrom) {
+        					normFrom.add(tableNode);
+        				}
+        				continue;
+        			}
+        			if (from == normFrom) {
+        				normFrom = Lists.newArrayList(from.subList(0, i));
+        			}
+        			TableNode normTableNode = NODE_FACTORY.join(joinTableNode.getType(), normOnNode, joinTableNode.getTable());
+        			normFrom.add(normTableNode);
+        		} else if (from != normFrom) {
+					normFrom.add(tableNode);
+				}
+        	}
+        }
+        ParseNode where = statement.getWhere();
+        ParseNode normWhere = where;
+        if (where != null) {
+            rewriter.reset();
+            normWhere = where.accept(rewriter);
+        }
+        ParseNode having = statement.getHaving();
+        ParseNode normHaving= having;
+        if (having != null) {
+            rewriter.reset();
+            normHaving = having.accept(rewriter);
+        }
+        List<AliasedNode> selectNodes = statement.getSelect();
+        List<AliasedNode> normSelectNodes = selectNodes;
+        for (int i = 0; i < selectNodes.size(); i++) {
+            AliasedNode aliasedNode = selectNodes.get(i);
+            ParseNode selectNode = aliasedNode.getNode();
+            rewriter.reset();
+            ParseNode normSelectNode = selectNode.accept(rewriter);
+            if (selectNode == normSelectNode) {
+                if (selectNodes != normSelectNodes) {
+                    normSelectNodes.add(aliasedNode);
+                }
+                continue;
+            }
+            if (selectNodes == normSelectNodes) {
+                normSelectNodes = Lists.newArrayList(selectNodes.subList(0, i));
+            }
+            AliasedNode normAliasedNode = NODE_FACTORY.aliasedNode(aliasedNode.isCaseSensitve() ? '"' + aliasedNode.getAlias() + '"' : aliasedNode.getAlias(), normSelectNode);
+            normSelectNodes.add(normAliasedNode);
+        }
+        // Add to map in separate pass so that we don't try to use aliases
+        // while processing the select expressions
+        if (aliasMap != null) {
+            for (int i = 0; i < normSelectNodes.size(); i++) {
+                AliasedNode aliasedNode = normSelectNodes.get(i);
+                ParseNode selectNode = aliasedNode.getNode();
+                String alias = aliasedNode.getAlias();
+                if (alias != null) {
+                    aliasMap.put(alias, selectNode);
+                }
+            }
+        }
+        
+        List<ParseNode> groupByNodes = statement.getGroupBy();
+        List<ParseNode> normGroupByNodes = groupByNodes;
+        for (int i = 0; i < groupByNodes.size(); i++) {
+            ParseNode groupByNode = groupByNodes.get(i);
+            rewriter.reset();
+            ParseNode normGroupByNode = groupByNode.accept(rewriter);
+            if (groupByNode == normGroupByNode) {
+                if (groupByNodes != normGroupByNodes) {
+                    normGroupByNodes.add(groupByNode);
+                }
+                continue;
+            }
+            if (groupByNodes == normGroupByNodes) {
+                normGroupByNodes = Lists.newArrayList(groupByNodes.subList(0, i));
+            }
+            normGroupByNodes.add(normGroupByNode);
+        }
+        List<OrderByNode> orderByNodes = statement.getOrderBy();
+        List<OrderByNode> normOrderByNodes = orderByNodes;
+        for (int i = 0; i < orderByNodes.size(); i++) {
+            OrderByNode orderByNode = orderByNodes.get(i);
+            ParseNode node = orderByNode.getNode();
+            rewriter.reset();
+            ParseNode normNode = node.accept(rewriter);
+            if (node == normNode) {
+                if (orderByNodes != normOrderByNodes) {
+                    normOrderByNodes.add(orderByNode);
+                }
+                continue;
+            }
+            if (orderByNodes == normOrderByNodes) {
+                normOrderByNodes = Lists.newArrayList(orderByNodes.subList(0, i));
+            }
+            normOrderByNodes.add(NODE_FACTORY.orderBy(normNode, orderByNode.isNullsLast(), orderByNode.isAscending()));
+        }
+        
+        // Return new SELECT statement with updated WHERE clause
+        if (normFrom == from && 
+        		normWhere == where && 
+                normHaving == having && 
+                selectNodes == normSelectNodes && 
+                groupByNodes == normGroupByNodes &&
+                orderByNodes == normOrderByNodes) {
+            return statement;
+        }
+        return NODE_FACTORY.select(normFrom, statement.getHint(), statement.isDistinct(),
+                normSelectNodes, normWhere, normGroupByNodes, normHaving, normOrderByNodes,
+                statement.getLimit(), statement.getBindCount(), statement.isAggregate());
+    }
+    
+    private Map<String, ParseNode> getAliasMap() {
+        return aliasMap;
+    }
+
+    private final ColumnResolver resolver;
+    private final Map<String, ParseNode> aliasMap;
+    private int nodeCount;
+    
+    public boolean isTopLevel() {
+        return nodeCount == 0;
+    }
+    
+    protected ParseNodeRewriter() {
+        this.resolver = null;
+        this.aliasMap = null;
+    }
+    
+    protected ParseNodeRewriter(ColumnResolver resolver) {
+        this.resolver = resolver;
+        this.aliasMap = null;
+    }
+    
+    protected ParseNodeRewriter(ColumnResolver resolver, int maxAliasCount) {
+        this.resolver = resolver;
+        this.aliasMap = Maps.newHashMapWithExpectedSize(maxAliasCount);
+    }
+    
+    protected ColumnResolver getResolver() {
+        return resolver;
+    }
+    
+    protected void reset() {
+        this.nodeCount = 0;
+    }
+    
+    private static interface CompoundNodeFactory {
+        ParseNode createNode(List<ParseNode> children);
+    }
+    
+    private ParseNode leaveCompoundNode(CompoundParseNode node, List<ParseNode> children, CompoundNodeFactory factory) {
+        if (children.equals(node.getChildren())) {
+            return node;
+        } else { // Child nodes have been inverted (because a literal was found on LHS)
+            return factory.createNode(children);
+        }
+    }
+    
+    @Override
+    public ParseNode visitLeave(AndParseNode node, List<ParseNode> nodes) throws SQLException {
+        return leaveCompoundNode(node, nodes, new CompoundNodeFactory() {
+            @Override
+            public ParseNode createNode(List<ParseNode> children) {
+                return NODE_FACTORY.and(children);
+            }
+        });
+    }
+
+    @Override
+    public ParseNode visitLeave(OrParseNode node, List<ParseNode> nodes) throws SQLException {
+        return leaveCompoundNode(node, nodes, new CompoundNodeFactory() {
+            @Override
+            public ParseNode createNode(List<ParseNode> children) {
+                return NODE_FACTORY.or(children);
+            }
+        });
+    }
+    
+    @Override
+    public ParseNode visitLeave(SubtractParseNode node, List<ParseNode> nodes) throws SQLException {
+        return leaveCompoundNode(node, nodes, new CompoundNodeFactory() {
+            @Override
+            public ParseNode createNode(List<ParseNode> children) {
+                return NODE_FACTORY.subtract(children);
+            }
+        });
+    }
+    
+    @Override
+    public ParseNode visitLeave(AddParseNode node, List<ParseNode> nodes) throws SQLException {
+        return leaveCompoundNode(node, nodes, new CompoundNodeFactory() {
+            @Override
+            public ParseNode createNode(List<ParseNode> children) {
+                return NODE_FACTORY.add(children);
+            }
+        });
+    }
+    
+    @Override
+    public ParseNode visitLeave(MultiplyParseNode node, List<ParseNode> nodes) throws SQLException {
+        return leaveCompoundNode(node, nodes, new CompoundNodeFactory() {
+            @Override
+            public ParseNode createNode(List<ParseNode> children) {
+                return NODE_FACTORY.multiply(children);
+            }
+        });
+    }
+    
+    @Override
+    public ParseNode visitLeave(DivideParseNode node, List<ParseNode> nodes) throws SQLException {
+        return leaveCompoundNode(node, nodes, new CompoundNodeFactory() {
+            @Override
+            public ParseNode createNode(List<ParseNode> children) {
+                return NODE_FACTORY.divide(children);
+            }
+        });
+    }
+    
+    @Override
+    public ParseNode visitLeave(final FunctionParseNode node, List<ParseNode> nodes) throws SQLException {
+        return leaveCompoundNode(node, nodes, new CompoundNodeFactory() {
+            @Override
+            public ParseNode createNode(List<ParseNode> children) {
+                return NODE_FACTORY.function(node.getName(),children);
+            }
+        });
+    }
+    
+    @Override
+    public ParseNode visitLeave(CaseParseNode node, List<ParseNode> nodes) throws SQLException {
+        return leaveCompoundNode(node, nodes, new CompoundNodeFactory() {
+            @Override
+            public ParseNode createNode(List<ParseNode> children) {
+                return NODE_FACTORY.caseWhen(children);
+            }
+        });
+    }
+
+    @Override
+    public ParseNode visitLeave(final LikeParseNode node, List<ParseNode> nodes) throws SQLException {
+        return leaveCompoundNode(node, nodes, new CompoundNodeFactory() {
+            @Override
+            public ParseNode createNode(List<ParseNode> children) {
+                return NODE_FACTORY.like(children.get(0),children.get(1),node.isNegate());
+            }
+        });
+    }
+    
+    @Override
+    public ParseNode visitLeave(NotParseNode node, List<ParseNode> nodes) throws SQLException {
+        return leaveCompoundNode(node, nodes, new CompoundNodeFactory() {
+            @Override
+            public ParseNode createNode(List<ParseNode> children) {
+                return NODE_FACTORY.not(children.get(0));
+            }
+        });
+    }
+    
+    @Override
+    public ParseNode visitLeave(final CastParseNode node, List<ParseNode> nodes) throws SQLException {
+        return leaveCompoundNode(node, nodes, new CompoundNodeFactory() {
+            @Override
+            public ParseNode createNode(List<ParseNode> children) {
+                return NODE_FACTORY.cast(children.get(0), node.getDataType());
+            }
+        });
+    }
+    
+    @Override
+    public ParseNode visitLeave(final InListParseNode node, List<ParseNode> nodes) throws SQLException {
+        return leaveCompoundNode(node, nodes, new CompoundNodeFactory() {
+            @Override
+            public ParseNode createNode(List<ParseNode> children) {
+                return NODE_FACTORY.inList(children, node.isNegate());
+            }
+        });
+    }
+    
+    @Override
+    public ParseNode visitLeave(final IsNullParseNode node, List<ParseNode> nodes) throws SQLException {
+        return leaveCompoundNode(node, nodes, new CompoundNodeFactory() {
+            @Override
+            public ParseNode createNode(List<ParseNode> children) {
+                return NODE_FACTORY.isNull(children.get(0), node.isNegate());
+            }
+        });
+    }
+    
+    /**
+     * Rewrites expressions of the form (a, b, c) = (1, 2) as a = 1 and b = 2 and c is null
+     * as this is equivalent and already optimized
+     * @param lhs
+     * @param rhs
+     * @param andNodes
+     * @throws SQLException 
+     */
+    private void rewriteRowValueConstuctorEqualityComparison(ParseNode lhs, ParseNode rhs, List<ParseNode> andNodes) throws SQLException {
+        if (lhs instanceof RowValueConstructorParseNode && rhs instanceof RowValueConstructorParseNode) {
+            int i = 0;
+            for (; i < Math.min(lhs.getChildren().size(),rhs.getChildren().size()); i++) {
+                rewriteRowValueConstuctorEqualityComparison(lhs.getChildren().get(i), rhs.getChildren().get(i), andNodes);
+            }
+            for (; i < lhs.getChildren().size(); i++) {
+                rewriteRowValueConstuctorEqualityComparison(lhs.getChildren().get(i), null, andNodes);
+            }
+            for (; i < rhs.getChildren().size(); i++) {
+                rewriteRowValueConstuctorEqualityComparison(null, rhs.getChildren().get(i), andNodes);
+            }
+        } else if (lhs instanceof RowValueConstructorParseNode) {
+            rewriteRowValueConstuctorEqualityComparison(lhs.getChildren().get(0), rhs, andNodes);
+            for (int i = 1; i < lhs.getChildren().size(); i++) {
+                rewriteRowValueConstuctorEqualityComparison(lhs.getChildren().get(i), null, andNodes);
+            }
+        } else if (rhs instanceof RowValueConstructorParseNode) {
+            rewriteRowValueConstuctorEqualityComparison(lhs, rhs.getChildren().get(0), andNodes);
+            for (int i = 1; i < rhs.getChildren().size(); i++) {
+                rewriteRowValueConstuctorEqualityComparison(null, rhs.getChildren().get(i), andNodes);
+            }
+        } else if (lhs == null && rhs == null) { // null == null will end up making the query degenerate
+            andNodes.add(NODE_FACTORY.comparison(CompareOp.EQUAL, null, null).accept(this));
+        } else if (lhs == null) { // AND rhs IS NULL
+            andNodes.add(NODE_FACTORY.isNull(rhs, false).accept(this));
+        } else if (rhs == null) { // AND lhs IS NULL
+            andNodes.add(NODE_FACTORY.isNull(lhs, false).accept(this));
+        } else { // AND lhs = rhs
+            andNodes.add(NODE_FACTORY.comparison(CompareOp.EQUAL, lhs, rhs).accept(this));
+        }
+    }
+    
+    @Override
+    public ParseNode visitLeave(final ComparisonParseNode node, List<ParseNode> nodes) throws SQLException {
+        ParseNode normNode = leaveCompoundNode(node, nodes, new CompoundNodeFactory() {
+            @Override
+            public ParseNode createNode(List<ParseNode> children) {
+                return NODE_FACTORY.comparison(node.getFilterOp(), children.get(0), children.get(1));
+            }
+        });
+        
+        CompareOp op = node.getFilterOp();
+        if (op == CompareOp.EQUAL || op == CompareOp.NOT_EQUAL) {
+            // Rewrite row value constructor in = or != expression, as this is the same as if it was
+            // used in an equality expression for each individual part.
+            ParseNode lhs = normNode.getChildren().get(0);
+            ParseNode rhs = normNode.getChildren().get(1);
+            if (lhs instanceof RowValueConstructorParseNode || rhs instanceof RowValueConstructorParseNode) {
+                List<ParseNode> andNodes = Lists.newArrayListWithExpectedSize(Math.max(lhs.getChildren().size(), rhs.getChildren().size()));
+                rewriteRowValueConstuctorEqualityComparison(lhs,rhs,andNodes);
+                normNode = NODE_FACTORY.and(andNodes);
+                if (op == CompareOp.NOT_EQUAL) {
+                    normNode = NODE_FACTORY.not(normNode);
+                }
+            }
+        }
+        return normNode;
+    }
+    
+    @Override
+    public ParseNode visitLeave(final BetweenParseNode node, List<ParseNode> nodes) throws SQLException {
+        return leaveCompoundNode(node, nodes, new CompoundNodeFactory() {
+            @Override
+            public ParseNode createNode(List<ParseNode> children) {
+                if(node.isNegate()) {
+                    return NODE_FACTORY.not(NODE_FACTORY.and(children));
+                } else {
+                    return NODE_FACTORY.and(children);
+                }
+            }
+        });
+    }
+    
+    @Override
+    public ParseNode visit(ColumnParseNode node) throws SQLException {
+        // If we're resolving aliases and we have an unqualified ColumnParseNode,
+        // check if we find the name in our alias map.
+        if (aliasMap != null && node.getTableName() == null) {
+            ParseNode aliasedNode = aliasMap.get(node.getName());
+            // If we found something, then try to resolve it unless the two nodes are the same
+            if (aliasedNode != null && !node.equals(aliasedNode)) {
+                try {
+                    // If we're able to resolve it, that means we have a conflict
+                    resolver.resolveColumn(node.getSchemaName(), node.getTableName(), node.getName());
+                    throw new AmbiguousColumnException(node.getName());
+                } catch (ColumnNotFoundException e) {
+                    // Not able to resolve alias as a column name as well, so we use the alias
+                    return aliasedNode;
+                }
+            }
+        }
+        return node;
+    }
+
+    @Override
+    public ParseNode visit(LiteralParseNode node) throws SQLException {
+        return node;
+    }
+    
+    @Override
+    public ParseNode visit(BindParseNode node) throws SQLException {
+        return node;
+    }
+    
+    @Override
+    public ParseNode visit(WildcardParseNode node) throws SQLException {
+        return node;
+    }
+    
+    @Override
+    public ParseNode visit(FamilyWildcardParseNode node) throws SQLException {
+        return node;
+    }
+    
+    @Override
+    public List<ParseNode> newElementList(int size) {
+        nodeCount += size;
+        return new ArrayList<ParseNode>(size);
+    }
+    
+    @Override
+    public ParseNode visitLeave(StringConcatParseNode node, List<ParseNode> l) throws SQLException {
+        return node;
+    }
+
+    @Override
+    public void addElement(List<ParseNode> l, ParseNode element) {
+        nodeCount--;
+        if (element != null) {
+            l.add(element);
+        }
+    }
+
+    @Override
+    public ParseNode visitLeave(RowValueConstructorParseNode node, List<ParseNode> children) throws SQLException {
+        // Strip trailing nulls from rvc as they have no meaning
+        if (children.get(children.size()-1) == null) {
+            children = Lists.newArrayList(children);
+            do {
+                children.remove(children.size()-1);
+            } while (children.size() > 0 && children.get(children.size()-1) == null);
+            // If we're down to a single child, it's not a rvc anymore
+            if (children.size() == 0) {
+                return null;
+            }
+            if (children.size() == 1) {
+                return children.get(0);
+            }
+        }
+        // Flatten nested row value constructors, as this makes little sense and adds no information
+        List<ParseNode> flattenedChildren = children;
+        for (int i = 0; i < children.size(); i++) {
+            ParseNode child = children.get(i);
+            if (child instanceof RowValueConstructorParseNode) {
+                if (flattenedChildren == children) {
+                    flattenedChildren = Lists.newArrayListWithExpectedSize(children.size() + child.getChildren().size());
+                    flattenedChildren.addAll(children.subList(0, i));
+                }
+                flattenedChildren.addAll(child.getChildren());
+            }
+        }
+        
+        return leaveCompoundNode(node, flattenedChildren, new CompoundNodeFactory() {
+            @Override
+            public ParseNode createNode(List<ParseNode> children) {
+                return NODE_FACTORY.rowValueConstructor(children);
+            }
+        });
+    }
+
+	@Override
+	public ParseNode visit(SequenceValueParseNode node) throws SQLException {		
+		return node;
+	}
+
+	@Override
+	public ParseNode visitLeave(ArrayConstructorNode node, List<ParseNode> nodes) throws SQLException {
+	    return leaveCompoundNode(node, nodes, new CompoundNodeFactory() {
+            @Override
+            public ParseNode createNode(List<ParseNode> children) {
+                return NODE_FACTORY.upsertStmtArrayNode(children);
+            }
+        });
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeVisitor.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeVisitor.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeVisitor.java
new file mode 100644
index 0000000..f300173
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeVisitor.java
@@ -0,0 +1,108 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+import java.util.List;
+
+
+
+/**
+ * 
+ * Visitor for ParseNode in the node tree. Uses composite
+ * visitor pattern with enter/leave calls for any
+ * compound expression node. Only supported SQL constructs
+ * have visit methods.  Unsupported constructs fall through
+ * to {@link #visitEnter(CompoundParseNode)} for
+ * compound parse nodes and {@link #visit(ParseNode)}
+ * for terminal parse nodes.
+ * 
+ * @author jtaylor
+ * @since 0.1
+ */
+public interface ParseNodeVisitor<E> {
+    public List<E> newElementList(int size);
+    public void addElement(List<E> a, E element);
+    
+    public boolean visitEnter(LikeParseNode node) throws SQLException;
+    public E visitLeave(LikeParseNode node, List<E> l) throws SQLException;
+    
+    public boolean visitEnter(AndParseNode node) throws SQLException;
+    public E visitLeave(AndParseNode node, List<E> l) throws SQLException;
+    
+    public boolean visitEnter(OrParseNode node) throws SQLException;
+    public E visitLeave(OrParseNode node, List<E> l) throws SQLException;
+    
+    public boolean visitEnter(FunctionParseNode node) throws SQLException;
+    public E visitLeave(FunctionParseNode node, List<E> l) throws SQLException;
+    
+    public boolean visitEnter(ComparisonParseNode node) throws SQLException;
+    public E visitLeave(ComparisonParseNode node, List<E> l) throws SQLException;
+
+    public boolean visitEnter(CaseParseNode node) throws SQLException;
+    public E visitLeave(CaseParseNode node, List<E> l) throws SQLException;
+    
+    public boolean visitEnter(CompoundParseNode node) throws SQLException;
+    public E visitLeave(CompoundParseNode node, List<E> l) throws SQLException;
+    
+    public boolean visitEnter(AddParseNode node) throws SQLException;
+    public E visitLeave(AddParseNode node, List<E> l) throws SQLException;
+    
+    public boolean visitEnter(MultiplyParseNode node) throws SQLException;
+    public E visitLeave(MultiplyParseNode node, List<E> l) throws SQLException;
+    
+    public boolean visitEnter(DivideParseNode node) throws SQLException;
+    public E visitLeave(DivideParseNode node, List<E> l) throws SQLException;
+    
+    public boolean visitEnter(SubtractParseNode node) throws SQLException;
+    public E visitLeave(SubtractParseNode node, List<E> l) throws SQLException;
+    
+    public boolean visitEnter(NotParseNode node) throws SQLException;
+    public E visitLeave(NotParseNode node, List<E> l) throws SQLException;
+    
+    public boolean visitEnter(InListParseNode node) throws SQLException;
+    public E visitLeave(InListParseNode node, List<E> l) throws SQLException;
+    
+    public boolean visitEnter(IsNullParseNode node) throws SQLException;
+    public E visitLeave(IsNullParseNode node, List<E> l) throws SQLException;
+    
+    public E visit(ColumnParseNode node) throws SQLException;
+    public E visit(LiteralParseNode node) throws SQLException;
+    public E visit(BindParseNode node) throws SQLException;
+    public E visit(WildcardParseNode node) throws SQLException;  
+    public E visit(FamilyWildcardParseNode node) throws SQLException;  
+    public E visit(ParseNode node) throws SQLException;  
+    
+    public boolean visitEnter(StringConcatParseNode node) throws SQLException;
+    public E visitLeave(StringConcatParseNode node, List<E> l) throws SQLException;
+	
+    public boolean visitEnter(BetweenParseNode node) throws SQLException;
+    public E visitLeave(BetweenParseNode node, List<E> l) throws SQLException;
+    
+    public boolean visitEnter(CastParseNode node) throws SQLException;
+    public E visitLeave(CastParseNode node, List<E> l) throws SQLException;
+    
+    public boolean visitEnter(RowValueConstructorParseNode node) throws SQLException;
+    public E visitLeave(RowValueConstructorParseNode node, List<E> l) throws SQLException;
+    
+    public boolean visitEnter(ArrayConstructorNode node) throws SQLException;
+    public E visitLeave(ArrayConstructorNode node, List<E> l) throws SQLException;
+    public E visit(SequenceValueParseNode node) throws SQLException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/PrimaryKeyConstraint.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/PrimaryKeyConstraint.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/PrimaryKeyConstraint.java
new file mode 100644
index 0000000..97c99cb
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/PrimaryKeyConstraint.java
@@ -0,0 +1,69 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+
+import org.apache.hadoop.hbase.util.Pair;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Maps;
+import org.apache.phoenix.schema.ColumnModifier;
+
+public class PrimaryKeyConstraint extends NamedNode {
+    public static final PrimaryKeyConstraint EMPTY = new PrimaryKeyConstraint(null, Collections.<Pair<ColumnName, ColumnModifier>>emptyList());
+
+    private final List<Pair<ColumnName, ColumnModifier>> columns;
+    private final HashMap<ColumnName, Pair<ColumnName, ColumnModifier>> columnNameToModifier;
+    
+    PrimaryKeyConstraint(String name, List<Pair<ColumnName, ColumnModifier>> columns) {
+        super(name);
+        this.columns = columns == null ? Collections.<Pair<ColumnName, ColumnModifier>>emptyList() : ImmutableList.copyOf(columns);
+        this.columnNameToModifier = Maps.newHashMapWithExpectedSize(this.columns.size());
+        for (Pair<ColumnName, ColumnModifier> p : this.columns) {
+            this.columnNameToModifier.put(p.getFirst(), p);
+        }
+    }
+
+    public List<Pair<ColumnName, ColumnModifier>> getColumnNames() {
+        return columns;
+    }
+    
+    public Pair<ColumnName, ColumnModifier> getColumn(ColumnName columnName) {
+    	return columnNameToModifier.get(columnName);
+    }
+    
+    public boolean contains(ColumnName columnName) {
+        return columnNameToModifier.containsKey(columnName);
+    }
+    
+    @Override
+    public int hashCode() {
+        return super.hashCode();
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        return super.equals(obj);
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/PropertyName.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/PropertyName.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/PropertyName.java
new file mode 100644
index 0000000..167b323
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/PropertyName.java
@@ -0,0 +1,25 @@
+package org.apache.phoenix.parse;
+
+import org.apache.phoenix.util.SchemaUtil;
+
+public class PropertyName {
+    private final NamedNode familyName;
+    private final String propertyName;
+    
+    PropertyName(String familyName, String propertyName) {
+        this.familyName = familyName == null ? null : new NamedNode(familyName);
+        this.propertyName = SchemaUtil.normalizeIdentifier(propertyName);;
+    }
+
+    PropertyName(String columnName) {
+        this(null, columnName);
+    }
+
+    public String getFamilyName() {
+        return familyName == null ? "" : familyName.getName();
+    }
+
+    public String getPropertyName() {
+        return propertyName;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/RoundParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/RoundParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/RoundParseNode.java
new file mode 100644
index 0000000..b7e2c0c
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/RoundParseNode.java
@@ -0,0 +1,83 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.function.RoundDateExpression;
+import org.apache.phoenix.expression.function.RoundDecimalExpression;
+import org.apache.phoenix.expression.function.RoundFunction;
+import org.apache.phoenix.expression.function.RoundTimestampExpression;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.TypeMismatchException;
+
+/**
+ * 
+ * Parse node corresponding to {@link RoundFunction}. 
+ * It also acts as a factory for creating the right kind of
+ * round expression according to the data type of the 
+ * first child.
+ *
+ * @author samarth.jain
+ * @since 3.0.0
+ */
+public class RoundParseNode extends FunctionParseNode {
+
+    RoundParseNode(String name, List<ParseNode> children, BuiltInFunctionInfo info) {
+        super(name, children, info);
+    }
+
+    @Override
+    public Expression create(List<Expression> children, StatementContext context) throws SQLException {
+        return getRoundExpression(children);
+    }
+
+    public static Expression getRoundExpression(List<Expression> children) throws SQLException {
+        final Expression firstChild = children.get(0);
+        final PDataType firstChildDataType = firstChild.getDataType();
+        
+        if(firstChildDataType.isCoercibleTo(PDataType.DATE)) {
+            return RoundDateExpression.create(children); // FIXME: remove cast
+        } else if (firstChildDataType.isCoercibleTo(PDataType.TIMESTAMP)) {
+            return RoundTimestampExpression.create(children); // FIXME: remove cast
+        } else if(firstChildDataType.isCoercibleTo(PDataType.DECIMAL)) {
+            return new RoundDecimalExpression(children);
+        } else {
+            throw TypeMismatchException.newException(firstChildDataType, "1");
+        }
+    }
+    
+    /**
+     * When rounding off decimals, user need not specify the scale. In such cases, 
+     * we need to prevent the function from getting evaluated as null. This is really
+     * a hack. A better way would have been if {@link org.apache.phoenix.parse.FunctionParseNode.BuiltInFunctionInfo} provided a 
+     * way of associating default values for each permissible data type.
+     * Something like: @ Argument(allowedTypes={PDataType.VARCHAR, PDataType.INTEGER}, defaultValues = {"null", "1"} isConstant=true)
+     * Till then, this will have to do.
+     */
+    @Override
+    public boolean evalToNullIfParamIsNull(StatementContext context, int index) throws SQLException {
+        return index == 0;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/RowValueConstructorParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/RowValueConstructorParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/RowValueConstructorParseNode.java
new file mode 100644
index 0000000..e35646c
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/RowValueConstructorParseNode.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * 
+ * Node representing a row value constructor in SQL.  
+ *
+ * @author samarth.jain
+ * @since 0.1
+ */
+public class RowValueConstructorParseNode extends CompoundParseNode {
+    
+    public RowValueConstructorParseNode(List<ParseNode> l) {
+        super(l);
+    }
+
+    @Override
+    public <T> T accept(ParseNodeVisitor<T> visitor) throws SQLException {
+        List<T> l = Collections.emptyList();
+        if (visitor.visitEnter(this)) {
+            l = acceptChildren(visitor);
+        }
+        return visitor.visitLeave(this, l);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/SQLParser.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/SQLParser.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/SQLParser.java
new file mode 100644
index 0000000..4a0139e
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/SQLParser.java
@@ -0,0 +1,198 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.io.IOException;
+import java.io.Reader;
+import java.io.StringReader;
+import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
+
+import org.antlr.runtime.ANTLRReaderStream;
+import org.antlr.runtime.CharStream;
+import org.antlr.runtime.CommonTokenStream;
+import org.antlr.runtime.RecognitionException;
+
+import org.apache.phoenix.exception.PhoenixParserException;
+
+/**
+ * 
+ * SQL Parser for Phoenix
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class SQLParser {
+    private static final ParseNodeFactory DEFAULT_NODE_FACTORY = new ParseNodeFactory();
+
+    private final PhoenixSQLParser parser;
+
+    public static ParseNode parseCondition(String expression) throws SQLException {
+        if (expression == null) return null;
+        SQLParser parser = new SQLParser(expression);
+        return parser.parseCondition();
+    }
+    
+    public SQLParser(String query) {
+        this(query,DEFAULT_NODE_FACTORY);
+    }
+
+    public SQLParser(String query, ParseNodeFactory factory) {
+        PhoenixSQLLexer lexer;
+        try {
+            lexer = new PhoenixSQLLexer(new CaseInsensitiveReaderStream(new StringReader(query)));
+        } catch (IOException e) {
+            throw new RuntimeException(e); // Impossible
+        }
+        CommonTokenStream cts = new CommonTokenStream(lexer);
+        parser = new PhoenixSQLParser(cts);
+        parser.setParseNodeFactory(factory);
+    }
+
+    public SQLParser(Reader queryReader, ParseNodeFactory factory) throws IOException {
+        PhoenixSQLLexer lexer = new PhoenixSQLLexer(new CaseInsensitiveReaderStream(queryReader));
+        CommonTokenStream cts = new CommonTokenStream(lexer);
+        parser = new PhoenixSQLParser(cts);
+        parser.setParseNodeFactory(factory);
+    }
+
+    public SQLParser(Reader queryReader) throws IOException {
+        PhoenixSQLLexer lexer = new PhoenixSQLLexer(new CaseInsensitiveReaderStream(queryReader));
+        CommonTokenStream cts = new CommonTokenStream(lexer);
+        parser = new PhoenixSQLParser(cts);
+        parser.setParseNodeFactory(DEFAULT_NODE_FACTORY);
+    }
+
+    /**
+     * Parses the input as a series of semicolon-terminated SQL statements.
+     * @throws SQLException 
+     */
+    public BindableStatement nextStatement(ParseNodeFactory factory) throws SQLException {
+        try {
+            parser.resetBindCount();
+            parser.setParseNodeFactory(factory);
+            BindableStatement statement = parser.nextStatement();
+            return statement;
+        } catch (RecognitionException e) {
+            throw PhoenixParserException.newException(e, parser.getTokenNames());
+        } catch (UnsupportedOperationException e) {
+            throw new SQLFeatureNotSupportedException(e);
+        } catch (RuntimeException e) {
+            if (e.getCause() instanceof SQLException) {
+                throw (SQLException) e.getCause();
+            }
+            throw PhoenixParserException.newException(e, parser.getTokenNames());
+        }
+    }
+
+    /**
+     * Parses the input as a SQL select or upsert statement.
+     * @throws SQLException 
+     */
+    public BindableStatement parseStatement() throws SQLException {
+        try {
+            BindableStatement statement = parser.statement();
+            return statement;
+        } catch (RecognitionException e) {
+            throw PhoenixParserException.newException(e, parser.getTokenNames());
+        } catch (UnsupportedOperationException e) {
+            throw new SQLFeatureNotSupportedException(e);
+        } catch (RuntimeException e) {
+            if (e.getCause() instanceof SQLException) {
+                throw (SQLException) e.getCause();
+            }
+            throw PhoenixParserException.newException(e, parser.getTokenNames());
+        }
+    }
+
+    /**
+     * Parses the input as a SQL select statement.
+     * Used only in tests
+     * @throws SQLException 
+     */
+    public SelectStatement parseQuery() throws SQLException {
+        try {
+            SelectStatement statement = parser.query();
+            return statement;
+        } catch (RecognitionException e) {
+            throw PhoenixParserException.newException(e, parser.getTokenNames());
+        } catch (RuntimeException e) {
+            if (e.getCause() instanceof SQLException) {
+                throw (SQLException) e.getCause();
+            }
+            throw PhoenixParserException.newException(e, parser.getTokenNames());
+        }
+    }
+
+    /**
+     * Parses the input as a SQL select statement.
+     * Used only in tests
+     * @throws SQLException 
+     */
+    public ParseNode parseCondition() throws SQLException {
+        try {
+            ParseNode node = parser.condition();
+            return node;
+        } catch (RecognitionException e) {
+            throw PhoenixParserException.newException(e, parser.getTokenNames());
+        } catch (RuntimeException e) {
+            if (e.getCause() instanceof SQLException) {
+                throw (SQLException) e.getCause();
+            }
+            throw PhoenixParserException.newException(e, parser.getTokenNames());
+        }
+    }
+
+    /**
+     * Parses the input as a SQL literal
+     * @throws SQLException 
+     */
+    public LiteralParseNode parseLiteral() throws SQLException {
+        try {
+            LiteralParseNode literalNode = parser.literal();
+            return literalNode;
+        } catch (RecognitionException e) {
+            throw PhoenixParserException.newException(e, parser.getTokenNames());
+        } catch (RuntimeException e) {
+            if (e.getCause() instanceof SQLException) {
+                throw (SQLException) e.getCause();
+            }
+            throw PhoenixParserException.newException(e, parser.getTokenNames());
+        }
+    }
+
+    private static class CaseInsensitiveReaderStream extends ANTLRReaderStream {
+        CaseInsensitiveReaderStream(Reader script) throws IOException {
+            super(script);
+        }
+
+        @Override
+        public int LA(int i) {
+            if (i == 0) { return 0; // undefined
+            }
+            if (i < 0) {
+                i++; // e.g., translate LA(-1) to use offset 0
+            }
+
+            if ((p + i - 1) >= n) { return CharStream.EOF; }
+            return Character.toLowerCase(data[p + i - 1]);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/SelectStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/SelectStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/SelectStatement.java
new file mode 100644
index 0000000..1f2be75
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/SelectStatement.java
@@ -0,0 +1,179 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.phoenix.expression.function.CountAggregateFunction;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunctionInfo;
+
+/**
+ * 
+ * Top level node representing a SQL statement
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class SelectStatement implements FilterableStatement {
+    public static final SelectStatement SELECT_ONE =
+            new SelectStatement(
+                    Collections.<TableNode>emptyList(), null, false, 
+                    Collections.<AliasedNode>singletonList(new AliasedNode(null,new LiteralParseNode(1))),
+                    null, Collections.<ParseNode>emptyList(),
+                    null, Collections.<OrderByNode>emptyList(),
+                    null, 0, false);
+    public static final SelectStatement COUNT_ONE =
+            new SelectStatement(
+                    Collections.<TableNode>emptyList(), null, false,
+                    Collections.<AliasedNode>singletonList(
+                    new AliasedNode(null, 
+                        new AggregateFunctionParseNode(
+                                CountAggregateFunction.NORMALIZED_NAME, 
+                                LiteralParseNode.STAR, 
+                                new BuiltInFunctionInfo(CountAggregateFunction.class, CountAggregateFunction.class.getAnnotation(BuiltInFunction.class))))),
+                    null, Collections.<ParseNode>emptyList(), 
+                    null, Collections.<OrderByNode>emptyList(), 
+                    null, 0, true);
+    public static SelectStatement create(SelectStatement select, HintNode hint) {
+        if (select.getHint() == hint || hint.isEmpty()) {
+            return select;
+        }
+        return new SelectStatement(select.getFrom(), hint, select.isDistinct(), 
+                select.getSelect(), select.getWhere(), select.getGroupBy(), select.getHaving(), 
+                select.getOrderBy(), select.getLimit(), select.getBindCount(), select.isAggregate());
+    }
+    
+    public SelectStatement combine(ParseNode where) {
+        if (where == null) {
+            return this;
+        }
+        if (this.getWhere() != null) {
+            where = new AndParseNode(Arrays.asList(this.getWhere(), where));
+        }
+        return new SelectStatement(this.getFrom(), this.getHint(), this.isDistinct(), 
+                this.getSelect(), where, this.getGroupBy(), this.getHaving(), 
+                this.getOrderBy(), this.getLimit(), this.getBindCount(), this.isAggregate());
+    }
+    
+    public static SelectStatement create(SelectStatement select, List<AliasedNode> selects) {
+        return new SelectStatement(select.getFrom(), select.getHint(), select.isDistinct(), 
+                selects, select.getWhere(), select.getGroupBy(), select.getHaving(), 
+                select.getOrderBy(), select.getLimit(), select.getBindCount(), select.isAggregate());
+    }
+    
+    private final List<TableNode> fromTable;
+    private final HintNode hint;
+    private final boolean isDistinct;
+    private final List<AliasedNode> select;
+    private final ParseNode where;
+    private final List<ParseNode> groupBy;
+    private final ParseNode having;
+    private final List<OrderByNode> orderBy;
+    private final LimitNode limit;
+    private final int bindCount;
+    private final boolean isAggregate;
+    
+    // Count constant expressions
+    private static int countConstants(List<ParseNode> nodes) {
+        int count = 0;
+        for (int i = 0; i < nodes.size(); i++) {
+            if (nodes.get(i).isStateless()) {
+                count++;
+            }
+        }
+        return count;
+    }
+    
+    protected SelectStatement(List<? extends TableNode> from, HintNode hint, boolean isDistinct, List<AliasedNode> select, ParseNode where, List<ParseNode> groupBy, ParseNode having, List<OrderByNode> orderBy, LimitNode limit, int bindCount, boolean isAggregate) {
+        this.fromTable = Collections.unmodifiableList(from);
+        this.hint = hint == null ? HintNode.EMPTY_HINT_NODE : hint;
+        this.isDistinct = isDistinct;
+        this.select = Collections.unmodifiableList(select);
+        this.where = where;
+        this.groupBy = Collections.unmodifiableList(groupBy);
+        this.having = having;
+        this.orderBy = Collections.unmodifiableList(orderBy);
+        this.limit = limit;
+        this.bindCount = bindCount;
+        this.isAggregate = isAggregate || groupBy.size() != countConstants(groupBy) || this.having != null;
+    }
+    
+    @Override
+    public boolean isDistinct() {
+        return isDistinct;
+    }
+    
+    @Override
+    public LimitNode getLimit() {
+        return limit;
+    }
+    
+    @Override
+    public int getBindCount() {
+        return bindCount;
+    }
+    
+    public List<TableNode> getFrom() {
+        return fromTable;
+    }
+    
+    @Override
+    public HintNode getHint() {
+        return hint;
+    }
+    
+    public List<AliasedNode> getSelect() {
+        return select;
+    }
+    /**
+     * Gets the where condition, or null if none.
+     */
+    @Override
+    public ParseNode getWhere() {
+        return where;
+    }
+    
+    /**
+     * Gets the group-by, containing at least 1 element, or null, if none.
+     */
+    public List<ParseNode> getGroupBy() {
+        return groupBy;
+    }
+    
+    public ParseNode getHaving() {
+        return having;
+    }
+    
+    /**
+     * Gets the order-by, containing at least 1 element, or null, if none.
+     */
+    @Override
+    public List<OrderByNode> getOrderBy() {
+        return orderBy;
+    }
+
+    @Override
+    public boolean isAggregate() {
+        return isAggregate;
+    }
+}


[03/51] [partial] Initial commit of master branch from github

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/site/language_reference_source/index.html
----------------------------------------------------------------------
diff --git a/phoenix-core/src/site/language_reference_source/index.html b/phoenix-core/src/site/language_reference_source/index.html
new file mode 100644
index 0000000..04bcec6
--- /dev/null
+++ b/phoenix-core/src/site/language_reference_source/index.html
@@ -0,0 +1,947 @@
+<!DOCTYPE html PUBLIC "-//W3C//DTD XHTML 1.0 Strict//EN" "http://www.w3.org/TR/xhtml1/DTD/xhtml1-strict.dtd">
+
+<html xmlns="http://www.w3.org/1999/xhtml" lang="en" xml:lang="en">
+<head><meta http-equiv="Content-Type" content="text/html;charset=utf-8" /><title>
+SQL Grammar
+</title><link rel="stylesheet" type="text/css" href="stylesheet.css" />
+<!-- [search] { -->
+</head>
+<body>
+
+<!-- } -->
+<h3 id="grammar">Commands</h3>
+<!-- syntax-start
+<p class="notranslate">
+
+    <a href="#select">SELECT</a><br />
+
+    <a href="#upsert_values">UPSERT VALUES</a><br />
+
+    <a href="#upsert_select">UPSERT SELECT</a><br />
+
+    <a href="#delete">DELETE</a><br />
+
+    <a href="#create">CREATE</a><br />
+
+    <a href="#drop">DROP</a><br />
+
+    <a href="#alter_table">ALTER TABLE</a><br />
+
+    <a href="#create_index">CREATE INDEX</a><br />
+
+    <a href="#drop_index">DROP INDEX</a><br />
+
+    <a href="#alter_index">ALTER INDEX</a><br />
+
+    <a href="#explain">EXPLAIN</a><br />
+</p>
+syntax-end -->
+<!-- railroad-start -->
+<table class="notranslate index">
+    <tr>
+        <td class="index">
+            
+                <a href="#select" >SELECT</a><br />
+            
+                <a href="#upsert_values" >UPSERT VALUES</a><br />
+            
+                <a href="#upsert_select" >UPSERT SELECT</a><br />
+            
+                <a href="#delete" >DELETE</a><br />
+                    </td><td class="index">
+            
+                <a href="#create" >CREATE</a><br />
+            
+                <a href="#drop" >DROP</a><br />
+            
+                <a href="#alter_table" >ALTER TABLE</a><br />
+            
+                <a href="#create_index" >CREATE INDEX</a><br />
+                    </td><td class="index">
+            
+                <a href="#drop_index" >DROP INDEX</a><br />
+            
+                <a href="#alter_index" >ALTER INDEX</a><br />
+            
+                <a href="#explain" >EXPLAIN</a><br />
+                    </td>
+    </tr>
+</table>
+<!-- railroad-end -->
+
+<h3>Other Grammar</h3>
+<!-- syntax-start
+<p class="notranslate">
+
+    <a href="#constraint" >Constraint</a><br />
+
+    <a href="#options" >Options</a><br />
+
+    <a href="#hint" >Hint</a><br />
+
+    <a href="#column_def" >Column Def</a><br />
+
+    <a href="#table_ref" >Table Ref</a><br />
+
+    <a href="#column_ref" >Column Ref</a><br />
+
+    <a href="#select_expression" >Select Expression</a><br />
+
+    <a href="#split_point" >Split Point</a><br />
+
+    <a href="#table_expression" >Table Expression</a><br />
+
+    <a href="#order" >Order</a><br />
+
+    <a href="#expression" >Expression</a><br />
+
+    <a href="#and_condition" >And Condition</a><br />
+
+    <a href="#condition" >Condition</a><br />
+
+    <a href="#compare" >Compare</a><br />
+
+    <a href="#operand" >Operand</a><br />
+
+    <a href="#summand" >Summand</a><br />
+
+    <a href="#factor" >Factor</a><br />
+
+    <a href="#term" >Term</a><br />
+
+    <a href="#row_value_constructor" >Row Value Constructor</a><br />
+
+    <a href="#bind_parameter" >Bind Parameter</a><br />
+
+    <a href="#value" >Value</a><br />
+
+    <a href="#case" >Case</a><br />
+
+    <a href="#case_when" >Case When</a><br />
+
+    <a href="#name" >Name</a><br />
+
+    <a href="#quoted_name" >Quoted Name</a><br />
+
+    <a href="#alias" >Alias</a><br />
+
+    <a href="#null" >Null</a><br />
+
+    <a href="#data_type" >Data Type</a><br />
+
+    <a href="#string" >String</a><br />
+
+    <a href="#boolean" >Boolean</a><br />
+
+    <a href="#numeric" >Numeric</a><br />
+
+    <a href="#int" >Int</a><br />
+
+    <a href="#long" >Long</a><br />
+
+    <a href="#decimal" >Decimal</a><br />
+
+    <a href="#number" >Number</a><br />
+
+    <a href="#comments" >Comments</a><br />
+</p>
+syntax-end -->
+<!-- railroad-start -->
+<table class="notranslate index">
+    <tr>
+        <td class="index">
+            
+                <a href="#constraint" >Constraint</a><br />
+            
+                <a href="#options" >Options</a><br />
+            
+                <a href="#hint" >Hint</a><br />
+            
+                <a href="#column_def" >Column Def</a><br />
+            
+                <a href="#table_ref" >Table Ref</a><br />
+            
+                <a href="#column_ref" >Column Ref</a><br />
+            
+                <a href="#select_expression" >Select Expression</a><br />
+            
+                <a href="#split_point" >Split Point</a><br />
+            
+                <a href="#table_expression" >Table Expression</a><br />
+            
+                <a href="#order" >Order</a><br />
+            
+                <a href="#expression" >Expression</a><br />
+            
+                <a href="#and_condition" >And Condition</a><br />
+                    </td><td class="index">
+            
+                <a href="#condition" >Condition</a><br />
+            
+                <a href="#compare" >Compare</a><br />
+            
+                <a href="#operand" >Operand</a><br />
+            
+                <a href="#summand" >Summand</a><br />
+            
+                <a href="#factor" >Factor</a><br />
+            
+                <a href="#term" >Term</a><br />
+            
+                <a href="#row_value_constructor" >Row Value Constructor</a><br />
+            
+                <a href="#bind_parameter" >Bind Parameter</a><br />
+            
+                <a href="#value" >Value</a><br />
+            
+                <a href="#case" >Case</a><br />
+            
+                <a href="#case_when" >Case When</a><br />
+            
+                <a href="#name" >Name</a><br />
+                    </td><td class="index">
+            
+                <a href="#quoted_name" >Quoted Name</a><br />
+            
+                <a href="#alias" >Alias</a><br />
+            
+                <a href="#null" >Null</a><br />
+            
+                <a href="#data_type" >Data Type</a><br />
+            
+                <a href="#string" >String</a><br />
+            
+                <a href="#boolean" >Boolean</a><br />
+            
+                <a href="#numeric" >Numeric</a><br />
+            
+                <a href="#int" >Int</a><br />
+            
+                <a href="#long" >Long</a><br />
+            
+                <a href="#decimal" >Decimal</a><br />
+            
+                <a href="#number" >Number</a><br />
+            
+                <a href="#comments" >Comments</a><br />
+                    </td>
+    </tr>
+</table>
+<!-- railroad-end -->
+
+
+<h3 id="select" class="notranslate">SELECT</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+SELECT [/*+ <a href="index.html#hint">hint</a> */] [DISTINCT | ALL] <a href="index.html#select_expression">selectExpression</a> [,...]
+FROM <a href="index.html#table_expression">tableExpression</a> [( <a href="index.html#column_def">columnDef</a> [,...] )] [ WHERE <a href="index.html#expression">expression</a> ]
+[ GROUP BY <a href="index.html#expression">expression</a> [,...] ] [ HAVING <a href="index.html#expression">expression</a> ]
+[ ORDER BY <a href="index.html#order">order</a> [,...] ] [ LIMIT {<a href="index.html#bind_parameter">bindParameter</a> | <a href="index.html#number">number</a>} ]
+</pre>
+<div name="railroad">
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c">SELECT</code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">/ * + <a href="index.html#hint">hint</a> * /</code></td></tr></table></td><td class="le"></td></tr></table></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d"><code class="c">DISTINCT</code></td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c">ALL</code></td><td class="le"></td></tr></table></td><td class="le"></td></tr></table></td><td class="d"><code class="c"><a href="index.ht
 ml#select_expression">selectExpression</a></code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c">, ...</code></td><td class="le"></td></tr></table></td></tr></table><br /><table class="railroad"><tr class="railroad"><td class="d"><code class="c">FROM <a href="index.html#table_expression">tableExpression</a></code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">( <a href="index.html#column_def">columnDef</a></code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c">, 
 ...</code></td><td class="le"></td></tr></table></td><td class="d"><code class="c">)</code></td></tr></table></td><td class="le"></td></tr></table></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">WHERE <a href="index.html#expression">expression</a></code></td></tr></table></td><td class="le"></td></tr></table></td></tr></table><br /><table class="railroad"><tr class="railroad"><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">GROUP BY <a href="index.html#expression">expression</a></code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></
 td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c">, ...</code></td><td class="le"></td></tr></table></td></tr></table></td><td class="le"></td></tr></table></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">HAVING <a href="index.html#expression">expression</a></code></td></tr></table></td><td class="le"></td></tr></table></td></tr></table><br /><table class="railroad"><tr class="railroad"><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">ORDER BY <a href="index.html#order">order</a></code></td><t
 d class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c">, ...</code></td><td class="le"></td></tr></table></td></tr></table></td><td class="le"></td></tr></table></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">LIMIT</code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d"><code class="c"><a href="index.html#bind_parameter">bindParameter</a></code></td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c"><a href="index.html#number">number</a></code></td><td class="le"></td></tr></table></td></tr></table></td><td class="le"></td></tr></table></t
 d></tr></table>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+SELECT [/*+ <a href="index.html#hint">hint</a> */] [DISTINCT | ALL] <a href="index.html#select_expression">selectExpression</a> [,...]
+FROM <a href="index.html#table_expression">tableExpression</a> [( <a href="index.html#column_def">columnDef</a> [,...] )] [ WHERE <a href="index.html#expression">expression</a> ]
+[ GROUP BY <a href="index.html#expression">expression</a> [,...] ] [ HAVING <a href="index.html#expression">expression</a> ]
+[ ORDER BY <a href="index.html#order">order</a> [,...] ] [ LIMIT {<a href="index.html#bind_parameter">bindParameter</a> | <a href="index.html#number">number</a>} ]
+</pre>
+syntax-end -->
+<p>Selects data from a table. <code>DISTINCT</code> filters out duplicate results while <code>ALL</code>, the default, includes all results. <code>FROM</code> identifies the table being queried (single table only currently - no joins or derived tables yet). Dynamic columns not declared at create time may be defined in parenthesis after the table name and then used in the query. <code>GROUP BY</code> groups the the result by the given expression(s). <code>HAVING</code> filter rows after grouping. <code>ORDER BY</code> sorts the result by the given column(s) or expression(s) and is only allowed for aggregate queries or queries with a <code>LIMIT</code> clause. <code>LIMIT</code> limits the number of rows returned by the query with no limit applied if specified as null or less than zero. The <code>LIMIT</code> clause is executed after the <code>ORDER BY</code> clause to support TopN type queries. An optional hint overrides the default query plan.</p>
+<p>Example:</p>
+<p class="notranslate">
+SELECT * FROM TEST;<br />SELECT a.* FROM TEST;<br />SELECT DISTINCT NAME FROM TEST;<br />SELECT ID, COUNT(1) FROM TEST GROUP BY ID;<br />SELECT NAME, SUM(VAL) FROM TEST GROUP BY NAME HAVING COUNT(1) &gt; 2;<br />SELECT &#39;ID&#39; COL, MAX(ID) AS MAX FROM TEST;<br />SELECT * FROM TEST LIMIT 1000;</p>
+
+<h3 id="upsert_values" class="notranslate">UPSERT VALUES</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+UPSERT INTO <a href="index.html#name">tableName</a> [( { <a href="index.html#column_ref">columnRef</a> | <a href="index.html#column_def">columnDef</a> } [,...] )] VALUES ( <a href="index.html#term">constantTerm</a> [,...] )
+</pre>
+<div name="railroad">
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c">UPSERT INTO <a href="index.html#name">tableName</a></code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">(</code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d"><code class="c"><a href="index.html#column_ref">columnRef</a></code></td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c"><a href="index.html#column_def">columnDef</a></code></td><td class="le"></td></tr></table></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c">, ...</code></td><td class="le
 "></td></tr></table></td><td class="d"><code class="c">)</code></td></tr></table></td><td class="le"></td></tr></table></td><td class="d"><code class="c">VALUES ( <a href="index.html#term">constantTerm</a></code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c">, ...</code></td><td class="le"></td></tr></table></td><td class="d"><code class="c">)</code></td></tr></table>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+UPSERT INTO <a href="index.html#name">tableName</a> [( { <a href="index.html#column_ref">columnRef</a> | <a href="index.html#column_def">columnDef</a> } [,...] )] VALUES ( <a href="index.html#term">constantTerm</a> [,...] )
+</pre>
+syntax-end -->
+<p>Inserts if not present and updates otherwise the value in the table. The list of columns is optional and if not present, the values will map to the column in the order they are declared in the schema. The values must evaluate to constants.</p>
+<p>Example:</p>
+<p class="notranslate">
+UPSERT INTO TEST VALUES(&#39;foo&#39;,&#39;bar&#39;,3);<br />UPSERT INTO TEST(NAME,ID) VALUES(&#39;foo&#39;,123);</p>
+
+<h3 id="upsert_select" class="notranslate">UPSERT SELECT</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+UPSERT [/*+ <a href="index.html#hint">hint</a> */] INTO <a href="index.html#name">tableName</a> [( { <a href="index.html#column_ref">columnRef</a> | <a href="index.html#column_def">columnDef</a> } [,...] )] <a href="index.html#select">select</a>
+</pre>
+<div name="railroad">
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c">UPSERT</code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">/ * + <a href="index.html#hint">hint</a> * /</code></td></tr></table></td><td class="le"></td></tr></table></td><td class="d"><code class="c">INTO <a href="index.html#name">tableName</a></code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">(</code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d"><code class="c"><a href="index.html#column_ref">columnRef</a></code></td><td class
 ="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c"><a href="index.html#column_def">columnDef</a></code></td><td class="le"></td></tr></table></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c">, ...</code></td><td class="le"></td></tr></table></td><td class="d"><code class="c">)</code></td></tr></table></td><td class="le"></td></tr></table></td><td class="d"><code class="c"><a href="index.html#select">select</a></code></td></tr></table>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+UPSERT [/*+ <a href="index.html#hint">hint</a> */] INTO <a href="index.html#name">tableName</a> [( { <a href="index.html#column_ref">columnRef</a> | <a href="index.html#column_def">columnDef</a> } [,...] )] <a href="index.html#select">select</a>
+</pre>
+syntax-end -->
+<p>Inserts if not present and updates otherwise rows in the table based on the results of running another query. The values are set based on their matching position between the source and target tables. The list of columns is optional and if not present will map to the column in the order they are declared in the schema. If auto commit is on, and both a) the target table matches the source table, and b) the select performs no aggregation, then the population of the target table will be done completely on the server-side (with constraint violations logged, but otherwise ignored). Otherwise, data is buffered on the client and, if auto commit is on, committed in row batches as specified by the UpsertBatchSize connection property (or the phoenix.mutate.upsertBatchSize <code>HBase</code> config property which defaults to 10000 rows)</p>
+<p>Example:</p>
+<p class="notranslate">
+UPSERT INTO test.targetTable(col1, col2) SELECT col3, col4 FROM test.sourceTable WHERE col5 &lt; 100<br />UPSERT INTO foo SELECT * FROM bar;</p>
+
+<h3 id="delete" class="notranslate">DELETE</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+DELETE [/*+ <a href="index.html#hint">hint</a> */] FROM <a href="index.html#name">tableName</a> [ WHERE <a href="index.html#expression">expression</a> ]
+[ ORDER BY <a href="index.html#order">order</a> [,...] ] [ LIMIT {<a href="index.html#bind_parameter">bindParameter</a> | <a href="index.html#number">number</a>} ]
+</pre>
+<div name="railroad">
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c">DELETE</code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">/ * + <a href="index.html#hint">hint</a> * /</code></td></tr></table></td><td class="le"></td></tr></table></td><td class="d"><code class="c">FROM <a href="index.html#name">tableName</a></code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">WHERE <a href="index.html#expression">expression</a></code></td></tr></table></td><td class="le"></td></tr></table></td></tr></table><br /><table class="railroad"><tr class="railroad"><td 
 class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">ORDER BY <a href="index.html#order">order</a></code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c">, ...</code></td><td class="le"></td></tr></table></td></tr></table></td><td class="le"></td></tr></table></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">LIMIT</code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d"><code
  class="c"><a href="index.html#bind_parameter">bindParameter</a></code></td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c"><a href="index.html#number">number</a></code></td><td class="le"></td></tr></table></td></tr></table></td><td class="le"></td></tr></table></td></tr></table>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+DELETE [/*+ <a href="index.html#hint">hint</a> */] FROM <a href="index.html#name">tableName</a> [ WHERE <a href="index.html#expression">expression</a> ]
+[ ORDER BY <a href="index.html#order">order</a> [,...] ] [ LIMIT {<a href="index.html#bind_parameter">bindParameter</a> | <a href="index.html#number">number</a>} ]
+</pre>
+syntax-end -->
+<p>Deletes the rows selected by the where clause. If auto commit is on, the deletion is performed completely server-side.</p>
+<p>Example:</p>
+<p class="notranslate">
+DELETE FROM TEST;<br />DELETE FROM TEST WHERE ID=123;<br />DELETE FROM TEST WHERE NAME LIKE &#39;foo%&#39;;</p>
+
+<h3 id="create" class="notranslate">CREATE</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+CREATE { TABLE | VIEW } [IF NOT EXISTS] <a href="index.html#table_ref">tableRef</a>
+( <a href="index.html#column_def">columnDef</a> [,...] [<a href="index.html#constraint">constraint</a>] )
+[<a href="index.html#options">tableOptions</a>] [ SPLIT ON ( <a href="index.html#split_point">splitPoint</a> [,...] ) ]
+</pre>
+<div name="railroad">
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c">CREATE</code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d"><code class="c">TABLE</code></td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c">VIEW</code></td><td class="le"></td></tr></table></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">IF NOT EXISTS</code></td></tr></table></td><td class="le"></td></tr></table></td><td class="d"><code class="c"><a href="index.html#table_ref">tableRef</a></code></td></tr></table><br /><table class="railroad"><tr class="railroad"><td class="d"><code class="c">( <a href="index.html#column_def">columnDef</a></code></td><td class="d"><table class="railroad"><tr class="r
 ailroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c">, ...</code></td><td class="le"></td></tr></table></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c"><a href="index.html#constraint">constraint</a></code></td><td class="le"></td></tr></table></td><td class="d"><code class="c">)</code></td></tr></table><br /><table class="railroad"><tr class="railroad"><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c"><a href="index.html#options">tableOptions</a></code></td><td class="le"></td></tr></table></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&
 nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">SPLIT ON ( <a href="index.html#split_point">splitPoint</a></code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c">, ...</code></td><td class="le"></td></tr></table></td><td class="d"><code class="c">)</code></td></tr></table></td><td class="le"></td></tr></table></td></tr></table>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+CREATE { TABLE | VIEW } [IF NOT EXISTS] <a href="index.html#table_ref">tableRef</a>
+( <a href="index.html#column_def">columnDef</a> [,...] [<a href="index.html#constraint">constraint</a>] )
+[<a href="index.html#options">tableOptions</a>] [ SPLIT ON ( <a href="index.html#split_point">splitPoint</a> [,...] ) ]
+</pre>
+syntax-end -->
+<p>Creates a new table or view. For the creation of a table, the <code>HBase</code> table and any column families referenced are created if they don&#39;t already exist (using uppercase names unless they are double quoted in which case they are case sensitive). Column families outside of the ones listed are not affected. At create time, an empty key value is added to the first column family of any existing rows. Upserts will also add this empty key value. This is done to improve query performance by having a key value column we can guarantee always being there (minimizing the amount of data that must be projected). Alternately, if a view is created, the <code>HBase</code> table and column families must already exist. No empty key value is added to existing rows and no data mutations are allowed - the view is read-only. Query performance for a view will not be as good as performance for a table. For a table only, <code>HBase</code> table and column configuration options may be passed
  through as key/value pairs to setup the <code>HBase</code> table as needed.</p>
+<p>Example:</p>
+<p class="notranslate">
+CREATE TABLE my_schema.my_table ( id BIGINT not null primary key, date DATE not null)<br />CREATE TABLE my_table ( id INTEGER not null primary key desc, date DATE not null,<br />&nbsp;&nbsp;&nbsp;&nbsp;m.db_utilization DECIMAL, i.db_utilization)<br />&nbsp;&nbsp;&nbsp;&nbsp;m.DATA_BLOCK_ENCODING=&#39;DIFF&#39;<br />CREATE TABLE stats.prod_metrics ( host char(50) not null, created_date date not null,<br />&nbsp;&nbsp;&nbsp;&nbsp;txn_count bigint CONSTRAINT pk PRIMARY KEY (host, created_date) )<br />CREATE TABLE IF NOT EXISTS my_table ( id char(10) not null primary key, value integer)<br />&nbsp;&nbsp;&nbsp;&nbsp;DATA_BLOCK_ENCODING=&#39;NONE&#39;,VERSIONS=?,MAX_FILESIZE=2000000 split on (?, ?, ?)</p>
+
+<h3 id="drop" class="notranslate">DROP</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+DROP {TABLE | VIEW} [IF EXISTS] <a href="index.html#table_ref">tableRef</a>
+</pre>
+<div name="railroad">
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c">DROP</code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d"><code class="c">TABLE</code></td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c">VIEW</code></td><td class="le"></td></tr></table></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">IF EXISTS</code></td></tr></table></td><td class="le"></td></tr></table></td><td class="d"><code class="c"><a href="index.html#table_ref">tableRef</a></code></td></tr></table>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+DROP {TABLE | VIEW} [IF EXISTS] <a href="index.html#table_ref">tableRef</a>
+</pre>
+syntax-end -->
+<p>Drops a table or view. When dropping a table, the data in the table is deleted. For a view, on the other hand, the data is not affected. Note that the schema is versioned, such that snapshot queries connecting at an earlier time stamp may still query against the dropped table, as the <code>HBase</code> table itself is not deleted.</p>
+<p>Example:</p>
+<p class="notranslate">
+DROP TABLE my_schema.my_table<br />DROP VIEW my_view</p>
+
+<h3 id="alter_table" class="notranslate">ALTER TABLE</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+ALTER TABLE <a href="index.html#table_ref">tableRef</a> { { ADD [IF NOT EXISTS] <a href="index.html#column_def">columnDef</a> [<a href="index.html#options">options</a>] } | { DROP COLUMN [IF EXISTS] <a href="index.html#column_ref">columnRef</a> } | { SET <a href="index.html#options">options</a> } }
+</pre>
+<div name="railroad">
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c">ALTER TABLE <a href="index.html#table_ref">tableRef</a></code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">ADD</code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">IF NOT EXISTS</code></td></tr></table></td><td class="le"></td></tr></table></td><td class="d"><code class="c"><a href="index.html#column_def">columnDef</a></code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c"><a href="index.html#options">options</a></cod
 e></td><td class="le"></td></tr></table></td></tr></table></td><td class="te"></td></tr><tr class="railroad"><td class="ks"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">DROP COLUMN</code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">IF EXISTS</code></td></tr></table></td><td class="le"></td></tr></table></td><td class="d"><code class="c"><a href="index.html#column_ref">columnRef</a></code></td></tr></table></td><td class="ke"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">SET <a href="index.html#options">options</a></code></td></tr></table></td><td class="le"></td></tr></table></td></tr></table>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+ALTER TABLE <a href="index.html#table_ref">tableRef</a> { { ADD [IF NOT EXISTS] <a href="index.html#column_def">columnDef</a> [<a href="index.html#options">options</a>] } | { DROP COLUMN [IF EXISTS] <a href="index.html#column_ref">columnRef</a> } | { SET <a href="index.html#options">options</a> } }
+</pre>
+syntax-end -->
+<p>Alters an existing table by adding or removing a column or updating table options. When a column is dropped from a table, the data in that column is deleted as well. <code>PK</code> columns may not be dropped, and only nullable <code>PK</code> columns may be added. For a view, the data is not affected when a column is dropped. Note that creating or dropping columns only affects subsequent queries and data modifications. Snapshot queries that are connected at an earlier timestamp will still use the prior schema that was in place when the data was written.</p>
+<p>Example:</p>
+<p class="notranslate">
+ALTER TABLE my_schema.my_table ADD d.dept_id char(10) VERSIONS=10<br />ALTER TABLE my_table ADD dept_name char(50)<br />ALTER TABLE my_table ADD parent_id char(15) null primary key<br />ALTER TABLE my_table DROP COLUMN d.dept_id<br />ALTER TABLE my_table DROP COLUMN dept_name<br />ALTER TABLE my_table DROP COLUMN parent_id<br />ALTER TABLE my_table SET IMMUTABLE_ROWS=true</p>
+
+<h3 id="create_index" class="notranslate">CREATE INDEX</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+CREATE INDEX [IF NOT EXISTS] <a href="index.html#name">indexName</a>
+ON <a href="index.html#table_ref">tableRef</a> ( <a href="index.html#column_ref">columnRef</a> [ASC | DESC] [,...] )
+[ INCLUDE ( <a href="index.html#column_ref">columnRef</a> [,...] ) ]
+[<a href="index.html#options">indexOptions</a>] [ SPLIT ON ( <a href="index.html#split_point">splitPoint</a> [,...] ) ]
+</pre>
+<div name="railroad">
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c">CREATE INDEX</code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">IF NOT EXISTS</code></td></tr></table></td><td class="le"></td></tr></table></td><td class="d"><code class="c"><a href="index.html#name">indexName</a></code></td></tr></table><br /><table class="railroad"><tr class="railroad"><td class="d"><code class="c">ON <a href="index.html#table_ref">tableRef</a> ( <a href="index.html#column_ref">columnRef</a></code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d"><code class="c
 ">ASC</code></td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c">DESC</code></td><td class="le"></td></tr></table></td><td class="le"></td></tr></table></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c">, ...</code></td><td class="le"></td></tr></table></td><td class="d"><code class="c">)</code></td></tr></table><br /><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">INCLUDE ( <a href="index.html#column_ref">columnRef</a></code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls
 "></td><td class="d"><code class="c">, ...</code></td><td class="le"></td></tr></table></td><td class="d"><code class="c">)</code></td></tr></table></td><td class="le"></td></tr></table><br /><table class="railroad"><tr class="railroad"><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c"><a href="index.html#options">indexOptions</a></code></td><td class="le"></td></tr></table></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">SPLIT ON ( <a href="index.html#split_point">splitPoint</a></code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr cla
 ss="railroad"><td class="ls"></td><td class="d"><code class="c">, ...</code></td><td class="le"></td></tr></table></td><td class="d"><code class="c">)</code></td></tr></table></td><td class="le"></td></tr></table></td></tr></table>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+CREATE INDEX [IF NOT EXISTS] <a href="index.html#name">indexName</a>
+ON <a href="index.html#table_ref">tableRef</a> ( <a href="index.html#column_ref">columnRef</a> [ASC | DESC] [,...] )
+[ INCLUDE ( <a href="index.html#column_ref">columnRef</a> [,...] ) ]
+[<a href="index.html#options">indexOptions</a>] [ SPLIT ON ( <a href="index.html#split_point">splitPoint</a> [,...] ) ]
+</pre>
+syntax-end -->
+<p>Creates a new secondary index on a table or view. The index will be automatically kept in sync with the table as the data changes. At query time, the optimizer will use the index if it contains all columns referenced in the query and produces the most efficient execution plan. If a table has rows that are write-once and append-only, then the table may set the <code>IMMUTABLE_ROWS</code> property to true (either up-front in the <code>CREATE TABLE</code> statement or afterwards in an <code>ALTER TABLE</code> statement). This reduces the overhead at write time to maintain the index. Otherwise, if this property is not set on the table, then incremental index maintenance will be performed on the server side when the data changes.</p>
+<p>Example:</p>
+<p class="notranslate">
+CREATE INDEX my_idx ON sales.opportunity(last_updated_date DESC)<br />CREATE INDEX my_idx ON log.event(created_date DESC) INCLUDE (name, payload) SALT_BUCKETS=10<br />CREATE INDEX IF NOT EXISTS my_comp_idx ON server_metrics ( gc_time DESC, created_date DESC )<br />&nbsp;&nbsp;&nbsp;&nbsp;DATA_BLOCK_ENCODING=&#39;NONE&#39;,VERSIONS=?,MAX_FILESIZE=2000000 split on (?, ?, ?)</p>
+
+<h3 id="drop_index" class="notranslate">DROP INDEX</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+DROP INDEX [IF EXISTS] <a href="index.html#name">indexName</a> ON <a href="index.html#table_ref">tableRef</a>
+</pre>
+<div name="railroad">
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c">DROP INDEX</code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">IF EXISTS</code></td></tr></table></td><td class="le"></td></tr></table></td><td class="d"><code class="c"><a href="index.html#name">indexName</a> ON <a href="index.html#table_ref">tableRef</a></code></td></tr></table>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+DROP INDEX [IF EXISTS] <a href="index.html#name">indexName</a> ON <a href="index.html#table_ref">tableRef</a>
+</pre>
+syntax-end -->
+<p>Drops an index from a table. When dropping an index, the data in the index is deleted. Note that since metadata is versioned, snapshot queries connecting at an earlier time stamp may still use the index, as the <code>HBase</code> table backing the index is not deleted.</p>
+<p>Example:</p>
+<p class="notranslate">
+DROP INDEX my_idx ON sales.opportunity<br />DROP INDEX IF EXISTS my_idx ON server_metrics</p>
+
+<h3 id="alter_index" class="notranslate">ALTER INDEX</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+ALTER INDEX [IF EXISTS] <a href="index.html#name">indexName</a> ON <a href="index.html#table_ref">tableRef</a> { DISABLE | REBUILD | UNUSABLE | USABLE }
+</pre>
+<div name="railroad">
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c">ALTER INDEX</code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">IF EXISTS</code></td></tr></table></td><td class="le"></td></tr></table></td><td class="d"><code class="c"><a href="index.html#name">indexName</a> ON <a href="index.html#table_ref">tableRef</a></code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d"><code class="c">DISABLE</code></td><td class="te"></td></tr><tr class="railroad"><td class="ks"></td><td class="d"><code class="c">REBUILD</code></td><td class="ke"></td></tr><tr class="railroad"><td class="ks"></td><td class="d"><code class="c">UNUSABLE</code></td><td class="ke"></td></tr><tr class="railroad"><td class="ls"></td><td class=
 "d"><code class="c">USABLE</code></td><td class="le"></td></tr></table></td></tr></table>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+ALTER INDEX [IF EXISTS] <a href="index.html#name">indexName</a> ON <a href="index.html#table_ref">tableRef</a> { DISABLE | REBUILD | UNUSABLE | USABLE }
+</pre>
+syntax-end -->
+<p>Alters the state of an existing index. &nbsp;<code>DISABLE</code> will cause the no further index maintenance to be performed on the index and it will no longer be considered for use in queries. <code>REBUILD</code> will completely rebuild the index and upon completion will enable the index to be used in queries again. <code>UNUSABLE</code> will cause the index to no longer be considered for use in queries, however index maintenance will continue to be performed. <code>USABLE</code> will cause the index to again be considered for use in queries. Note that a disabled index must be rebuild and cannot be set as <code>USABLE</code>.</p>
+<p>Example:</p>
+<p class="notranslate">
+ALTER INDEX my_idx ON sales.opportunity DISABLE<br />ALTER INDEX IF EXISTS my_idx ON server_metrics REBUILD</p>
+
+<h3 id="explain" class="notranslate">EXPLAIN</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+EXPLAIN {<a href="index.html#select">select</a>|<a href="index.html#upsert_select">upsertSelect</a>|<a href="index.html#delete">delete</a>}
+</pre>
+<div name="railroad">
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c">EXPLAIN</code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d"><code class="c"><a href="index.html#select">select</a></code></td><td class="te"></td></tr><tr class="railroad"><td class="ks"></td><td class="d"><code class="c"><a href="index.html#upsert_select">upsertSelect</a></code></td><td class="ke"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c"><a href="index.html#delete">delete</a></code></td><td class="le"></td></tr></table></td></tr></table>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+EXPLAIN {<a href="index.html#select">select</a>|<a href="index.html#upsert_select">upsertSelect</a>|<a href="index.html#delete">delete</a>}
+</pre>
+syntax-end -->
+<p>Computes the logical steps necessary to execute the given command. Each step is represented as a string in a single column result set row.</p>
+<p>Example:</p>
+<p class="notranslate">
+EXPLAIN SELECT NAME, COUNT(*) FROM TEST GROUP BY NAME HAVING COUNT(*) &gt; 2;<br />EXPLAIN SELECT entity_id FROM CORE.CUSTOM_ENTITY_DATA WHERE organization_id=&#39;00D300000000XHP&#39; AND SUBSTR(entity_id,1,3) = &#39;002&#39; AND created_date &lt; CURRENT_DATE()-1;</p>
+
+
+<h3 id="constraint" class="notranslate">Constraint</h3>
+<!-- railroad-start -->
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c">CONSTRAINT <a href="index.html#name">constraintName</a> PRIMARY KEY ( <a href="index.html#name">columnName</a></code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d"><code class="c">ASC</code></td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c">DESC</code></td><td class="le"></td></tr></table></td><td class="le"></td></tr></table></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c">, ...</code></td><td class="le"></td></tr></table></td><td class="d"><code class="c">)</code></td></tr></table>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+CONSTRAINT <a href="index.html#name">constraintName</a> PRIMARY KEY (<a href="index.html#name">columnName</a> [ASC | DESC] [,...])
+</pre>
+syntax-end -->
+<p>Defines a multi-part primary key constraint. Each column may be declared to be sorted in ascending or descending ordering. The default is ascending.</p>
+<p>Example:</p>
+<p class="notranslate">CONSTRAINT my_pk PRIMARY KEY (host,created_date)<br />CONSTRAINT my_pk PRIMARY KEY (host ASC,created_date DESC)</p>
+
+<h3 id="options" class="notranslate">Options</h3>
+<!-- railroad-start -->
+<table class="railroad"><tr class="railroad"><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c"><a href="index.html#name">familyName</a> .</code></td></tr></table></td><td class="le"></td></tr></table></td><td class="d"><code class="c"><a href="index.html#name">name</a> =</code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d"><code class="c"><a href="index.html#value">value</a></code></td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c"><a href="index.html#bind_parameter">bindParameter</a></code></td><td class="le"></td></tr></table></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"
 ><td class="ls"></td><td class="d"><code class="c">, ...</code></td><td class="le"></td></tr></table></td></tr></table>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+{ [<a href="index.html#name">familyName</a> .] <a href="index.html#name">name</a>= {<a href="index.html#value">value</a> | <a href="index.html#bind_parameter">bindParameter</a>}} [,...]
+</pre>
+syntax-end -->
+<p>Sets an option on an <code>HBase</code> table or column by modifying the respective <code>HBase</code> metadata. The option applies to the named family or if omitted to all families if the name references an <code>HColumnDescriptor</code> property. Otherwise, the option applies to the <code>HTableDescriptor</code>.</p><p>One built-in option is <code>SALT_BUCKETS</code>. This option causes an extra byte to be transparently prepended to every row key to ensure an even distribution of write load across all your region servers. This is useful when your row key is always monotonically increasing causing hot spotting on a single region server. The byte is determined by hashing the row key and modding it with the <code>SALT_BUCKETS</code> value. The value may be from 1 to 256. If not split points are defined for the table, it will automatically be pre-split at each possible salt bucket value. For an excellent write-up of this technique, see http://blog.sematext.com/2012/04/09/hbasewd-av
 oid-regionserver-hotspotting-despite-writing-records-with-sequential-keys/</p><p>Another built-in options is <code>IMMUTABLE_ROWS</code>. Only tables with immutable rows are allowed to have indexes. Immutable rows are expected to be inserted once in their entirety and then never updated. This limitation will be removed once incremental index maintenance has been implemented. The current implementation inserts the index rows when the data row is inserted.</p>
+<p>Example:</p>
+<p class="notranslate">IMMUTABLE_ROWS=true<br />SALT_BUCKETS=10<br />DATA_BLOCK_ENCODING=&#39;NONE&#39;,a.VERSIONS=10<br />MAX_FILESIZE=2000000000,MEMSTORE_FLUSHSIZE=80000000</p>
+
+<h3 id="hint" class="notranslate">Hint</h3>
+<!-- railroad-start -->
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c"><a href="index.html#name">name</a></code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c">, ...</code></td><td class="le"></td></tr></table></td></tr></table>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+<a href="index.html#name">name</a> [,...]
+</pre>
+syntax-end -->
+<p>Advanced features that overrides default query processing behavior. The supported hints include 1) <code>SKIP_SCAN</code> to force a skip scan to be performed on the query when it otherwise would not be. This option may improve performance if a query does not include the leading primary key column, but does include other, very selective primary key columns. 2) <code>RANGE_SCAN</code> to force a range scan to be performed on the query. This option may improve performance if a query filters on a range for non selective leading primary key column along with other primary key columns 3) <code>NO_INTRA_REGION_PARALLELIZATION</code> to prevent the spawning of multiple threads to process data within a single region. This option is useful when the overall data set being queries is known to be small. 4) <code>NO_INDEX</code> to force the data table to be used for a query, and 5) <code>INDEX</code>(&lt;table_name&gt; &lt;index_name&gt;...) to suggest which index to use for a given query. D
 ouble quotes may be used to surround a table_name and/or index_name that is case sensitive.</p>
+<p>Example:</p>
+<p class="notranslate">/*+ SKIP_SCAN */<br />/*+ RANGE_SCAN */<br />/*+ NO_INTRA_REGION_PARALLELIZATION */<br />/*+ NO_INDEX */<br />/*+ INDEX(employee emp_name_idx emp_start_date_idx) */</p>
+
+<h3 id="column_def" class="notranslate">Column Def</h3>
+<!-- railroad-start -->
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c"><a href="index.html#column_ref">columnRef</a> <a href="index.html#data_type">dataType</a></code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c">NOT</code></td><td class="le"></td></tr></table></td><td class="d"><code class="c">NULL</code></td></tr></table></td><td class="le"></td></tr></table></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td clas
 s="d"><code class="c">PRIMARY KEY</code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d"><code class="c">ASC</code></td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c">DESC</code></td><td class="le"></td></tr></table></td><td class="le"></td></tr></table></td></tr></table></td><td class="le"></td></tr></table></td></tr></table>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+<a href="index.html#column_ref">columnRef</a> <a href="index.html#data_type">dataType</a> [[NOT] NULL] [PRIMARY KEY [ASC | DESC] ]
+</pre>
+syntax-end -->
+<p>Define a new primary key column. The column name is case insensitive by default and case sensitive if double quoted. The sort order of a primary key may be ascending (<code>ASC</code>) or descending. The default is ascending.</p>
+<p>Example:</p>
+<p class="notranslate">id char(15) not null primary key<br />key integer null<br />m.response_time bigint</p>
+
+<h3 id="table_ref" class="notranslate">Table Ref</h3>
+<!-- railroad-start -->
+<table class="railroad"><tr class="railroad"><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c"><a href="index.html#name">schemaName</a> .</code></td></tr></table></td><td class="le"></td></tr></table></td><td class="d"><code class="c"><a href="index.html#name">tableName</a></code></td></tr></table>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+[<a href="index.html#name">schemaName</a> .] <a href="index.html#name">tableName</a>
+</pre>
+syntax-end -->
+<p>References a table with an optional schema name qualifier</p>
+<p>Example:</p>
+<p class="notranslate">Sales.Contact<br />HR.Employee<br />Department</p>
+
+<h3 id="column_ref" class="notranslate">Column Ref</h3>
+<!-- railroad-start -->
+<table class="railroad"><tr class="railroad"><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c"><a href="index.html#name">familyName</a> .</code></td></tr></table></td><td class="le"></td></tr></table></td><td class="d"><code class="c"><a href="index.html#name">columnName</a></code></td></tr></table>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+[<a href="index.html#name">familyName</a> .] <a href="index.html#name">columnName</a>
+</pre>
+syntax-end -->
+<p>References a column with an optional family name qualifier</p>
+<p>Example:</p>
+<p class="notranslate">e.salary<br />dept_name</p>
+
+<h3 id="select_expression" class="notranslate">Select Expression</h3>
+<!-- railroad-start -->
+<table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d"><code class="c">*</code></td><td class="te"></td></tr><tr class="railroad"><td class="ks"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">( <a href="index.html#name">familyName</a> . * )</code></td></tr></table></td><td class="ke"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c"><a href="index.html#term">term</a></code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c">AS</code></td><td class="l
 e"></td></tr></table></td><td class="d"><code class="c"><a href="index.html#alias">columnAlias</a></code></td></tr></table></td><td class="le"></td></tr></table></td></tr></table></td><td class="le"></td></tr></table>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+* | ( <a href="index.html#name">familyName</a> . *) | <a href="index.html#term">term</a> [ [ AS ] <a href="index.html#alias">columnAlias</a> ]
+</pre>
+syntax-end -->
+<p>An expression in a <code>SELECT</code> statement. All columns in a table may be selected using *, and all columns in a column family may be selected using &lt;familyName&gt;.*.</p>
+<p>Example:</p>
+<p class="notranslate">*<br />cf.*<br />ID AS VALUE<br />VALUE + 1 VALUE_PLUS_ONE</p>
+
+<h3 id="split_point" class="notranslate">Split Point</h3>
+<!-- railroad-start -->
+<table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d"><code class="c"><a href="index.html#value">value</a></code></td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c"><a href="index.html#bind_parameter">bindParameter</a></code></td><td class="le"></td></tr></table>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+<a href="index.html#value">value</a> | <a href="index.html#bind_parameter">bindParameter</a>
+</pre>
+syntax-end -->
+<p>Defines a split point for a table. Use a bind parameter with preparedStatement.setBinary(int,byte[]) to supply arbitrary bytes.</p>
+<p>Example:</p>
+<p class="notranslate">&#39;A&#39;</p>
+
+<h3 id="table_expression" class="notranslate">Table Expression</h3>
+<!-- railroad-start -->
+<table class="railroad"><tr class="railroad"><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c"><a href="index.html#name">schemaName</a> .</code></td></tr></table></td><td class="le"></td></tr></table></td><td class="d"><code class="c"><a href="index.html#name">tableName</a></code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c">AS</code></td><td class="le"></td></tr></table></td><td class="d"><code c
 lass="c"><a href="index.html#alias">tableAlias</a></code></td></tr></table></td><td class="le"></td></tr></table></td></tr></table>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+{ [ <a href="index.html#name">schemaName</a>. ] <a href="index.html#name">tableName</a> } [ [ AS ] <a href="index.html#alias">tableAlias</a> ]
+</pre>
+syntax-end -->
+<p>A reference to a table. Joins and sub queries are not currently supported.</p>
+<p>Example:</p>
+<p class="notranslate">PRODUCT_METRICS AS PM</p>
+
+<h3 id="order" class="notranslate">Order</h3>
+<!-- railroad-start -->
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c"><a href="index.html#expression">expression</a></code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d"><code class="c">ASC</code></td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c">DESC</code></td><td class="le"></td></tr></table></td><td class="le"></td></tr></table></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">NULLS</code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d"><code class="
 c">FIRST</code></td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c">LAST</code></td><td class="le"></td></tr></table></td></tr></table></td><td class="le"></td></tr></table></td></tr></table>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+{ <a href="index.html#expression">expression</a> } [ ASC | DESC ] [ NULLS { FIRST | LAST } ]
+</pre>
+syntax-end -->
+<p>Sorts the result by an expression.</p>
+<p>Example:</p>
+<p class="notranslate">NAME DESC NULLS LAST</p>
+
+<h3 id="expression" class="notranslate">Expression</h3>
+<!-- railroad-start -->
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c"><a href="index.html#and_condition">andCondition</a></code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">OR <a href="index.html#and_condition">andCondition</a></code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c">...</code></td><td class="le"></td></tr></table></td></tr></table></td><td class="le"></td></tr></table></td></tr></table>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+<a href="index.html#and_condition">andCondition</a> [ { OR <a href="index.html#and_condition">andCondition</a> } [...] ]
+</pre>
+syntax-end -->
+<p>Value or condition.</p>
+<p>Example:</p>
+<p class="notranslate">ID=1 OR NAME=&#39;Hi&#39;</p>
+
+<h3 id="and_condition" class="notranslate">And Condition</h3>
+<!-- railroad-start -->
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c"><a href="index.html#condition">condition</a></code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">AND <a href="index.html#condition">condition</a></code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c">...</code></td><td class="le"></td></tr></table></td></tr></table></td><td class="le"></td></tr></table></td></tr></table>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+<a href="index.html#condition">condition</a> [ { AND <a href="index.html#condition">condition</a> } [...] ]
+</pre>
+syntax-end -->
+<p>Value or condition.</p>
+<p>Example:</p>
+<p class="notranslate">ID=1 AND NAME=&#39;Hi&#39;</p>
+
+<h3 id="condition" class="notranslate">Condition</h3>
+<!-- railroad-start -->
+<table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c"><a href="index.html#operand">operand</a></code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c"><a href="index.html#compare">compare</a> <a href="index.html#operand">operand</a></code></td></tr></table></td><td class="te"></td></tr><tr class="railroad"><td class="ks"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c">NOT</code></td><t
 d class="le"></td></tr></table></td><td class="d"><code class="c">IN ( <a href="index.html#operand">constantOperand</a></code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c">, ...</code></td><td class="le"></td></tr></table></td><td class="d"><code class="c">)</code></td></tr></table></td><td class="ke"></td></tr><tr class="railroad"><td class="ks"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c">NOT</code></td><td class="le"></td></tr></table></td><td class="d"><code class="c">LIKE <a href="index.html#operand">operand</a></code></td></tr></table></td><td class="ke"></td></tr><tr class="railroad"><td class="ks"></td><
 td class="d"><table class="railroad"><tr class="railroad"><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c">NOT</code></td><td class="le"></td></tr></table></td><td class="d"><code class="c">BETWEEN <a href="index.html#operand">operand</a> AND <a href="index.html#operand">operand</a></code></td></tr></table></td><td class="ke"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">IS</code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c">NOT</code></td><td class="le"></td></tr></table></td><td class="d"><code class="c">NULL</code></td></tr></table></td><td class="le"></td></tr></table></td><td class
 ="le"></td></tr></table></td></tr></table></td><td class="te"></td></tr><tr class="railroad"><td class="ks"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">NOT <a href="index.html#expression">expression</a></code></td></tr></table></td><td class="ke"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">( <a href="index.html#expression">expression</a> )</code></td></tr></table></td><td class="le"></td></tr></table>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+<a href="index.html#operand">operand</a> [ <a href="index.html#compare">compare</a> { <a href="index.html#operand">operand</a> }
+| [ NOT ] IN ( { <a href="index.html#operand">constantOperand</a> [,...] } )
+| [ NOT ] LIKE <a href="index.html#operand">operand</a>
+| [ NOT ] BETWEEN <a href="index.html#operand">operand</a> AND <a href="index.html#operand">operand</a>
+| IS [ NOT ] NULL ]
+| NOT <a href="index.html#expression">expression</a>
+| ( <a href="index.html#expression">expression</a> )
+</pre>
+syntax-end -->
+<p>Boolean value or condition. When comparing with <code>LIKE</code>, the wildcards characters are <code>_</code> (any one character) and <code>%</code> (any characters). To search for the characters <code>%</code> and <code>_</code>, the characters need to be escaped. The escape character is <code> \ </code> (backslash). Patterns that end with an escape character are invalid and the expression returns <code>NULL</code>. <code>BETWEEN</code> does an inclusive comparison for both operands.</p>
+<p>Example:</p>
+<p class="notranslate">NAME LIKE &#39;Jo%&#39;</p>
+
+<h3 id="compare" class="notranslate">Compare</h3>
+<!-- railroad-start -->
+<table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">&lt; &gt;</code></td></tr></table></td><td class="te"></td></tr><tr class="railroad"><td class="ks"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">&lt; =</code></td></tr></table></td><td class="ke"></td></tr><tr class="railroad"><td class="ks"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">&gt; =</code></td></tr></table></td><td class="ke"></td></tr><tr class="railroad"><td class="ks"></td><td class="d"><code class="c">=</code></td><td class="ke"></td></tr><tr class="railroad"><td class="ks"></td><td class="d"><code class="c">&lt;</code></td><td class="ke"></td></tr><tr class="railroad"><td class="ks"></td><td class="d"><code class="c">&gt;</code></td><td class="ke"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table cl
 ass="railroad"><tr class="railroad"><td class="d"><code class="c">! =</code></td></tr></table></td><td class="le"></td></tr></table>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+&lt;&gt; | &lt;= | &gt;= | = | &lt; | &gt; | !=
+</pre>
+syntax-end -->
+<p>Comparison operator. The operator != is the same as &lt;&gt;.</p>
+<p>Example:</p>
+<p class="notranslate">&lt;&gt;</p>
+
+<h3 id="operand" class="notranslate">Operand</h3>
+<!-- railroad-start -->
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c"><a href="index.html#summand">summand</a></code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">|| <a href="index.html#summand">summand</a></code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c">...</code></td><td class="le"></td></tr></table></td></tr></table></td><td class="le"></td></tr></table></td></tr></table>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+<a href="index.html#summand">summand</a> [ { || } <a href="index.html#summand">summand</a> [...] ]
+</pre>
+syntax-end -->
+<p>A string concatenation.</p>
+<p>Example:</p>
+<p class="notranslate">&#39;foo&#39;|| s</p>
+
+<h3 id="summand" class="notranslate">Summand</h3>
+<!-- railroad-start -->
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c"><a href="index.html#factor">factor</a></code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d"><code class="c">+</code></td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c">-</code></td><td class="le"></td></tr></table></td><td class="d"><code class="c"><a href="index.html#factor">factor</a></code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c">...</code></td><td class="le"></td></tr></table></td></tr></table></td><td class="le"></td></t
 r></table></td></tr></table>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+<a href="index.html#factor">factor</a> [ { + | - } <a href="index.html#factor">factor</a> [...] ]
+</pre>
+syntax-end -->
+<p>An addition or subtraction of numeric or date type values</p>
+<p>Example:</p>
+<p class="notranslate">a + b<br />a - b</p>
+
+<h3 id="factor" class="notranslate">Factor</h3>
+<!-- railroad-start -->
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c"><a href="index.html#term">term</a></code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d"><code class="c">*</code></td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c">/</code></td><td class="le"></td></tr></table></td><td class="d"><code class="c"><a href="index.html#term">term</a></code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c">...</code></td><td class="le"></td></tr></table></td></tr></table></td><td class="le"></td></tr></tabl
 e></td></tr></table>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+<a href="index.html#term">term</a> [ { * | / } <a href="index.html#term">term</a> [...] ]
+</pre>
+syntax-end -->
+<p>A multiplication or division.</p>
+<p>Example:</p>
+<p class="notranslate">c * d<br />e / 5</p>
+
+<h3 id="term" class="notranslate">Term</h3>
+<!-- railroad-start -->
+<table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d"><code class="c"><a href="index.html#value">value</a></code></td><td class="te"></td></tr><tr class="railroad"><td class="ks"></td><td class="d"><code class="c"><a href="index.html#bind_parameter">bindParameter</a></code></td><td class="ke"></td></tr><tr class="railroad"><td class="ks"></td><td class="d"><code class="c"><a href="functions.html">Function</a></code></td><td class="ke"></td></tr><tr class="railroad"><td class="ks"></td><td class="d"><code class="c"><a href="index.html#case">case</a></code></td><td class="ke"></td></tr><tr class="railroad"><td class="ks"></td><td class="d"><code class="c"><a href="index.html#case_when">caseWhen</a></code></td><td class="ke"></td></tr><tr class="railroad"><td class="ks"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">( <a href="index.html#operand">operand</a> )</code></td></tr></table></td><td class="ke"></td></tr><
 tr class="railroad"><td class="ks"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c"><a href="index.html#alias">tableAlias</a> .</code></td></tr></table></td><td class="le"></td></tr></table></td><td class="d"><code class="c"><a href="index.html#column_ref">columnRef</a></code></td></tr></table></td><td class="ke"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c"><a href="index.html#row_value_constructor">rowValueConstructor</a></code></td><td class="le"></td></tr></table>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+<a href="index.html#value">value</a>
+| <a href="index.html#bind_parameter">bindParameter</a>
+| function
+| <a href="index.html#case">case</a>
+| <a href="index.html#case_when">caseWhen</a>
+| ( <a href="index.html#operand">operand</a> )
+| [ <a href="index.html#alias">tableAlias</a>. ] <a href="index.html#column_ref">columnRef</a>
+| <a href="index.html#row_value_constructor">rowValueConstructor</a>
+</pre>
+syntax-end -->
+<p>A value.</p>
+<p>Example:</p>
+<p class="notranslate">&#39;Hello&#39;</p>
+
+<h3 id="row_value_constructor" class="notranslate">Row Value Constructor</h3>
+<!-- railroad-start -->
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c">( <a href="index.html#term">term</a> , <a href="index.html#term">term</a></code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c">...</code></td><td class="le"></td></tr></table></td><td class="d"><code class="c">)</code></td></tr></table>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+( <a href="index.html#term">term</a> { ,<a href="index.html#term">term</a> } [...] )
+</pre>
+syntax-end -->
+<p>A row value constructor is a list of other terms which are treated together as a kind of composite structure. They may be compared to each other or to other other terms. The main use case is 1) to enable efficiently stepping through a set of rows in support of query-more type functionality, or 2) to allow <code>IN</code> clause to perform point gets on composite row keys.</p>
+<p>Example:</p>
+<p class="notranslate">(col1, col2, 5)</p>
+
+<h3 id="bind_parameter" class="notranslate">Bind Parameter</h3>
+<!-- railroad-start -->
+<table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d"><code class="c">?</code></td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">: <a href="index.html#number">number</a></code></td></tr></table></td><td class="le"></td></tr></table>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+?
+| :<a href="index.html#number">number</a>
+</pre>
+syntax-end -->
+<p>A parameters can be indexed, for example <code>:1</code> meaning the first parameter.</p>
+<p>Example:</p>
+<p class="notranslate">:1<br />?</p>
+
+<h3 id="value" class="notranslate">Value</h3>
+<!-- railroad-start -->
+<table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d"><code class="c"><a href="index.html#string">string</a></code></td><td class="te"></td></tr><tr class="railroad"><td class="ks"></td><td class="d"><code class="c"><a href="index.html#numeric">numeric</a></code></td><td class="ke"></td></tr><tr class="railroad"><td class="ks"></td><td class="d"><code class="c"><a href="index.html#boolean">boolean</a></code></td><td class="ke"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c"><a href="index.html#null">null</a></code></td><td class="le"></td></tr></table>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+<a href="index.html#string">string</a> | <a href="index.html#numeric">numeric</a> | <a href="index.html#boolean">boolean</a> | <a href="index.html#null">null</a>
+</pre>
+syntax-end -->
+<p>A literal value of any data type, or null.</p>
+<p>Example:</p>
+<p class="notranslate">10</p>
+
+<h3 id="case" class="notranslate">Case</h3>
+<!-- railroad-start -->
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c">CASE <a href="index.html#term">term</a> WHEN <a href="index.html#expression">expression</a> THEN <a href="index.html#term">term</a></code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c">...</code></td><td class="le"></td></tr></table></td></tr></table><br /><table class="railroad"><tr class="railroad"><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">ELSE <a href="index.html#expression">expression</a></code></td></tr></table></td><td class="le"></td></tr></table></td><td class="d"><code class="c">END</code></td></tr></table>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+CASE <a href="index.html#term">term</a> { WHEN <a href="index.html#expression">expression</a> THEN <a href="index.html#term">term</a> } [...]
+[ ELSE <a href="index.html#expression">expression</a> ] END
+</pre>
+syntax-end -->
+<p>Returns the first expression where the value is equal to the test expression. If no else part is specified, return <code>NULL</code>.</p>
+<p>Example:</p>
+<p class="notranslate">CASE CNT WHEN 0 THEN &#39;No&#39; WHEN 1 THEN &#39;One&#39; ELSE &#39;Some&#39; END</p>
+
+<h3 id="case_when" class="notranslate">Case When</h3>
+<!-- railroad-start -->
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c">CASE WHEN <a href="index.html#expression">expression</a> THEN <a href="index.html#term">term</a></code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c">...</code></td><td class="le"></td></tr></table></td></tr></table><br /><table class="railroad"><tr class="railroad"><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">ELSE <a href="index.html#term">term</a></code></td></tr></table></td><td class="le"></td></tr></table></td><td class="d"><code class="c">END</code></td></tr></table>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+CASE { WHEN <a href="index.html#expression">expression</a> THEN <a href="index.html#term">term</a>} [...]
+[ ELSE <a href="index.html#term">term</a> ] END
+</pre>
+syntax-end -->
+<p>Returns the first expression where the condition is true. If no else part is specified, return <code>NULL</code>.</p>
+<p>Example:</p>
+<p class="notranslate">CASE WHEN CNT&lt;10 THEN &#39;Low&#39; ELSE &#39;High&#39; END</p>
+
+<h3 id="name" class="notranslate">Name</h3>
+<!-- railroad-start -->
+<table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">A-Z | _</code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d"><code class="c">A-Z | _</code></td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c">0-9</code></td><td class="le"></td></tr></table></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c">...</code></td><td class="le"></td></tr></table></td></tr></table></td><td class="le"></td></tr></table></td></tr></ta
 ble></td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c"><a href="index.html#quoted_name">quotedName</a></code></td><td class="le"></td></tr></table>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+{ { A-Z|_  } [ { A-Z|_|0-9 } [...] ] } | <a href="index.html#quoted_name">quotedName</a>
+</pre>
+syntax-end -->
+<p>Unquoted names are not case sensitive. There is no maximum name length.</p>
+<p>Example:</p>
+<p class="notranslate">my_column</p>
+
+<h3 id="quoted_name" class="notranslate">Quoted Name</h3>
+<!-- railroad-start -->
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c">&quot; anything &quot;</code></td></tr></table>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+&quot;anything&quot;
+</pre>
+syntax-end -->
+<p>Quoted names are case sensitive, and can contain spaces. There is no maximum name length. Two double quotes can be used to create a single double quote inside an identifier.</p>
+<p>Example:</p>
+<p class="notranslate">&quot;first-name&quot;</p>
+
+<h3 id="alias" class="notranslate">Alias</h3>
+<!-- railroad-start -->
+<code class="c"><a href="index.html#name">name</a></code>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+<a href="index.html#name">name</a>
+</pre>
+syntax-end -->
+<p>An alias is a name that is only valid in the context of the statement.</p>
+<p>Example:</p>
+<p class="notranslate">A</p>
+
+<h3 id="null" class="notranslate">Null</h3>
+<!-- railroad-start -->
+<code class="c">NULL</code>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+NULL
+</pre>
+syntax-end -->
+<p><code>NULL</code> is a value without data type and means &#39;unknown value&#39;.</p>
+<p>Example:</p>
+<p class="notranslate">NULL</p>
+
+<h3 id="data_type" class="notranslate">Data Type</h3>
+<!-- railroad-start -->
+<table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d"><code class="c"><a href="datatypes.html#char_type">charType</a></code></td><td class="te"></td></tr><tr class="railroad"><td class="ks"></td><td class="d"><code class="c"><a href="datatypes.html#varchar_type">varcharType</a></code></td><td class="ke"></td></tr><tr class="railroad"><td class="ks"></td><td class="d"><code class="c"><a href="datatypes.html#decimal_type">decimalType</a></code></td><td class="ke"></td></tr><tr class="railroad"><td class="ks"></td><td class="d"><code class="c"><a href="datatypes.html#tinyint_type">tinyintType</a></code></td><td class="ke"></td></tr><tr class="railroad"><td class="ks"></td><td class="d"><code class="c"><a href="datatypes.html#smallint_type">smallintType</a></code></td><td class="ke"></td></tr><tr class="railroad"><td class="ks"></td><td class="d"><code class="c"><a href="datatypes.html#integer_type">integerType</a></code></td><td class="ke"></td></tr><tr class="
 railroad"><td class="ks"></td><td class="d"><code class="c"><a href="datatypes.html#bigint_type">bigintType</a></code></td><td class="ke"></td></tr><tr class="railroad"><td class="ks"></td><td class="d"><code class="c"><a href="datatypes.html#float_type">floatType</a></code></td><td class="ke"></td></tr><tr class="railroad"><td class="ks"></td><td class="d"><code class="c"><a href="datatypes.html#double_type">doubleType</a></code></td><td class="ke"></td></tr><tr class="railroad"><td class="ks"></td><td class="d"><code class="c"><a href="datatypes.html#timestamp_type">timestampType</a></code></td><td class="ke"></td></tr><tr class="railroad"><td class="ks"></td><td class="d"><code class="c"><a href="datatypes.html#date_type">dateType</a></code></td><td class="ke"></td></tr><tr class="railroad"><td class="ks"></td><td class="d"><code class="c"><a href="datatypes.html#time_type">timeType</a></code></td><td class="ke"></td></tr><tr class="railroad"><td class="ks"></td><td class="d"><co
 de class="c"><a href="datatypes.html#unsigned_tinyint_type">unsignedTinyintType</a></code></td><td class="ke"></td></tr><tr class="railroad"><td class="ks"></td><td class="d"><code class="c"><a href="datatypes.html#unsigned_smallint_type">unsignedSmallintType</a></code></td><td class="ke"></td></tr><tr class="railroad"><td class="ks"></td><td class="d"><code class="c"><a href="datatypes.html#unsigned_int_type">unsignedIntType</a></code></td><td class="ke"></td></tr><tr class="railroad"><td class="ks"></td><td class="d"><code class="c"><a href="datatypes.html#unsigned_long_type">unsignedLongType</a></code></td><td class="ke"></td></tr><tr class="railroad"><td class="ks"></td><td class="d"><code class="c"><a href="datatypes.html#unsigned_float_type">unsignedFloatType</a></code></td><td class="ke"></td></tr><tr class="railroad"><td class="ks"></td><td class="d"><code class="c"><a href="datatypes.html#unsigned_double_type">unsignedDoubleType</a></code></td><td class="ke"></td></tr><tr c
 lass="railroad"><td class="ks"></td><td class="d"><code class="c"><a href="datatypes.html#binary_type">binaryType</a></code></td><td class="ke"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c"><a href="datatypes.html#varbinary_type">varbinaryType</a></code></td><td class="le"></td></tr></table>
+<!-- railroad-end -->
+<!-- syntax-star

<TRUNCATED>

[27/51] [partial] Initial commit of master branch from github

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToNumberFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToNumberFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToNumberFunction.java
new file mode 100644
index 0000000..84cc2b4
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToNumberFunction.java
@@ -0,0 +1,176 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import static org.apache.phoenix.util.ByteUtil.EMPTY_BYTE_ARRAY;
+
+import java.io.*;
+import java.math.BigDecimal;
+import java.sql.SQLException;
+import java.text.*;
+import java.util.Date;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.io.WritableUtils;
+
+import com.google.common.base.Preconditions;
+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.*;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+/**
+ * 
+ * Implementation of TO_NUMBER(&lt;string&gt;/&lt;date&gt;/&lt;timestamp&gt;, [&lt;pattern-string&gt;]) built-in function.  The format for the optional
+ * <code>pattern_string</code> param is specified in {@link DecimalFormat}.
+ *
+ * @author elevine
+ * @since 0.1
+ */
+@BuiltInFunction(name=ToNumberFunction.NAME,  nodeClass=ToNumberParseNode.class, args= {
+        @Argument(allowedTypes={PDataType.VARCHAR, PDataType.TIMESTAMP}),
+        @Argument(allowedTypes={PDataType.VARCHAR}, isConstant=true, defaultValue="null")} )
+public class ToNumberFunction extends ScalarFunction {
+	public static final String NAME = "TO_NUMBER";
+    
+    private String formatString = null;
+    private Format format = null;
+	private FunctionArgumentType type;
+    
+    public ToNumberFunction() {}
+
+    public ToNumberFunction(List<Expression> children, FunctionArgumentType type, String formatString, Format formatter) throws SQLException {
+        super(children.subList(0, 1));
+        Preconditions.checkNotNull(type);
+        this.type = type;
+        this.formatString = formatString;
+        this.format = formatter;
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        Expression expression = getExpression();
+        if (!expression.evaluate(tuple, ptr)) {
+            return false;
+        } else if (ptr.getLength() == 0) {
+            return true;
+        }
+
+        PDataType type = expression.getDataType();
+        if (type.isCoercibleTo(PDataType.TIMESTAMP)) {
+        	Date date = (Date) type.toObject(ptr, expression.getColumnModifier());
+        	BigDecimal time = new BigDecimal(date.getTime());
+            byte[] byteValue = getDataType().toBytes(time);
+            ptr.set(byteValue);
+            return true;
+        }
+        
+        String stringValue = (String)type.toObject(ptr, expression.getColumnModifier());
+        if (stringValue == null) {
+            ptr.set(EMPTY_BYTE_ARRAY);
+            return true;
+        }
+        stringValue = stringValue.trim();
+        BigDecimal decimalValue;
+        if (format == null) {
+            decimalValue = (BigDecimal) getDataType().toObject(stringValue);
+        } else {
+            ParsePosition parsePosition = new ParsePosition(0);
+            Number number = ((DecimalFormat) format).parse(stringValue, parsePosition);
+            if (parsePosition.getErrorIndex() > -1) {
+                ptr.set(EMPTY_BYTE_ARRAY);
+                return true;
+            }
+            
+            if (number instanceof BigDecimal) { 
+                // since we set DecimalFormat.setParseBigDecimal(true) we are guaranteeing result to be 
+                // of type BigDecimal in most cases.  see java.text.DecimalFormat.parse() JavaDoc.
+                decimalValue = (BigDecimal)number;
+            } else {
+                ptr.set(EMPTY_BYTE_ARRAY);
+                return true;
+            }
+        }
+        byte[] byteValue = getDataType().toBytes(decimalValue);
+        ptr.set(byteValue);
+        return true;
+    }
+
+    @Override
+    public PDataType getDataType() {
+    	return PDataType.DECIMAL;
+    }
+    
+    @Override
+    public boolean isNullable() {
+        return getExpression().isNullable();
+    }
+
+    private Expression getExpression() {
+        return children.get(0);
+    }
+
+    @Override
+    public String getName() {
+        return NAME;
+    }
+    
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        super.readFields(input);
+        formatString = WritableUtils.readString(input);
+        type = WritableUtils.readEnum(input, FunctionArgumentType.class);
+        if (formatString != null) {
+        	format = type.getFormatter(formatString);
+        }
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+        super.write(output);
+        WritableUtils.writeString(output, formatString);
+        WritableUtils.writeEnum(output, type);
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = super.hashCode();
+        result = prime * result + ((formatString == null) ? 0 : formatString.hashCode());
+        result = prime * result + getExpression().hashCode();
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) return true;
+        if (!super.equals(obj)) return false;
+        if (getClass() != obj.getClass()) return false;
+        ToNumberFunction other = (ToNumberFunction)obj;
+        if (formatString == null) {
+            if (other.formatString != null) return false;
+        } else if (!formatString.equals(other.formatString)) return false;
+        if (!getExpression().equals(other.getExpression())) return false;
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/TrimFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/TrimFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/TrimFunction.java
new file mode 100644
index 0000000..b8c06cd
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/TrimFunction.java
@@ -0,0 +1,108 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.parse.FunctionParseNode.Argument;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.StringUtil;
+
+
+/**
+ * Implementation of the Trim(<string>) build-in function. It removes from both end of <string>
+ * space character and other function bytes in single byte utf8 characters set.
+ * 
+ * @author zhuang
+ * @since 0.1
+ */
+@BuiltInFunction(name=TrimFunction.NAME, args={
+    @Argument(allowedTypes={PDataType.VARCHAR})} )
+public class TrimFunction extends ScalarFunction {
+    public static final String NAME = "TRIM";
+
+    private Integer byteSize;
+
+    public TrimFunction() { }
+
+    public TrimFunction(List<Expression> children) throws SQLException {
+        super(children);
+        if (getStringExpression().getDataType().isFixedWidth()) {
+            byteSize = getStringExpression().getByteSize();
+        }
+    }
+
+    private Expression getStringExpression() {
+        return children.get(0);
+    }
+
+    @Override
+    public ColumnModifier getColumnModifier() {
+        return children.get(0).getColumnModifier();
+    }    
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        if (!getStringExpression().evaluate(tuple, ptr)) {
+            return false;
+        }
+        if (ptr.getLength() == 0) {
+            ptr.set(ByteUtil.EMPTY_BYTE_ARRAY);
+            return true;
+        }
+        byte[] string = ptr.get();
+        int offset = ptr.getOffset();
+        int length = ptr.getLength();
+        
+        ColumnModifier columnModifier = getColumnModifier();
+        int end = StringUtil.getFirstNonBlankCharIdxFromEnd(string, offset, length, columnModifier);
+        if (end == offset - 1) {
+            ptr.set(ByteUtil.EMPTY_BYTE_ARRAY);
+            return true; 
+        }
+        int head = StringUtil.getFirstNonBlankCharIdxFromStart(string, offset, length, columnModifier);
+        ptr.set(string, head, end - head + 1);
+        return true;
+    }
+
+    @Override
+    public Integer getByteSize() {
+        return byteSize;
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return PDataType.VARCHAR;
+    }
+
+    @Override
+    public String getName() {
+        return NAME;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/TruncFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/TruncFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/TruncFunction.java
new file mode 100644
index 0000000..26a75ee
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/TruncFunction.java
@@ -0,0 +1,65 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.parse.FloorParseNode;
+import org.apache.phoenix.parse.FunctionParseNode.Argument;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.schema.PDataType;
+
+
+/**
+ * 
+ * Function used to bucketize date/time values by truncating them to
+ * an even increment.  Usage:
+ * TRUNC(<date/time col ref>,<'day'|'hour'|'minute'|'second'|'millisecond'>,[<optional integer multiplier>])
+ * The integer multiplier is optional and is used to do rollups to a partial time unit (i.e. 10 minute rollup)
+ * The function returns a {@link org.apache.phoenix.schema.PDataType#DATE}
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+@BuiltInFunction(name = TruncFunction.NAME,
+nodeClass = FloorParseNode.class,
+args = {
+       @Argument(allowedTypes={PDataType.TIMESTAMP, PDataType.DECIMAL}),
+       @Argument(allowedTypes={PDataType.VARCHAR, PDataType.INTEGER}, defaultValue = "null", isConstant=true),
+       @Argument(allowedTypes={PDataType.INTEGER}, defaultValue="1", isConstant=true)
+       } 
+)
+public abstract class TruncFunction extends ScalarFunction {
+    
+    public static final String NAME = "TRUNC";
+    
+    public TruncFunction(List<Expression> children) throws SQLException {
+        super(children);
+    }
+
+    @Override
+    public String getName() {
+        return NAME;
+    }
+    
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..a524617
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/BaseExpressionVisitor.java
@@ -0,0 +1,264 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.expression.visitor;
+
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.phoenix.expression.AddExpression;
+import org.apache.phoenix.expression.AndExpression;
+import org.apache.phoenix.expression.ArrayConstructorExpression;
+import org.apache.phoenix.expression.CaseExpression;
+import org.apache.phoenix.expression.CoerceExpression;
+import org.apache.phoenix.expression.ComparisonExpression;
+import org.apache.phoenix.expression.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;
+import org.apache.phoenix.expression.LikeExpression;
+import org.apache.phoenix.expression.LiteralExpression;
+import org.apache.phoenix.expression.MultiplyExpression;
+import org.apache.phoenix.expression.NotExpression;
+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.StringConcatExpression;
+import org.apache.phoenix.expression.SubtractExpression;
+import org.apache.phoenix.expression.function.ScalarFunction;
+import org.apache.phoenix.expression.function.SingleAggregateFunction;
+
+
+public abstract class BaseExpressionVisitor<E> implements ExpressionVisitor<E> {
+    @Override
+    public E visit(Expression node) {
+        return null;
+    }
+
+    @Override
+    public Iterator<Expression> visitEnter(Expression node) {
+        return null;
+    }
+
+    @Override
+    public E visitLeave(Expression node, List<E> l) {
+        return null;
+    }
+
+    @Override
+    public E defaultReturn(Expression node, List<E> l) {
+        return null;
+    }
+
+    @Override
+    public Iterator<Expression> visitEnter(AndExpression node) {
+        return null;
+    }
+
+    @Override
+    public E visitLeave(AndExpression node, List<E> l) {
+        return null;
+    }
+
+    @Override
+    public Iterator<Expression> visitEnter(OrExpression node) {
+        return null;
+    }
+
+    @Override
+    public E visitLeave(OrExpression node, List<E> l) {
+        return null;
+    }
+
+    @Override
+    public Iterator<Expression> visitEnter(ScalarFunction node) {
+        return null;
+    }
+
+    @Override
+    public E visitLeave(ScalarFunction node, List<E> l) {
+        return null;
+    }
+
+    @Override
+    public Iterator<Expression> visitEnter(ComparisonExpression node) {
+        return null;
+    }
+
+    @Override
+    public E visitLeave(ComparisonExpression node, List<E> l) {
+        return null;
+    }
+
+    @Override
+    public Iterator<Expression> visitEnter(LikeExpression node) {
+        return null;
+    }
+
+    @Override
+    public E visitLeave(LikeExpression node, List<E> l) {
+        return null;
+    }
+
+    @Override
+    public Iterator<Expression> visitEnter(SingleAggregateFunction node) {
+        return null;
+    }
+
+    @Override
+    public E visitLeave(SingleAggregateFunction node, List<E> l) {
+        return null;
+    }
+
+    @Override
+    public Iterator<Expression> visitEnter(CaseExpression node) {
+        return null;
+    }
+
+    @Override
+    public E visitLeave(CaseExpression node, List<E> l) {
+        return null;
+    }
+
+    @Override
+    public Iterator<Expression> visitEnter(NotExpression node) {
+        return null;
+    }
+
+    @Override
+    public E visitLeave(NotExpression node, List<E> l) {
+        return null;
+    }
+
+    @Override
+    public Iterator<Expression> visitEnter(IsNullExpression node) {
+        return null;
+    }
+
+    @Override
+    public E visitLeave(IsNullExpression node, List<E> l) {
+        return null;
+    }
+
+    @Override
+    public Iterator<Expression> visitEnter(InListExpression node) {
+        return null;
+    }
+
+    @Override
+    public E visitLeave(InListExpression node, List<E> l) {
+        return null;
+    }
+
+    @Override
+    public E visit(LiteralExpression node) {
+        return null;
+    }
+
+    @Override
+    public E visit(RowKeyColumnExpression node) {
+        return null;
+    }
+
+    @Override
+    public E visit(KeyValueColumnExpression node) {
+        return null;
+    }
+    
+    @Override
+    public E visit(ProjectedColumnExpression node) {
+        return null;
+    }
+
+    @Override
+    public Iterator<Expression> visitEnter(SubtractExpression node) {
+        return null;
+    }
+
+    @Override
+    public E visitLeave(SubtractExpression node, List<E> l) {
+        return null;
+    }
+
+    @Override
+    public Iterator<Expression> visitEnter(AddExpression node) {
+        return null;
+    }
+    @Override
+    public E visitLeave(AddExpression node, List<E> l) {
+        return null;
+    }
+
+    @Override
+    public Iterator<Expression> visitEnter(MultiplyExpression node) {
+        return null;
+    }
+    @Override
+    public E visitLeave(MultiplyExpression node, List<E> l) {
+        return null;
+    }
+
+    @Override
+    public Iterator<Expression> visitEnter(DivideExpression node) {
+        return null;
+    }
+    @Override
+    public E visitLeave(DivideExpression node, List<E> l) {
+        return null;
+    }
+    
+    @Override
+    public Iterator<Expression> visitEnter(StringConcatExpression node) {
+        return null;
+    }
+    @Override
+    public E visitLeave(StringConcatExpression node, List<E> l) {
+        return null;
+    }
+    
+    @Override
+    public Iterator<Expression> visitEnter(RowValueConstructorExpression node) {
+        return null;
+    }
+    @Override
+    public E visitLeave(RowValueConstructorExpression node, List<E> l) {
+        return null;
+    }
+    
+    @Override
+    public Iterator<Expression> visitEnter(CoerceExpression node) {
+        return null;
+    }
+    
+    @Override
+    public E visitLeave(CoerceExpression node, List<E> l) {
+        return null;
+    }
+
+    @Override
+    public Iterator<Expression> visitEnter(ArrayConstructorExpression node) {
+        return null;
+    }
+    @Override
+    public E visitLeave(ArrayConstructorExpression node, List<E> l) {
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..efde2a4
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/ExpressionVisitor.java
@@ -0,0 +1,142 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.expression.visitor;
+
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.phoenix.expression.AddExpression;
+import org.apache.phoenix.expression.AndExpression;
+import org.apache.phoenix.expression.ArrayConstructorExpression;
+import org.apache.phoenix.expression.CaseExpression;
+import org.apache.phoenix.expression.CoerceExpression;
+import org.apache.phoenix.expression.ComparisonExpression;
+import org.apache.phoenix.expression.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;
+import org.apache.phoenix.expression.LikeExpression;
+import org.apache.phoenix.expression.LiteralExpression;
+import org.apache.phoenix.expression.MultiplyExpression;
+import org.apache.phoenix.expression.NotExpression;
+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.StringConcatExpression;
+import org.apache.phoenix.expression.SubtractExpression;
+import org.apache.phoenix.expression.function.ScalarFunction;
+import org.apache.phoenix.expression.function.SingleAggregateFunction;
+
+
+/**
+ * 
+ * Visitor for an expression (which may contain other nested expressions)
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public interface ExpressionVisitor<E> {
+    /**
+     * Default visit method when an expression subclass doesn't
+     * define an accept method of its own. This will end up calling
+     * the {@link #defaultIterator(Expression)} to iterate over the
+     * children calling accept on them
+     */
+    public E visit(Expression node);
+    /**
+     * Default visitEnter method when an expression subclass doesn't
+     * define an accept method of its own. This will end up calling
+     * the {@link #defaultIterator(Expression)} to iterate over the
+     * children calling accept on them
+     */
+    public Iterator<Expression> visitEnter(Expression node);
+    /**
+     * Default visitLeave method when an expression subclass doesn't
+     * define an accept method of its own.  This will end up calling
+     * the {@link #defaultReturn(Expression, List)} with the list from
+     * the iteration over the children.
+     */
+    public E visitLeave(Expression node, List<E> l);
+
+    public E defaultReturn(Expression node, List<E> l);
+    public Iterator<Expression> defaultIterator(Expression node);
+    
+    public Iterator<Expression> visitEnter(AndExpression node);
+    public E visitLeave(AndExpression node, List<E> l);
+    
+    public Iterator<Expression> visitEnter(OrExpression node);
+    public E visitLeave(OrExpression node, List<E> l);
+    
+    public Iterator<Expression> visitEnter(ScalarFunction node);
+    public E visitLeave(ScalarFunction node, List<E> l);
+    
+    public Iterator<Expression> visitEnter(ComparisonExpression node);
+    public E visitLeave(ComparisonExpression node, List<E> l);
+
+    public Iterator<Expression> visitEnter(LikeExpression node);
+    public E visitLeave(LikeExpression node, List<E> l);
+    
+    public Iterator<Expression> visitEnter(SingleAggregateFunction node);
+    public E visitLeave(SingleAggregateFunction node, List<E> l);
+    
+    public Iterator<Expression> visitEnter(CaseExpression node);
+    public E visitLeave(CaseExpression node, List<E> l);
+    
+    public Iterator<Expression> visitEnter(NotExpression node);
+    public E visitLeave(NotExpression node, List<E> l);
+
+    public Iterator<Expression> visitEnter(InListExpression node);
+    public E visitLeave(InListExpression node, List<E> l);
+
+    public Iterator<Expression> visitEnter(IsNullExpression node);
+    public E visitLeave(IsNullExpression node, List<E> l);
+
+    public Iterator<Expression> visitEnter(SubtractExpression node);
+    public E visitLeave(SubtractExpression node, List<E> l);
+    
+    public Iterator<Expression> visitEnter(MultiplyExpression node);
+    public E visitLeave(MultiplyExpression node, List<E> l);
+    
+    public Iterator<Expression> visitEnter(AddExpression node);
+    public E visitLeave(AddExpression node, List<E> l);
+    
+    public Iterator<Expression> visitEnter(DivideExpression node);
+    public E visitLeave(DivideExpression node, List<E> l);
+    
+    public Iterator<Expression> visitEnter(CoerceExpression node);
+    public E visitLeave(CoerceExpression node, List<E> l);
+
+    public Iterator<Expression> visitEnter(ArrayConstructorExpression node);
+    public E visitLeave(ArrayConstructorExpression node, List<E> l);
+    
+    public E visit(LiteralExpression node);
+    public E visit(RowKeyColumnExpression node);
+    public E visit(KeyValueColumnExpression node);
+    public E visit(ProjectedColumnExpression node);
+    
+	public Iterator<Expression> visitEnter(StringConcatExpression node);
+	public E visitLeave(StringConcatExpression node, List<E> l);
+	
+	public Iterator<Expression> visitEnter(RowValueConstructorExpression node);
+    public E visitLeave(RowValueConstructorExpression node, List<E> l);
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/KeyValueExpressionVisitor.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/KeyValueExpressionVisitor.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/KeyValueExpressionVisitor.java
new file mode 100644
index 0000000..838bb0b
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/KeyValueExpressionVisitor.java
@@ -0,0 +1,38 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.expression.visitor;
+
+import org.apache.phoenix.expression.KeyValueColumnExpression;
+
+
+
+
+/**
+ * 
+ * Implementation of ExpressionVisitor where only KeyValueDataAccessor
+ * is being visited
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public abstract class KeyValueExpressionVisitor extends TraverseAllExpressionVisitor<Void> {
+    @Override
+    abstract public Void visit(KeyValueColumnExpression node);
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/SingleAggregateFunctionVisitor.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/SingleAggregateFunctionVisitor.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/SingleAggregateFunctionVisitor.java
new file mode 100644
index 0000000..63313a7
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/SingleAggregateFunctionVisitor.java
@@ -0,0 +1,40 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.expression.visitor;
+
+import java.util.Iterator;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.function.SingleAggregateFunction;
+
+
+
+/**
+ * 
+ * Implementation of ExpressionVisitor where only SingleAggregateFunction
+ * instances are visited
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public abstract class SingleAggregateFunctionVisitor extends TraverseAllExpressionVisitor<Void> {
+    @Override
+    abstract public Iterator<Expression> visitEnter(SingleAggregateFunction node);
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/TraverseAllExpressionVisitor.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/TraverseAllExpressionVisitor.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/TraverseAllExpressionVisitor.java
new file mode 100644
index 0000000..39e7a3d
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/TraverseAllExpressionVisitor.java
@@ -0,0 +1,57 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.expression.visitor;
+
+import java.util.*;
+
+import org.apache.phoenix.expression.Expression;
+
+
+
+
+public class TraverseAllExpressionVisitor<E> extends BaseExpressionVisitor<E> {
+
+    @Override
+    public Iterator<Expression> defaultIterator(Expression node) {
+        final List<Expression> children = node.getChildren();
+        return new Iterator<Expression>() {
+            private int position;
+            
+            @Override
+            public final boolean hasNext() {
+                return position < children.size();
+            }
+
+            @Override
+            public final Expression next() {
+                if (!hasNext()) {
+                    throw new NoSuchElementException();
+                }
+                return children.get(position++);
+            }
+
+            @Override
+            public final void remove() {
+                throw new UnsupportedOperationException();
+            }
+        };
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/TraverseNoExpressionVisitor.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/TraverseNoExpressionVisitor.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/TraverseNoExpressionVisitor.java
new file mode 100644
index 0000000..fe1d1f5
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/TraverseNoExpressionVisitor.java
@@ -0,0 +1,35 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.expression.visitor;
+
+import java.util.Iterator;
+
+
+import com.google.common.collect.Iterators;
+import org.apache.phoenix.expression.Expression;
+
+public class TraverseNoExpressionVisitor<E> extends BaseExpressionVisitor<E> {
+
+    @Override
+    public Iterator<Expression> defaultIterator(Expression node) {
+        return Iterators.emptyIterator();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/filter/BooleanExpressionFilter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/filter/BooleanExpressionFilter.java b/phoenix-core/src/main/java/org/apache/phoenix/filter/BooleanExpressionFilter.java
new file mode 100644
index 0000000..d7b2243
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/filter/BooleanExpressionFilter.java
@@ -0,0 +1,125 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.filter.FilterBase;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.ExpressionType;
+import org.apache.phoenix.schema.IllegalDataException;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.ServerUtil;
+
+
+/**
+ * 
+ * Base class for filter that evaluates a WHERE clause expression.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+abstract public class BooleanExpressionFilter extends FilterBase {
+
+    protected Expression expression;
+    protected boolean evaluateOnCompletion;
+    private ImmutableBytesWritable tempPtr = new ImmutableBytesWritable();
+    
+    public BooleanExpressionFilter() {
+    }
+
+    public BooleanExpressionFilter(Expression expression) {
+        this.expression = expression;
+    }
+
+    protected void setEvaluateOnCompletion(boolean evaluateOnCompletion) {
+        this.evaluateOnCompletion = evaluateOnCompletion;
+    }
+    
+    protected boolean evaluateOnCompletion() {
+        return evaluateOnCompletion;
+    }
+    
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + expression.hashCode();
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) return true;
+        if (obj == null) return false;
+        if (getClass() != obj.getClass()) return false;
+        BooleanExpressionFilter other = (BooleanExpressionFilter)obj;
+        if (!expression.equals(other.expression)) return false;
+        return true;
+    }
+
+    @Override
+    public String toString() {
+        return expression.toString();
+    }
+
+    @edu.umd.cs.findbugs.annotations.SuppressWarnings(
+            value="NP_BOOLEAN_RETURN_NULL",
+            justification="Returns null by design.")
+    protected Boolean evaluate(Tuple input) {
+        try {
+            if (!expression.evaluate(input, tempPtr)) {
+                return null;
+            }
+        } catch (IllegalDataException e) {
+            return Boolean.FALSE;
+        }
+        return (Boolean)expression.getDataType().toObject(tempPtr);
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        try {
+            expression = ExpressionType.values()[WritableUtils.readVInt(input)].newInstance();
+            expression.readFields(input);
+        } catch (Throwable t) { // Catches incompatibilities during reading/writing and doesn't retry
+            ServerUtil.throwIOException("BooleanExpressionFilter failed during reading", t);
+        }
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+        try {
+            WritableUtils.writeVInt(output, ExpressionType.valueOf(expression).ordinal());
+            expression.write(output);
+        } catch (Throwable t) { // Catches incompatibilities during reading/writing and doesn't retry
+            ServerUtil.throwIOException("BooleanExpressionFilter failed during writing", t);
+        }
+    }
+    
+    @Override
+    public void reset() {
+        expression.reset();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/filter/EvaluateOnCompletionVisitor.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/filter/EvaluateOnCompletionVisitor.java b/phoenix-core/src/main/java/org/apache/phoenix/filter/EvaluateOnCompletionVisitor.java
new file mode 100644
index 0000000..7dcc954
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/filter/EvaluateOnCompletionVisitor.java
@@ -0,0 +1,83 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.util.Iterator;
+
+import org.apache.phoenix.expression.ArrayConstructorExpression;
+import org.apache.phoenix.expression.CaseExpression;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.IsNullExpression;
+import org.apache.phoenix.expression.RowKeyColumnExpression;
+import org.apache.phoenix.expression.RowValueConstructorExpression;
+import org.apache.phoenix.expression.visitor.TraverseAllExpressionVisitor;
+
+
+/**
+ * 
+ * Implementation of ExpressionVisitor for the expression used by the
+ * BooleanExpressionFilter that looks for expressions that need to be
+ * evaluated upon completion. Examples include:
+ * - CaseExpression with an else clause, since upon completion, the
+ * else clause would apply if the when clauses could not be evaluated
+ * due to the absense of a value.
+ * - IsNullExpression that's not negated, since upon completion, we
+ * know definitively that a column value was not found.
+ * - row key columns are used, since we may never have encountered a
+ * key value column of interest, but the expression may evaluate to true
+ * just based on the row key columns.
+ * @author jtaylor
+ * @since 0.1
+ */
+public class EvaluateOnCompletionVisitor extends TraverseAllExpressionVisitor<Void> {
+    private boolean evaluateOnCompletion = false;
+    
+    public boolean evaluateOnCompletion() {
+        return evaluateOnCompletion;
+    }
+    
+    @Override
+    public Iterator<Expression> visitEnter(IsNullExpression node) {
+        evaluateOnCompletion |= !node.isNegate();
+        return null;
+    }
+    @Override
+    public Iterator<Expression> visitEnter(CaseExpression node) {
+        evaluateOnCompletion |= node.hasElse();
+        return null;
+    }
+    @Override
+    public Void visit(RowKeyColumnExpression node) {
+        evaluateOnCompletion = true;
+        return null;
+    }
+    @Override
+    public Iterator<Expression> visitEnter(RowValueConstructorExpression node) {
+        evaluateOnCompletion = true;
+        return null;
+    }
+    
+    @Override
+    public Iterator<Expression> visitEnter(ArrayConstructorExpression node) {
+        evaluateOnCompletion = true;
+        return null;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/filter/MultiCFCQKeyValueComparisonFilter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/filter/MultiCFCQKeyValueComparisonFilter.java b/phoenix-core/src/main/java/org/apache/phoenix/filter/MultiCFCQKeyValueComparisonFilter.java
new file mode 100644
index 0000000..27a356b
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/filter/MultiCFCQKeyValueComparisonFilter.java
@@ -0,0 +1,134 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.util.TreeSet;
+
+import org.apache.hadoop.hbase.util.Bytes;
+
+import org.apache.phoenix.expression.Expression;
+
+
+/**
+ *
+ * Filter that evaluates WHERE clause expression, used in the case where there
+ * are references to multiple column qualifiers over multiple column families.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class MultiCFCQKeyValueComparisonFilter extends MultiKeyValueComparisonFilter {
+    private final ImmutablePairBytesPtr ptr = new ImmutablePairBytesPtr();
+    private TreeSet<byte[]> cfSet;
+
+    public MultiCFCQKeyValueComparisonFilter() {
+    }
+
+    public MultiCFCQKeyValueComparisonFilter(Expression expression) {
+        super(expression);
+    }
+
+    @Override
+    protected void init() {
+        cfSet = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
+        super.init();
+    }
+
+    @Override
+    protected Object setColumnKey(byte[] cf, int cfOffset, int cfLength,
+            byte[] cq, int cqOffset, int cqLength) {
+        ptr.set(cf, cfOffset, cfLength, cq, cqOffset, cqLength);
+        return ptr;
+    }
+
+    @Override
+    protected Object newColumnKey(byte[] cf, int cfOffset, int cfLength,
+            byte[] cq, int cqOffset, int cqLength) {
+
+        byte[] cfKey;
+        if (cfOffset == 0 && cf.length == cfLength) {
+            cfKey = cf;
+        } else {
+            // Copy bytes here, but figure cf names are typically a few bytes at most,
+            // so this will be better than creating an ImmutableBytesPtr
+            cfKey = new byte[cfLength];
+            System.arraycopy(cf, cfOffset, cfKey, 0, cfLength);
+        }
+        cfSet.add(cfKey);
+        return new ImmutablePairBytesPtr(cf, cfOffset, cfLength, cq, cqOffset, cqLength);
+    }
+
+    private static class ImmutablePairBytesPtr {
+        private byte[] bytes1;
+        private int offset1;
+        private int length1;
+        private byte[] bytes2;
+        private int offset2;
+        private int length2;
+        private int hashCode;
+
+        private ImmutablePairBytesPtr() {
+        }
+
+        private ImmutablePairBytesPtr(byte[] bytes1, int offset1, int length1, byte[] bytes2, int offset2, int length2) {
+            set(bytes1, offset1, length1, bytes2, offset2, length2);
+        }
+
+        @Override
+        public int hashCode() {
+            return hashCode;
+        }
+
+        public void set(byte[] bytes1, int offset1, int length1, byte[] bytes2, int offset2, int length2) {
+            this.bytes1 = bytes1;
+            this.offset1 = offset1;
+            this.length1 = length1;
+            this.bytes2 = bytes2;
+            this.offset2 = offset2;
+            this.length2 = length2;
+            int hash = 1;
+            for (int i = offset1; i < offset1 + length1; i++)
+                hash = (31 * hash) + bytes1[i];
+            for (int i = offset2; i < offset2 + length2; i++)
+                hash = (31 * hash) + bytes2[i];
+            hashCode = hash;
+        }
+
+        @Override
+        public boolean equals(Object obj) {
+            if (this == obj) return true;
+            if (obj == null) return false;
+            if (getClass() != obj.getClass()) return false;
+            ImmutablePairBytesPtr that = (ImmutablePairBytesPtr)obj;
+            if (this.hashCode != that.hashCode) return false;
+            if (Bytes.compareTo(this.bytes2, this.offset2, this.length2, that.bytes2, that.offset2, that.length2) != 0) return false;
+            if (Bytes.compareTo(this.bytes1, this.offset1, this.length1, that.bytes1, that.offset1, that.length1) != 0) return false;
+            return true;
+        }
+    }
+
+
+    @SuppressWarnings("all") // suppressing missing @Override since this doesn't exist for HBase 0.94.4
+    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/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/filter/MultiCQKeyValueComparisonFilter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/filter/MultiCQKeyValueComparisonFilter.java b/phoenix-core/src/main/java/org/apache/phoenix/filter/MultiCQKeyValueComparisonFilter.java
new file mode 100644
index 0000000..c66b788
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/filter/MultiCQKeyValueComparisonFilter.java
@@ -0,0 +1,70 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 org.apache.hadoop.hbase.util.Bytes;
+
+import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.expression.Expression;
+
+/**
+ *
+ * Filter that evaluates WHERE clause expression, used in the case where there
+ * are references to multiple column qualifiers over a single column family.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class MultiCQKeyValueComparisonFilter extends MultiKeyValueComparisonFilter {
+    private ImmutableBytesPtr ptr = new ImmutableBytesPtr();
+    private byte[] cf;
+
+    public MultiCQKeyValueComparisonFilter() {
+    }
+
+    public MultiCQKeyValueComparisonFilter(Expression expression) {
+        super(expression);
+    }
+
+    @Override
+    protected Object setColumnKey(byte[] cf, int cfOffset, int cfLength, byte[] cq, int cqOffset,
+            int cqLength) {
+        ptr.set(cq, cqOffset, cqLength);
+        return ptr;
+    }
+
+    @Override
+    protected Object newColumnKey(byte[] cf, int cfOffset, int cfLength, byte[] cq, int cqOffset,
+            int cqLength) {
+        if (cfOffset == 0 && cf.length == cfLength) {
+            this.cf = cf;
+        } else {
+            this.cf = new byte[cfLength];
+            System.arraycopy(cf, cfOffset, this.cf, 0, cfLength);
+        }
+        return new ImmutableBytesPtr(cq, cqOffset, cqLength);
+    }
+
+
+    @SuppressWarnings("all") // suppressing missing @Override since this doesn't exist for HBase 0.94.4
+    public boolean isFamilyEssential(byte[] name) {
+        return Bytes.compareTo(cf, name) == 0;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..ab8e1c0
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/filter/MultiKeyValueComparisonFilter.java
@@ -0,0 +1,247 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.io.DataInput;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.KeyValueColumnExpression;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+
+/**
+ * 
+ * Modeled after {@link org.apache.hadoop.hbase.filter.SingleColumnValueFilter},
+ * but for general expression evaluation in the case where multiple KeyValue
+ * columns are referenced in the expression.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public abstract class MultiKeyValueComparisonFilter extends BooleanExpressionFilter {
+    private static final byte[] UNITIALIZED_KEY_BUFFER = new byte[0];
+
+    private Boolean matchedColumn;
+    protected final IncrementalResultTuple inputTuple = new IncrementalResultTuple();
+
+    public MultiKeyValueComparisonFilter() {
+    }
+
+    public MultiKeyValueComparisonFilter(Expression expression) {
+        super(expression);
+        init();
+    }
+
+    private static final class KeyValueRef {
+        public KeyValue keyValue;
+        
+        @Override
+        public String toString() {
+            if(keyValue != null) {
+                return keyValue.toString() + " value = " + Bytes.toStringBinary(keyValue.getValue());
+            } else {
+                return super.toString();
+            }
+        }
+    }
+    
+    protected abstract Object setColumnKey(byte[] cf, int cfOffset, int cfLength, byte[] cq, int cqOffset, int cqLength);
+    protected abstract Object newColumnKey(byte[] cf, int cfOffset, int cfLength, byte[] cq, int cqOffset, int cqLength);
+    
+    private final class IncrementalResultTuple implements Tuple {
+        private int refCount;
+        private final ImmutableBytesWritable keyPtr = new ImmutableBytesWritable(UNITIALIZED_KEY_BUFFER);
+        private final Map<Object,KeyValueRef> foundColumns = new HashMap<Object,KeyValueRef>(5);
+        
+        public void reset() {
+            refCount = 0;
+            keyPtr.set(UNITIALIZED_KEY_BUFFER);
+            for (KeyValueRef ref : foundColumns.values()) {
+                ref.keyValue = null;
+            }
+        }
+        
+        @Override
+        public boolean isImmutable() {
+            return refCount == foundColumns.size();
+        }
+        
+        public void setImmutable() {
+            refCount = foundColumns.size();
+        }
+        
+        public ReturnCode resolveColumn(KeyValue value) {
+            // Always set key, in case we never find a key value column of interest,
+            // and our expression uses row key columns.
+            setKey(value);
+            byte[] buf = value.getBuffer();
+            Object ptr = setColumnKey(buf, value.getFamilyOffset(), value.getFamilyLength(), buf, value.getQualifierOffset(), value.getQualifierLength());
+            KeyValueRef ref = foundColumns.get(ptr);
+            if (ref == null) {
+                // Return INCLUDE here. Although this filter doesn't need this KV
+                // it should still be projected into the Result
+                return ReturnCode.INCLUDE;
+            }
+            // Since we only look at the latest key value for a given column,
+            // we are not interested in older versions
+            // TODO: test with older versions to confirm this doesn't get tripped
+            // This shouldn't be necessary, because a scan only looks at the latest
+            // version
+            if (ref.keyValue != null) {
+                // Can't do NEXT_ROW, because then we don't match the other columns
+                // SKIP, INCLUDE, and NEXT_COL seem to all act the same
+                return ReturnCode.NEXT_COL;
+            }
+            ref.keyValue = value;
+            refCount++;
+            return null;
+        }
+        
+        public void addColumn(byte[] cf, byte[] cq) {
+            Object ptr = MultiKeyValueComparisonFilter.this.newColumnKey(cf, 0, cf.length, cq, 0, cq.length);
+            foundColumns.put(ptr, new KeyValueRef());
+        }
+        
+        public void setKey(KeyValue value) {
+            keyPtr.set(value.getBuffer(), value.getRowOffset(), value.getRowLength());
+        }
+        
+        @Override
+        public void getKey(ImmutableBytesWritable ptr) {
+            ptr.set(keyPtr.get(),keyPtr.getOffset(),keyPtr.getLength());
+        }
+        
+        @Override
+        public KeyValue getValue(byte[] cf, byte[] cq) {
+            Object ptr = setColumnKey(cf, 0, cf.length, cq, 0, cq.length);
+            KeyValueRef ref = foundColumns.get(ptr);
+            return ref == null ? null : ref.keyValue;
+        }
+        
+        @Override
+        public String toString() {
+            return foundColumns.toString();
+        }
+
+        @Override
+        public int size() {
+            return refCount;
+        }
+
+        @Override
+        public KeyValue getValue(int index) {
+            // This won't perform very well, but it's not
+            // currently used anyway
+            for (KeyValueRef ref : foundColumns.values()) {
+                if (ref.keyValue == null) {
+                    continue;
+                }
+                if (index == 0) {
+                    return ref.keyValue;
+                }
+                index--;
+            }
+            throw new IndexOutOfBoundsException(Integer.toString(index));
+        }
+
+        @Override
+        public boolean getValue(byte[] family, byte[] qualifier,
+                ImmutableBytesWritable ptr) {
+            KeyValue kv = getValue(family, qualifier);
+            if (kv == null)
+                return false;
+            ptr.set(kv.getBuffer(), kv.getValueOffset(), kv.getValueLength());
+            return true;
+        }
+    }
+    
+    protected void init() {
+        EvaluateOnCompletionVisitor visitor = new EvaluateOnCompletionVisitor() {
+            @Override
+            public Void visit(KeyValueColumnExpression expression) {
+                inputTuple.addColumn(expression.getColumnFamily(), expression.getColumnName());
+                return null;
+            }
+        };
+        expression.accept(visitor);
+        this.evaluateOnCompletion = visitor.evaluateOnCompletion();
+        expression.reset();
+    }
+    
+    @Override
+    public ReturnCode filterKeyValue(KeyValue keyValue) {
+        if (Boolean.TRUE.equals(this.matchedColumn)) {
+          // We already found and matched the single column, all keys now pass
+          return ReturnCode.INCLUDE;
+        }
+        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;
+        }
+        // This is a key value we're not interested in (TODO: why INCLUDE here instead of NEXT_COL?)
+        ReturnCode code = inputTuple.resolveColumn(keyValue);
+        if (code != null) {
+            return code;
+        }
+        
+        // We found a new column, so we can re-evaluate
+        // TODO: if we have row key columns in our expression, should
+        // we always evaluate or just wait until the end?
+        this.matchedColumn = this.evaluate(inputTuple);
+        if (this.matchedColumn == null) {
+            if (inputTuple.isImmutable()) {
+                this.matchedColumn = Boolean.FALSE;
+            } else {
+                return ReturnCode.INCLUDE;
+            }
+        }
+        return this.matchedColumn ? ReturnCode.INCLUDE : ReturnCode.NEXT_ROW;
+    }
+
+    @Override
+    public boolean filterRow() {
+        if (this.matchedColumn == null && !inputTuple.isImmutable() && evaluateOnCompletion()) {
+            inputTuple.setImmutable();
+            this.matchedColumn = this.evaluate(inputTuple);
+        }
+        
+        return ! (Boolean.TRUE.equals(this.matchedColumn));
+    }
+
+      @Override
+    public void reset() {
+        matchedColumn = null;
+        inputTuple.reset();
+        super.reset();
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        super.readFields(input);
+        init();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/filter/RowKeyComparisonFilter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/filter/RowKeyComparisonFilter.java b/phoenix-core/src/main/java/org/apache/phoenix/filter/RowKeyComparisonFilter.java
new file mode 100644
index 0000000..bd47a8c
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/filter/RowKeyComparisonFilter.java
@@ -0,0 +1,156 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.WritableUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+
+/**
+ *
+ * Filter for use when expressions only reference row key columns
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class RowKeyComparisonFilter extends BooleanExpressionFilter {
+    private static final Logger logger = LoggerFactory.getLogger(RowKeyComparisonFilter.class);
+
+    private boolean evaluate = true;
+    private boolean keepRow = false;
+    private RowKeyTuple inputTuple = new RowKeyTuple();
+    private byte[] essentialCF;
+
+    public RowKeyComparisonFilter() {
+    }
+
+    public RowKeyComparisonFilter(Expression expression, byte[] essentialCF) {
+        super(expression);
+        this.essentialCF = essentialCF;
+    }
+
+    @Override
+    public void reset() {
+        this.keepRow = false;
+        this.evaluate = true;
+        super.reset();
+    }
+
+    /**
+     * Evaluate in filterKeyValue instead of filterRowKey, because HBASE-6562 causes filterRowKey
+     * to be called with deleted or partial row keys.
+     */
+    @Override
+    public ReturnCode filterKeyValue(KeyValue v) {
+        if (evaluate) {
+            inputTuple.setKey(v.getBuffer(), v.getRowOffset(), v.getRowLength());
+            this.keepRow = Boolean.TRUE.equals(evaluate(inputTuple));
+            if (logger.isDebugEnabled()) {
+                logger.debug("RowKeyComparisonFilter: " + (this.keepRow ? "KEEP" : "FILTER")  + " row " + inputTuple);
+            }
+            evaluate = false;
+        }
+        return keepRow ? ReturnCode.INCLUDE : ReturnCode.NEXT_ROW;
+    }
+
+    private final class RowKeyTuple implements Tuple {
+        private byte[] buf;
+        private int offset;
+        private int length;
+
+        public void setKey(byte[] buf, int offset, int length) {
+            this.buf = buf;
+            this.offset = offset;
+            this.length = length;
+        }
+
+        @Override
+        public void getKey(ImmutableBytesWritable ptr) {
+            ptr.set(buf, offset, length);
+        }
+
+        @Override
+        public KeyValue getValue(byte[] cf, byte[] cq) {
+            return null;
+        }
+
+        @Override
+        public boolean isImmutable() {
+            return true;
+        }
+
+        @Override
+        public String toString() {
+            return Bytes.toStringBinary(buf, offset, length);
+        }
+
+        @Override
+        public int size() {
+            return 0;
+        }
+
+        @Override
+        public KeyValue getValue(int index) {
+            throw new IndexOutOfBoundsException(Integer.toString(index));
+        }
+
+        @Override
+        public boolean getValue(byte[] family, byte[] qualifier,
+                ImmutableBytesWritable ptr) {
+            return false;
+        }
+    }
+
+    @Override
+    public boolean filterRow() {
+        return !this.keepRow;
+    }
+
+    @SuppressWarnings("all") // suppressing missing @Override since this doesn't exist for HBase 0.94.4
+    public boolean isFamilyEssential(byte[] name) {
+        // We only need our "guaranteed to have a key value" column family,
+        // which we pass in and serialize through. In the case of a VIEW where
+        // we don't have this, we have to say that all families are essential.
+        return this.essentialCF.length == 0 ? true : Bytes.compareTo(this.essentialCF, name) == 0;
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        super.readFields(input);
+        this.essentialCF = WritableUtils.readCompressedByteArray(input);
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+        super.write(output);
+        WritableUtils.writeCompressedByteArray(output, this.essentialCF);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/filter/SingleCFCQKeyValueComparisonFilter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/filter/SingleCFCQKeyValueComparisonFilter.java b/phoenix-core/src/main/java/org/apache/phoenix/filter/SingleCFCQKeyValueComparisonFilter.java
new file mode 100644
index 0000000..5b8a5f0
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/filter/SingleCFCQKeyValueComparisonFilter.java
@@ -0,0 +1,51 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 org.apache.hadoop.hbase.util.Bytes;
+
+import org.apache.phoenix.expression.Expression;
+
+
+/**
+ * 
+ * SingleKeyValueComparisonFilter that needs to compare both the column family and
+ * column qualifier parts of the key value to disambiguate with another similarly
+ * named column qualifier in a different column family.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class SingleCFCQKeyValueComparisonFilter extends SingleKeyValueComparisonFilter {
+    public SingleCFCQKeyValueComparisonFilter() {
+    }
+
+    public SingleCFCQKeyValueComparisonFilter(Expression expression) {
+        super(expression);
+    }
+
+    @Override
+    protected final int compare(byte[] cfBuf, int cfOffset, int cfLength, byte[] cqBuf, int cqOffset, int cqLength) {
+        int c = Bytes.compareTo(cf, 0, cf.length, cfBuf, cfOffset, cfLength);
+        if (c != 0) return c;
+        return Bytes.compareTo(cq, 0, cq.length, cqBuf, cqOffset, cqLength);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..425839a
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/filter/SingleCQKeyValueComparisonFilter.java
@@ -0,0 +1,49 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 org.apache.hadoop.hbase.util.Bytes;
+
+import org.apache.phoenix.expression.Expression;
+
+
+/**
+ * 
+ * SingleKeyValueComparisonFilter that needs to only compare the column qualifier
+ * part of the key value since the column qualifier is unique across all column
+ * families.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class SingleCQKeyValueComparisonFilter extends SingleKeyValueComparisonFilter {
+    public SingleCQKeyValueComparisonFilter() {
+    }
+
+    public SingleCQKeyValueComparisonFilter(Expression expression) {
+        super(expression);
+    }
+
+    @Override
+    protected final int compare(byte[] cfBuf, int cfOffset, int cfLength, byte[] cqBuf, int cqOffset, int cqLength) {
+        return Bytes.compareTo(cq, 0, cq.length, cqBuf, cqOffset, cqLength);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..1caa332
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/filter/SingleKeyValueComparisonFilter.java
@@ -0,0 +1,147 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.io.DataInput;
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.KeyValueColumnExpression;
+import org.apache.phoenix.schema.tuple.SingleKeyValueTuple;
+
+
+
+/**
+ *
+ * Modeled after {@link org.apache.hadoop.hbase.filter.SingleColumnValueFilter},
+ * but for general expression evaluation in the case where only a single KeyValue
+ * column is referenced in the expression.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public abstract class SingleKeyValueComparisonFilter extends BooleanExpressionFilter {
+    private final SingleKeyValueTuple inputTuple = new SingleKeyValueTuple();
+    private boolean matchedColumn;
+    protected byte[] cf;
+    protected byte[] cq;
+
+    public SingleKeyValueComparisonFilter() {
+    }
+
+    public SingleKeyValueComparisonFilter(Expression expression) {
+        super(expression);
+        init();
+    }
+
+    protected abstract int compare(byte[] cfBuf, int cfOffset, int cfLength, byte[] cqBuf, int cqOffset, int cqLength);
+
+    private void init() {
+        EvaluateOnCompletionVisitor visitor = new EvaluateOnCompletionVisitor() {
+            @Override
+            public Void visit(KeyValueColumnExpression expression) {
+                cf = expression.getColumnFamily();
+                cq = expression.getColumnName();
+                return null;
+            }
+        };
+        expression.accept(visitor);
+        this.evaluateOnCompletion = visitor.evaluateOnCompletion();
+    }
+
+    private boolean foundColumn() {
+        return inputTuple.size() > 0;
+    }
+
+    @Override
+    public ReturnCode filterKeyValue(KeyValue keyValue) {
+        if (this.matchedColumn) {
+          // We already found and matched the single column, all keys now pass
+          // TODO: why won't this cause earlier versions of a kv to be included?
+          return ReturnCode.INCLUDE;
+        }
+        if (this.foundColumn()) {
+          // We found all the columns, but did not match the expression, so skip to next row
+          return ReturnCode.NEXT_ROW;
+        }
+        byte[] buf = keyValue.getBuffer();
+        if (compare(buf, keyValue.getFamilyOffset(), keyValue.getFamilyLength(), buf, keyValue.getQualifierOffset(), keyValue.getQualifierLength()) != 0) {
+            // Remember the key in case this is the only key value we see.
+            // We'll need it if we have row key columns too.
+            inputTuple.setKey(keyValue);
+            // This is a key value we're not interested in
+            // TODO: use NEXT_COL when bug fix comes through that includes the row still
+            return ReturnCode.INCLUDE;
+        }
+        inputTuple.setKeyValue(keyValue);
+
+        // We have the columns, so evaluate here
+        if (!Boolean.TRUE.equals(evaluate(inputTuple))) {
+            return ReturnCode.NEXT_ROW;
+        }
+        this.matchedColumn = true;
+        return ReturnCode.INCLUDE;
+    }
+
+    @Override
+    public boolean filterRow() {
+        // If column was found, return false if it was matched, true if it was not.
+        if (foundColumn()) {
+            return !this.matchedColumn;
+        }
+        // If column was not found, evaluate the expression here upon completion.
+        // This is required with certain expressions, for example, with IS NULL
+        // expressions where they'll evaluate to TRUE when the column being
+        // tested wasn't found.
+        // Since the filter is called also to position the scan initially, we have
+        // to guard against this by checking whether or not we've filtered in
+        // the key value (i.e. filterKeyValue was called and we found the keyValue
+        // for which we're looking).
+        if (inputTuple.hasKey() && evaluateOnCompletion()) {
+            return !Boolean.TRUE.equals(evaluate(inputTuple));
+        }
+        // Finally, if we have no values, and we're not required to re-evaluate it
+        // just filter the row
+        return true;
+    }
+
+      @Override
+    public void reset() {
+        inputTuple.reset();
+        matchedColumn = false;
+        super.reset();
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        super.readFields(input);
+        init();
+    }
+
+    @SuppressWarnings("all") // suppressing missing @Override since this doesn't exist for HBase 0.94.4
+    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 Bytes.compareTo(cf, name) == 0;
+    }
+}


[14/51] [partial] Initial commit of master branch from github

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
new file mode 100644
index 0000000..0649daf
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
@@ -0,0 +1,227 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.phoenix.client.KeyValueBuilder;
+import org.apache.phoenix.compile.MutationPlan;
+import org.apache.phoenix.coprocessor.MetaDataProtocol.MetaDataMutationResult;
+import org.apache.phoenix.execute.MutationState;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PMetaData;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.schema.SequenceKey;
+
+
+public class DelegateConnectionQueryServices extends DelegateQueryServices implements ConnectionQueryServices {
+
+    public DelegateConnectionQueryServices(ConnectionQueryServices delegate) {
+        super(delegate);
+    }
+    
+    @Override
+    protected ConnectionQueryServices getDelegate() {
+        return (ConnectionQueryServices)super.getDelegate();
+    }
+    
+    @Override
+    public ConnectionQueryServices getChildQueryServices(ImmutableBytesWritable tenantId) {
+        return getDelegate().getChildQueryServices(tenantId);
+    }
+
+    @Override
+    public HTableInterface getTable(byte[] tableName) throws SQLException {
+        return getDelegate().getTable(tableName);
+    }
+
+    @Override
+    public StatsManager getStatsManager() {
+        return getDelegate().getStatsManager();
+    }
+
+    @Override
+    public List<HRegionLocation> getAllTableRegions(byte[] tableName) throws SQLException {
+        return getDelegate().getAllTableRegions(tableName);
+    }
+
+    @Override
+    public PMetaData addTable(PTable table) throws SQLException {
+        return getDelegate().addTable(table);
+    }
+
+    @Override
+    public PMetaData addColumn(String tableName, List<PColumn> columns, long tableTimeStamp, long tableSeqNum,
+            boolean isImmutableRows) throws SQLException {
+        return getDelegate().addColumn(tableName, columns, tableTimeStamp, tableSeqNum, isImmutableRows);
+    }
+
+    @Override
+    public PMetaData removeTable(String tableName)
+            throws SQLException {
+        return getDelegate().removeTable(tableName);
+    }
+
+    @Override
+    public PMetaData removeColumn(String tableName, String familyName, String columnName, long tableTimeStamp,
+            long tableSeqNum) throws SQLException {
+        return getDelegate().removeColumn(tableName, familyName, columnName, tableTimeStamp, tableSeqNum);
+    }
+
+    @Override
+    public PhoenixConnection connect(String url, Properties info) throws SQLException {
+        return getDelegate().connect(url, info);
+    }
+
+    @Override
+    public MetaDataMutationResult getTable(byte[] tenantId, byte[] schemaBytes, byte[] tableBytes, long tableTimestamp, long clientTimestamp) throws SQLException {
+        return getDelegate().getTable(tenantId, schemaBytes, tableBytes, tableTimestamp, clientTimestamp);
+    }
+
+    @Override
+    public MetaDataMutationResult createTable(List<Mutation> tableMetaData, byte[] physicalName,
+            PTableType tableType, Map<String, Object> tableProps, List<Pair<byte[], Map<String, Object>>> families, byte[][] splits)
+            throws SQLException {
+        return getDelegate().createTable(tableMetaData, physicalName, tableType, tableProps, families, splits);
+    }
+
+    @Override
+    public MetaDataMutationResult dropTable(List<Mutation> tabeMetaData, PTableType tableType) throws SQLException {
+        return getDelegate().dropTable(tabeMetaData, tableType);
+    }
+
+    @Override
+    public MetaDataMutationResult addColumn(List<Mutation> tabeMetaData, PTableType tableType, List<Pair<byte[],Map<String,Object>>> families ) throws SQLException {
+        return getDelegate().addColumn(tabeMetaData, tableType, families);
+    }
+
+
+    @Override
+    public MetaDataMutationResult dropColumn(List<Mutation> tabeMetaData, PTableType tableType) throws SQLException {
+        return getDelegate().dropColumn(tabeMetaData, tableType);
+    }
+
+    @Override
+    public MetaDataMutationResult updateIndexState(List<Mutation> tableMetadata, String parentTableName) throws SQLException {
+        return getDelegate().updateIndexState(tableMetadata, parentTableName);
+    }
+    
+    @Override
+    public void init(String url, Properties props) throws SQLException {
+        getDelegate().init(url, props);
+    }
+
+    @Override
+    public MutationState updateData(MutationPlan plan) throws SQLException {
+        return getDelegate().updateData(plan);
+    }
+
+    @Override
+    public int getLowestClusterHBaseVersion() {
+        return getDelegate().getLowestClusterHBaseVersion();
+    }
+
+    @Override
+    public HBaseAdmin getAdmin() throws SQLException {
+        return getDelegate().getAdmin();
+    }
+
+    @Override
+    public HTableDescriptor getTableDescriptor(byte[] tableName) throws SQLException {
+        return getDelegate().getTableDescriptor(tableName);
+    }
+
+    @Override
+    public void clearTableRegionCache(byte[] tableName) throws SQLException {
+        getDelegate().clearTableRegionCache(tableName);
+    }
+
+    @Override
+    public boolean hasInvalidIndexConfiguration() {
+        return getDelegate().hasInvalidIndexConfiguration();
+    }
+
+    @Override
+    public long createSequence(String tenantId, String schemaName, String sequenceName, long startWith,
+            long incrementBy, int batchSize, long timestamp) throws SQLException {
+        return getDelegate().createSequence(tenantId, schemaName, sequenceName, startWith, incrementBy, batchSize, timestamp);
+    }
+
+    @Override
+    public long dropSequence(String tenantId, String schemaName, String sequenceName, long timestamp)
+            throws SQLException {
+        return getDelegate().dropSequence(tenantId, schemaName, sequenceName, timestamp);
+    }
+
+    @Override
+    public void reserveSequenceValues(List<SequenceKey> sequenceKeys, long timestamp, long[] values,
+            SQLException[] exceptions) throws SQLException {
+        getDelegate().reserveSequenceValues(sequenceKeys, timestamp, values, exceptions);
+    }
+
+    @Override
+    public void incrementSequenceValues(List<SequenceKey> sequenceKeys, long timestamp, long[] values,
+            SQLException[] exceptions) throws SQLException {
+        getDelegate().incrementSequenceValues(sequenceKeys, timestamp, values, exceptions);
+    }
+
+    @Override
+    public long getSequenceValue(SequenceKey sequenceKey, long timestamp) throws SQLException {
+        return getDelegate().getSequenceValue(sequenceKey, timestamp);
+    }
+
+    @Override
+    public void returnSequenceValues(List<SequenceKey> sequenceKeys, long timestamp, SQLException[] exceptions)
+            throws SQLException {
+        getDelegate().returnSequenceValues(sequenceKeys, timestamp, exceptions);
+    }
+
+    @Override
+    public void addConnection(PhoenixConnection connection) throws SQLException {
+        getDelegate().addConnection(connection);
+    }
+
+    @Override
+    public void removeConnection(PhoenixConnection connection) throws SQLException {
+        getDelegate().removeConnection(connection);
+    }
+
+    @Override
+    public KeyValueBuilder getKeyValueBuilder() {
+        return getDelegate().getKeyValueBuilder();
+    }
+
+    @Override
+    public boolean supportsFeature(Feature feature) {
+        return getDelegate().supportsFeature(feature);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateQueryServices.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateQueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateQueryServices.java
new file mode 100644
index 0000000..6ddf5c5
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateQueryServices.java
@@ -0,0 +1,74 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.sql.SQLException;
+import java.util.concurrent.ExecutorService;
+
+import org.apache.phoenix.memory.MemoryManager;
+import org.apache.phoenix.optimize.QueryOptimizer;
+import org.apache.phoenix.util.ReadOnlyProps;
+
+
+
+/**
+ * 
+ * Class that delegates QueryService calls through to
+ * a parent QueryService.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class DelegateQueryServices implements QueryServices {
+    private final QueryServices parent;
+    
+    public DelegateQueryServices(QueryServices queryServices) {
+        parent = queryServices;
+    }
+    
+    protected QueryServices getDelegate() {
+        return parent;
+    }
+    
+    @Override
+    public ExecutorService getExecutor() {
+        return parent.getExecutor();
+    }
+
+    @Override
+    public MemoryManager getMemoryManager() {
+        return parent.getMemoryManager();
+    }
+
+    @Override
+    public void close() throws SQLException {
+        parent.close();
+    }
+
+    @Override
+    public ReadOnlyProps getProps() {
+        return parent.getProps();
+    }
+
+    @Override
+    public QueryOptimizer getOptimizer() {
+        return parent.getOptimizer();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/query/HBaseFactoryProvider.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/HBaseFactoryProvider.java b/phoenix-core/src/main/java/org/apache/phoenix/query/HBaseFactoryProvider.java
new file mode 100644
index 0000000..08e8575
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/HBaseFactoryProvider.java
@@ -0,0 +1,57 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 org.apache.phoenix.util.InstanceResolver;
+
+/**
+ * Manages factories that provide extension points for HBase.
+ * <p/>
+ * Dependent modules may register their own implementations of the following using {@link java.util.ServiceLoader}:
+ * <ul>
+ *     <li>{@link ConfigurationFactory}</li>
+ *     <li>{@link HTableFactory}</li>
+ *     <li> {@link HConnectionFactory} </li>
+ * </ul>
+ *
+ * If a custom implementation is not registered, the default implementations will be used.
+ *
+ * @author aaraujo
+ * @since 0.2
+ */
+public class HBaseFactoryProvider {
+
+    private static final HTableFactory DEFAULT_HTABLE_FACTORY = new HTableFactory.HTableFactoryImpl();
+    private static final HConnectionFactory DEFAULT_HCONNECTION_FACTORY =
+        new HConnectionFactory.HConnectionFactoryImpl();
+    private static final ConfigurationFactory DEFAULT_CONFIGURATION_FACTORY = new ConfigurationFactory.ConfigurationFactoryImpl();
+
+    public static HTableFactory getHTableFactory() {
+        return InstanceResolver.getSingleton(HTableFactory.class, DEFAULT_HTABLE_FACTORY);
+    }
+
+    public static HConnectionFactory getHConnectionFactory() {
+        return InstanceResolver.getSingleton(HConnectionFactory.class, DEFAULT_HCONNECTION_FACTORY);
+    }
+
+    public static ConfigurationFactory getConfigurationFactory() {
+        return InstanceResolver.getSingleton(ConfigurationFactory.class, DEFAULT_CONFIGURATION_FACTORY);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/query/HConnectionFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/HConnectionFactory.java b/phoenix-core/src/main/java/org/apache/phoenix/query/HConnectionFactory.java
new file mode 100644
index 0000000..d40c540
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/HConnectionFactory.java
@@ -0,0 +1,51 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ZooKeeperConnectionException;
+import org.apache.hadoop.hbase.client.HConnection;
+import org.apache.hadoop.hbase.client.HConnectionManager;
+
+/**
+ * Factory for creating {@link HConnection}
+ *
+ * @author ukuchibhotla
+ */
+public interface HConnectionFactory {
+
+    /**
+     * Creates HConnection to access HBase clusters.
+     * 
+     * @param configuration object
+     * @return A HConnection instance
+     */
+    HConnection createConnection(Configuration conf) throws ZooKeeperConnectionException;
+
+    /**
+     * Default implementation.  Uses standard HBase HConnections.
+     */
+    static class HConnectionFactoryImpl implements HConnectionFactory {
+        @Override
+        public HConnection createConnection(Configuration conf) throws ZooKeeperConnectionException {
+            return HConnectionManager.createConnection(conf);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/query/HTableFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/HTableFactory.java b/phoenix-core/src/main/java/org/apache/phoenix/query/HTableFactory.java
new file mode 100644
index 0000000..cf5b4da
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/HTableFactory.java
@@ -0,0 +1,56 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.io.IOException;
+import java.util.concurrent.ExecutorService;
+
+import org.apache.hadoop.hbase.client.HConnection;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.HTableInterface;
+
+/**
+ * Creates clients to access HBase tables.
+ *
+ * @author aaraujo
+ * @since 0.2
+ */
+public interface HTableFactory {
+    /**
+     * Creates an HBase client using an externally managed HConnection and Thread pool.
+     *
+     * @param tableName Name of the table.
+     * @param connection HConnection to use.
+     * @param pool ExecutorService to use.
+     * @return An client to access an HBase table.
+     * @throws IOException if a server or network exception occurs
+     */
+    HTableInterface getTable(byte[] tableName, HConnection connection, ExecutorService pool) throws IOException;
+
+    /**
+     * Default implementation.  Uses standard HBase HTables.
+     */
+    static class HTableFactoryImpl implements HTableFactory {
+        @Override
+        public HTableInterface getTable(byte[] tableName, HConnection connection, ExecutorService pool) throws IOException {
+            return new HTable(tableName, connection, pool);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/query/KeyRange.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/KeyRange.java b/phoenix-core/src/main/java/org/apache/phoenix/query/KeyRange.java
new file mode 100644
index 0000000..bc37c57
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/KeyRange.java
@@ -0,0 +1,623 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.query.QueryConstants.SEPARATOR_BYTE_ARRAY;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableUtils;
+
+import com.google.common.base.Function;
+import com.google.common.collect.ComparisonChain;
+import com.google.common.collect.Lists;
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.util.ByteUtil;
+
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ *
+ * Class that represents an upper/lower bound key range.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class KeyRange implements Writable {
+    public enum Bound { LOWER, UPPER };
+    private static final byte[] DEGENERATE_KEY = new byte[] {1};
+    public static final byte[] UNBOUND = new byte[0];
+    /**
+     * KeyRange for variable length null values. Since we need to represent this using an empty byte array (which
+     * is what we use for upper/lower bound), we create this range using the private constructor rather than
+     * going through the static creation method (where this would not be possible).
+     */
+    public static final KeyRange IS_NULL_RANGE = new KeyRange(ByteUtil.EMPTY_BYTE_ARRAY, true, ByteUtil.EMPTY_BYTE_ARRAY, true);
+    /**
+     * KeyRange for non null variable length values. Since we need to represent this using an empty byte array (which
+     * is what we use for upper/lower bound), we create this range using the private constructor rather than going
+     * through the static creation method (where this would not be possible).
+     */
+    public static final KeyRange IS_NOT_NULL_RANGE = new KeyRange(ByteUtil.nextKey(QueryConstants.SEPARATOR_BYTE_ARRAY), true, UNBOUND, false);
+    
+    /**
+     * KeyRange for an empty key range
+     */
+    public static final KeyRange EMPTY_RANGE = new KeyRange(DEGENERATE_KEY, false, DEGENERATE_KEY, false);
+    
+    /**
+     * KeyRange that contains all values
+     */
+    public static final KeyRange EVERYTHING_RANGE = new KeyRange(UNBOUND, false, UNBOUND, false);
+    
+    public static final Function<byte[], KeyRange> POINT = new Function<byte[], KeyRange>() {
+        @Override 
+        public KeyRange apply(byte[] input) {
+            return new KeyRange(input, true, input, true);
+        }
+    };
+    public static final Comparator<KeyRange> COMPARATOR = new Comparator<KeyRange>() {
+        @SuppressWarnings("deprecation")
+        @Override public int compare(KeyRange o1, KeyRange o2) {
+            return ComparisonChain.start()
+//                    .compareFalseFirst(o1.lowerUnbound(), o2.lowerUnbound())
+                    .compare(o2.lowerUnbound(), o1.lowerUnbound())
+                    .compare(o1.getLowerRange(), o2.getLowerRange(), Bytes.BYTES_COMPARATOR)
+                    // we want o1 lower inclusive to come before o2 lower inclusive, but
+                    // false comes before true, so we have to negate
+//                    .compareTrueFirst(o1.isLowerInclusive(), o2.isLowerInclusive())
+                    .compare(o2.isLowerInclusive(), o1.isLowerInclusive())
+                    // for the same lower bounding, we want a finite upper bound to
+                    // be ordered before an infinite upper bound
+//                    .compareTrueFirst(o1.upperUnbound(), o2.upperUnbound())
+                    .compare(o1.upperUnbound(), o2.upperUnbound())
+                    .compare(o1.getUpperRange(), o2.getUpperRange(), Bytes.BYTES_COMPARATOR)
+//                    .compareFalseFirst(o1.isUpperInclusive(), o2.isUpperInclusive())
+                    .compare(o2.isUpperInclusive(), o1.isUpperInclusive())
+                    .result();
+        }
+    };
+
+    private byte[] lowerRange;
+    private boolean lowerInclusive;
+    private byte[] upperRange;
+    private boolean upperInclusive;
+    private boolean isSingleKey;
+
+    public static KeyRange getKeyRange(byte[] point) {
+        return getKeyRange(point, true, point, true);
+    }
+    
+    public static KeyRange getKeyRange(byte[] lowerRange, byte[] upperRange) {
+        return getKeyRange(lowerRange, true, upperRange, false);
+    }
+
+    // TODO: make non public and move to org.apache.phoenix.type soon
+    public static KeyRange getKeyRange(byte[] lowerRange, boolean lowerInclusive,
+            byte[] upperRange, boolean upperInclusive) {
+        if (lowerRange == null || upperRange == null) {
+            return EMPTY_RANGE;
+        }
+        boolean unboundLower = false;
+        boolean unboundUpper = false;
+        if (lowerRange.length == 0) {
+            lowerRange = UNBOUND;
+            lowerInclusive = false;
+            unboundLower = true;
+        }
+        if (upperRange.length == 0) {
+            upperRange = UNBOUND;
+            upperInclusive = false;
+            unboundUpper = true;
+        }
+
+        if (unboundLower && unboundUpper) {
+            return EVERYTHING_RANGE;
+        }
+        if (!unboundLower && !unboundUpper) {
+            int cmp = Bytes.compareTo(lowerRange, upperRange);
+            if (cmp > 0 || (cmp == 0 && !(lowerInclusive && upperInclusive))) {
+                return EMPTY_RANGE;
+            }
+        }
+        return new KeyRange(lowerRange, unboundLower ? false : lowerInclusive,
+                upperRange, unboundUpper ? false : upperInclusive);
+    }
+
+    public KeyRange() {
+        this.lowerRange = DEGENERATE_KEY;
+        this.lowerInclusive = false;
+        this.upperRange = DEGENERATE_KEY;
+        this.upperInclusive = false;
+        this.isSingleKey = false;
+    }
+    
+    private KeyRange(byte[] lowerRange, boolean lowerInclusive, byte[] upperRange, boolean upperInclusive) {
+        this.lowerRange = lowerRange;
+        this.lowerInclusive = lowerInclusive;
+        this.upperRange = upperRange;
+        this.upperInclusive = upperInclusive;
+        init();
+    }
+    
+    private void init() {
+        this.isSingleKey = lowerRange != UNBOUND && upperRange != UNBOUND
+                && lowerInclusive && upperInclusive && Bytes.compareTo(lowerRange, upperRange) == 0;
+    }
+
+    public byte[] getRange(Bound bound) {
+        return bound == Bound.LOWER ? getLowerRange() : getUpperRange();
+    }
+    
+    public boolean isInclusive(Bound bound) {
+        return bound == Bound.LOWER ? isLowerInclusive() : isUpperInclusive();
+    }
+    
+    public boolean isUnbound(Bound bound) {
+        return bound == Bound.LOWER ? lowerUnbound() : upperUnbound();
+    }
+    
+    public boolean isSingleKey() {
+        return isSingleKey;
+    }
+    
+    public int compareLowerToUpperBound(ImmutableBytesWritable ptr, boolean isInclusive) {
+        return compareLowerToUpperBound(ptr.get(), ptr.getOffset(), ptr.getLength(), isInclusive);
+    }
+    
+    public int compareLowerToUpperBound(ImmutableBytesWritable ptr) {
+        return compareLowerToUpperBound(ptr, true);
+    }
+    
+    public int compareUpperToLowerBound(ImmutableBytesWritable ptr, boolean isInclusive) {
+        return compareUpperToLowerBound(ptr.get(), ptr.getOffset(), ptr.getLength(), isInclusive);
+    }
+    
+    public int compareUpperToLowerBound(ImmutableBytesWritable ptr) {
+        return compareUpperToLowerBound(ptr, true);
+    }
+    
+    public int compareLowerToUpperBound( byte[] b, int o, int l) {
+        return compareLowerToUpperBound(b,o,l,true);
+    }
+
+    /**
+     * Compares a lower bound against an upper bound
+     * @param b upper bound byte array
+     * @param o upper bound offset
+     * @param l upper bound length
+     * @param isInclusive upper bound inclusive
+     * @return -1 if the lower bound is less than the upper bound,
+     *          1 if the lower bound is greater than the upper bound,
+     *          and 0 if they are equal.
+     */
+    public int compareLowerToUpperBound( byte[] b, int o, int l, boolean isInclusive) {
+        if (lowerUnbound() || b == KeyRange.UNBOUND) {
+            return -1;
+        }
+        int cmp = Bytes.compareTo(lowerRange, 0, lowerRange.length, b, o, l);
+        if (cmp > 0) {
+            return 1;
+        }
+        if (cmp < 0) {
+            return -1;
+        }
+        if (lowerInclusive && isInclusive) {
+            return 0;
+        }
+        return 1;
+    }
+    
+    public int compareUpperToLowerBound(byte[] b, int o, int l) {
+        return compareUpperToLowerBound(b,o,l, true);
+    }
+    
+    public int compareUpperToLowerBound(byte[] b, int o, int l, boolean isInclusive) {
+        if (upperUnbound() || b == KeyRange.UNBOUND) {
+            return 1;
+        }
+        int cmp = Bytes.compareTo(upperRange, 0, upperRange.length, b, o, l);
+        if (cmp > 0) {
+            return 1;
+        }
+        if (cmp < 0) {
+            return -1;
+        }
+        if (upperInclusive && isInclusive) {
+            return 0;
+        }
+        return -1;
+    }
+    
+    public byte[] getLowerRange() {
+        return lowerRange;
+    }
+
+    public boolean isLowerInclusive() {
+        return lowerInclusive;
+    }
+
+    public byte[] getUpperRange() {
+        return upperRange;
+    }
+
+    public boolean isUpperInclusive() {
+        return upperInclusive;
+    }
+
+    public boolean isUnbound() {
+        return lowerUnbound() || upperUnbound();
+    }
+
+    public boolean upperUnbound() {
+        return upperRange == UNBOUND;
+    }
+
+    public boolean lowerUnbound() {
+        return lowerRange == UNBOUND;
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + Arrays.hashCode(lowerRange);
+        if (lowerRange != null)
+            result = prime * result + (lowerInclusive ? 1231 : 1237);
+        result = prime * result + Arrays.hashCode(upperRange);
+        if (upperRange != null)
+            result = prime * result + (upperInclusive ? 1231 : 1237);
+        return result;
+    }
+
+    @Override
+    public String toString() {
+        if (isSingleKey()) {
+            return Bytes.toStringBinary(lowerRange);
+        }
+        return (lowerInclusive ? "[" : 
+            "(") + (lowerUnbound() ? "*" : 
+                Bytes.toStringBinary(lowerRange)) + " - " + (upperUnbound() ? "*" : 
+                    Bytes.toStringBinary(upperRange)) + (upperInclusive ? "]" : ")" );
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (!(o instanceof KeyRange)) {
+            return false;
+        }
+        KeyRange that = (KeyRange)o;
+        return Bytes.compareTo(this.lowerRange,that.lowerRange) == 0 && this.lowerInclusive == that.lowerInclusive &&
+               Bytes.compareTo(this.upperRange, that.upperRange) == 0 && this.upperInclusive == that.upperInclusive;
+    }
+
+    public KeyRange intersect(KeyRange range) {
+        byte[] newLowerRange;
+        byte[] newUpperRange;
+        boolean newLowerInclusive;
+        boolean newUpperInclusive;
+        if (lowerUnbound()) {
+            newLowerRange = range.lowerRange;
+            newLowerInclusive = range.lowerInclusive;
+        } else if (range.lowerUnbound()) {
+            newLowerRange = lowerRange;
+            newLowerInclusive = lowerInclusive;
+        } else {
+            int cmp = Bytes.compareTo(lowerRange, range.lowerRange);
+            if (cmp != 0 || lowerInclusive == range.lowerInclusive) {
+                if (cmp <= 0) {
+                    newLowerRange = range.lowerRange;
+                    newLowerInclusive = range.lowerInclusive;
+                } else {
+                    newLowerRange = lowerRange;
+                    newLowerInclusive = lowerInclusive;
+                }
+            } else { // Same lower range, but one is not inclusive
+                newLowerRange = range.lowerRange;
+                newLowerInclusive = false;
+            }
+        }
+        if (upperUnbound()) {
+            newUpperRange = range.upperRange;
+            newUpperInclusive = range.upperInclusive;
+        } else if (range.upperUnbound()) {
+            newUpperRange = upperRange;
+            newUpperInclusive = upperInclusive;
+        } else {
+            int cmp = Bytes.compareTo(upperRange, range.upperRange);
+            if (cmp != 0 || upperInclusive == range.upperInclusive) {
+                if (cmp >= 0) {
+                    newUpperRange = range.upperRange;
+                    newUpperInclusive = range.upperInclusive;
+                } else {
+                    newUpperRange = upperRange;
+                    newUpperInclusive = upperInclusive;
+                }
+            } else { // Same upper range, but one is not inclusive
+                newUpperRange = range.upperRange;
+                newUpperInclusive = false;
+            }
+        }
+        if (newLowerRange == lowerRange && newLowerInclusive == lowerInclusive
+                && newUpperRange == upperRange && newUpperInclusive == upperInclusive) {
+            return this;
+        }
+        return getKeyRange(newLowerRange, newLowerInclusive, newUpperRange, newUpperInclusive);
+    }
+
+    public static boolean isDegenerate(byte[] lowerRange, byte[] upperRange) {
+        return lowerRange == KeyRange.EMPTY_RANGE.getLowerRange() && upperRange == KeyRange.EMPTY_RANGE.getUpperRange();
+    }
+
+    public KeyRange appendSeparator() {
+        byte[] lowerBound = getLowerRange();
+        byte[] upperBound = getUpperRange();
+        if (lowerBound != UNBOUND) {
+            lowerBound = ByteUtil.concat(lowerBound, SEPARATOR_BYTE_ARRAY);
+        }
+        if (upperBound != UNBOUND) {
+            upperBound = ByteUtil.concat(upperBound, SEPARATOR_BYTE_ARRAY);
+        }
+        return getKeyRange(lowerBound, lowerInclusive, upperBound, upperInclusive);
+    }
+
+    /**
+     * @return list of at least size 1
+     */
+    @NonNull
+    public static List<KeyRange> coalesce(List<KeyRange> keyRanges) {
+        List<KeyRange> tmp = new ArrayList<KeyRange>();
+        for (KeyRange keyRange : keyRanges) {
+            if (EMPTY_RANGE == keyRange) {
+                continue;
+            }
+            if (EVERYTHING_RANGE == keyRange) {
+                tmp.clear();
+                tmp.add(keyRange);
+                break;
+            }
+            tmp.add(keyRange);
+        }
+        if (tmp.size() == 1) {
+            return tmp;
+        }
+        if (tmp.size() == 0) {
+            return Collections.singletonList(EMPTY_RANGE);
+        }
+
+        Collections.sort(tmp, COMPARATOR);
+        List<KeyRange> tmp2 = new ArrayList<KeyRange>();
+        KeyRange range = tmp.get(0);
+        for (int i=1; i<tmp.size(); i++) {
+            KeyRange otherRange = tmp.get(i);
+            KeyRange intersect = range.intersect(otherRange);
+            if (EMPTY_RANGE == intersect) {
+                tmp2.add(range);
+                range = otherRange;
+            } else {
+                range = range.union(otherRange);
+            }
+        }
+        tmp2.add(range);
+        List<KeyRange> tmp3 = new ArrayList<KeyRange>();
+        range = tmp2.get(0);
+        for (int i=1; i<tmp2.size(); i++) {
+            KeyRange otherRange = tmp2.get(i);
+            assert !range.upperUnbound();
+            assert !otherRange.lowerUnbound();
+            if (range.isUpperInclusive() != otherRange.isLowerInclusive()
+                    && Bytes.equals(range.getUpperRange(), otherRange.getLowerRange())) {
+                range = KeyRange.getKeyRange(range.getLowerRange(), range.isLowerInclusive(), otherRange.getUpperRange(), otherRange.isUpperInclusive());
+            } else {
+                tmp3.add(range);
+                range = otherRange;
+            }
+        }
+        tmp3.add(range);
+        
+        return tmp3;
+    }
+
+    public KeyRange union(KeyRange other) {
+        if (EMPTY_RANGE == other) return this;
+        if (EMPTY_RANGE == this) return other;
+        byte[] newLower, newUpper;
+        boolean newLowerInclusive, newUpperInclusive;
+        if (this.lowerUnbound() || other.lowerUnbound()) {
+            newLower = UNBOUND;
+            newLowerInclusive = false;
+        } else {
+            int lowerCmp = Bytes.compareTo(this.lowerRange, other.lowerRange);
+            if (lowerCmp < 0) {
+                newLower = lowerRange;
+                newLowerInclusive = lowerInclusive;
+            } else if (lowerCmp == 0) {
+                newLower = lowerRange;
+                newLowerInclusive = this.lowerInclusive || other.lowerInclusive;
+            } else {
+                newLower = other.lowerRange;
+                newLowerInclusive = other.lowerInclusive;
+            }
+        }
+
+        if (this.upperUnbound() || other.upperUnbound()) {
+            newUpper = UNBOUND;
+            newUpperInclusive = false;
+        } else {
+            int upperCmp = Bytes.compareTo(this.upperRange, other.upperRange);
+            if (upperCmp > 0) {
+                newUpper = upperRange;
+                newUpperInclusive = this.upperInclusive;
+            } else if (upperCmp == 0) {
+                newUpper = upperRange;
+                newUpperInclusive = this.upperInclusive || other.upperInclusive;
+            } else {
+                newUpper = other.upperRange;
+                newUpperInclusive = other.upperInclusive;
+            }
+        }
+        return KeyRange.getKeyRange(newLower, newLowerInclusive, newUpper, newUpperInclusive);
+    }
+
+    public static List<KeyRange> of(List<byte[]> keys) {
+        return Lists.transform(keys, POINT);
+    }
+
+    public static List<KeyRange> intersect(List<KeyRange> keyRanges, List<KeyRange> keyRanges2) {
+        List<KeyRange> tmp = new ArrayList<KeyRange>();
+        for (KeyRange r1 : keyRanges) {
+            for (KeyRange r2 : keyRanges2) {
+                KeyRange r = r1.intersect(r2);
+                if (EMPTY_RANGE != r) {
+                    tmp.add(r);
+                }
+            }
+        }
+        if (tmp.size() == 0) {
+            return Collections.singletonList(KeyRange.EMPTY_RANGE);
+        }
+        Collections.sort(tmp, KeyRange.COMPARATOR);
+        List<KeyRange> tmp2 = new ArrayList<KeyRange>();
+        KeyRange r = tmp.get(0);
+        for (int i=1; i<tmp.size(); i++) {
+            if (EMPTY_RANGE == r.intersect(tmp.get(i))) {
+                tmp2.add(r);
+                r = tmp.get(i);
+            } else {
+                r = r.intersect(tmp.get(i));
+            }
+        }
+        tmp2.add(r);
+        return tmp2;
+    }
+    
+    /**
+     * Fill both upper and lower range of keyRange to keyLength bytes.
+     * If the upper bound is inclusive, it must be filled such that an
+     * intersection with a longer key would still match if the shorter
+     * length matches.  For example: (*,00C] intersected with [00Caaa,00Caaa]
+     * should still return [00Caaa,00Caaa] since the 00C matches and is
+     * inclusive.
+     * @param keyLength
+     * @return the newly filled KeyRange
+     */
+    public KeyRange fill(int keyLength) {
+        byte[] lowerRange = this.getLowerRange();
+        byte[] newLowerRange = lowerRange;
+        if (!this.lowerUnbound()) {
+            // If lower range is inclusive, fill with 0x00 since conceptually these bytes are included in the range
+            newLowerRange = ByteUtil.fillKey(lowerRange, keyLength);
+        }
+        byte[] upperRange = this.getUpperRange();
+        byte[] newUpperRange = upperRange;
+        if (!this.upperUnbound()) {
+            // If upper range is inclusive, fill with 0xFF since conceptually these bytes are included in the range
+            newUpperRange = ByteUtil.fillKey(upperRange, keyLength);
+        }
+        if (newLowerRange != lowerRange || newUpperRange != upperRange) {
+            return KeyRange.getKeyRange(newLowerRange, this.isLowerInclusive(), newUpperRange, this.isUpperInclusive());
+        }
+        return this;
+    }
+    
+    public KeyRange invert() {
+        byte[] lower = this.getLowerRange();
+        if (!this.lowerUnbound()) {
+            lower = ColumnModifier.SORT_DESC.apply(lower, 0, lower.length);
+        }
+        byte[] upper;
+        if (this.isSingleKey()) {
+            upper = lower;
+        } else {
+            upper = this.getUpperRange();
+            if (!this.upperUnbound()) {
+                upper = ColumnModifier.SORT_DESC.apply(upper, 0, upper.length);
+            }
+        }
+        return KeyRange.getKeyRange(lower, this.isLowerInclusive(), upper, this.isUpperInclusive());
+    }
+
+    @Override
+    public void readFields(DataInput in) throws IOException {
+        int len = WritableUtils.readVInt(in);
+        if (len == 0) {
+            lowerRange = KeyRange.UNBOUND;
+            lowerInclusive = false;
+        } else {
+            if (len < 0) {
+                lowerInclusive = false;
+                lowerRange = new byte[-len - 1];
+                in.readFully(lowerRange);
+            } else {
+                lowerInclusive = true;
+                lowerRange = new byte[len - 1];
+                in.readFully(lowerRange);
+            }
+        }
+        len = WritableUtils.readVInt(in);
+        if (len == 0) {
+            upperRange = KeyRange.UNBOUND;
+            upperInclusive = false;
+        } else {
+            if (len < 0) {
+                upperInclusive = false;
+                upperRange = new byte[-len - 1];
+                in.readFully(upperRange);
+            } else {
+                upperInclusive = true;
+                upperRange = new byte[len - 1];
+                in.readFully(upperRange);
+            }
+        }
+        init();
+    }
+
+    private void writeBound(Bound bound, DataOutput out) throws IOException {
+        // Encode unbound by writing a zero
+        if (isUnbound(bound)) {
+            WritableUtils.writeVInt(out, 0);
+            return;
+        }
+        // Otherwise, inclusive is positive and exclusive is negative, offset by 1
+        byte[] range = getRange(bound);
+        if (isInclusive(bound)){
+            WritableUtils.writeVInt(out, range.length+1);
+        } else {
+            WritableUtils.writeVInt(out, -(range.length+1));
+        }
+        out.write(range);
+    }
+    
+    @Override
+    public void write(DataOutput out) throws IOException {
+        writeBound(Bound.LOWER, out);
+        writeBound(Bound.UPPER, out);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/query/MetaDataMutated.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/MetaDataMutated.java b/phoenix-core/src/main/java/org/apache/phoenix/query/MetaDataMutated.java
new file mode 100644
index 0000000..0ed1d56
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/MetaDataMutated.java
@@ -0,0 +1,42 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.sql.SQLException;
+import java.util.List;
+
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PMetaData;
+import org.apache.phoenix.schema.PTable;
+
+
+/**
+ * 
+ * Interface for applying schema mutations to our client-side schema cache
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public interface MetaDataMutated {
+    PMetaData addTable(PTable table) throws SQLException;
+    PMetaData removeTable(String tableName) throws SQLException;
+    PMetaData addColumn(String tableName, List<PColumn> columns, long tableTimeStamp, long tableSeqNum, boolean isImmutableRows) throws SQLException;
+    PMetaData removeColumn(String tableName, String familyName, String columnName, long tableTimeStamp, long tableSeqNum) throws SQLException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..48bd7f7
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
@@ -0,0 +1,215 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.jdbc.PhoenixDatabaseMetaData.ARRAY_SIZE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.BUFFER_LENGTH;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CACHE_SIZE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CHAR_OCTET_LENGTH;
+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_MODIFIER;
+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.CURRENT_VALUE;
+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.DISABLE_WAL;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IMMUTABLE_ROWS;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INCREMENT_BY;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_STATE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_AUTOINCREMENT;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_NULLABLE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.LINK_TYPE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.MULTI_TENANT;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.NULLABLE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.NUM_PREC_RADIX;
+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.REF_GENERATION_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.REMARKS_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SALT_BUCKETS;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SCOPE_CATALOG;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SCOPE_SCHEMA;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SCOPE_TABLE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SELF_REFERENCING_COL_NAME_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SEQUENCE_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SEQUENCE_SCHEMA;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SOURCE_DATA_TYPE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SQL_DATA_TYPE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SQL_DATETIME_SUB;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.START_WITH;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_CAT_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_NAME_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_SCHEM_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_SEQ_NUM;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_TYPE_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TENANT_ID;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TYPE_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TYPE_SCHEMA;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TYPE_SEQUENCE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TYPE_TABLE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_STATEMENT;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_TYPE;
+
+import java.math.BigDecimal;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.coprocessor.MetaDataProtocol;
+import org.apache.phoenix.schema.MetaDataSplitPolicy;
+import org.apache.phoenix.schema.PName;
+import org.apache.phoenix.schema.PNormalizedName;
+
+
+/**
+ * 
+ * Constants used during querying
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public interface QueryConstants {
+    public static final String NAME_SEPARATOR = ".";
+    public final static byte[] NAME_SEPARATOR_BYTES = Bytes.toBytes(NAME_SEPARATOR);
+    public static final byte NAME_SEPARATOR_BYTE = NAME_SEPARATOR_BYTES[0];
+    public static final String NULL_SCHEMA_NAME = "";
+    public static final String NULL_DISPLAY_TEXT = "<null>";
+    public static final long UNSET_TIMESTAMP = -1;
+    
+    public enum JoinType {INNER, LEFT_OUTER}
+    public final static String PHOENIX_SCHEMA = "system";
+    public final static String PHOENIX_METADATA = "table";
+
+    public final static PName SINGLE_COLUMN_NAME = new PNormalizedName("s");
+    public final static PName SINGLE_COLUMN_FAMILY_NAME = new PNormalizedName("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[] TRUE = new byte[] {1};
+
+    /**
+     * Separator used between variable length keys for a composite key.
+     * Variable length data types may not use this byte value.
+     */
+    public static final byte SEPARATOR_BYTE = (byte) 0;
+    public static final byte[] SEPARATOR_BYTE_ARRAY = new byte[] {SEPARATOR_BYTE};
+    
+    public static final String DEFAULT_COPROCESS_PATH = "phoenix.jar";
+    public final static int MILLIS_IN_DAY = 1000 * 60 * 60 * 24;
+
+    public static final String EMPTY_COLUMN_NAME = "_0";
+    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 String DEFAULT_COLUMN_FAMILY = EMPTY_COLUMN_NAME;
+    public static final byte[] DEFAULT_COLUMN_FAMILY_BYTES = EMPTY_COLUMN_BYTES;
+    public static final String ALL_FAMILY_PROPERTIES_KEY = "";
+    public static final String SYSTEM_TABLE_PK_NAME = "pk";
+    
+    public static final double MILLIS_TO_NANOS_CONVERTOR = Math.pow(10, 6);
+    public static final BigDecimal BD_MILLIS_NANOS_CONVERSION = BigDecimal.valueOf(MILLIS_TO_NANOS_CONVERTOR);
+    public static final BigDecimal BD_MILLIS_IN_DAY = BigDecimal.valueOf(QueryConstants.MILLIS_IN_DAY);
+    
+
+    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.
+            "CREATE TABLE " + TYPE_SCHEMA + ".\"" + TYPE_TABLE + "\"(\n" +
+            // PK columns
+            TENANT_ID + " VARCHAR NULL," +
+            TABLE_SCHEM_NAME + " VARCHAR NULL," +
+            TABLE_NAME_NAME + " VARCHAR NOT NULL," +
+            COLUMN_NAME + " VARCHAR NULL," + // null only for table row
+            TABLE_CAT_NAME + " VARCHAR NULL," + // using for CF - ensures uniqueness for columns
+            // Table metadata (will be null for column rows)
+            TABLE_TYPE_NAME + " CHAR(1)," +
+            REMARKS_NAME + " VARCHAR," +
+            DATA_TYPE + " INTEGER," +
+            PK_NAME + " VARCHAR," +
+            TYPE_NAME + " VARCHAR," +
+            SELF_REFERENCING_COL_NAME_NAME + " VARCHAR," +
+            REF_GENERATION_NAME + " VARCHAR," +
+            TABLE_SEQ_NUM + " BIGINT," +
+            COLUMN_COUNT + " INTEGER," +
+            // Column metadata (will be null for table row)
+            COLUMN_SIZE + " INTEGER," +
+            BUFFER_LENGTH + " INTEGER," +
+            DECIMAL_DIGITS + " INTEGER," +
+            NUM_PREC_RADIX + " INTEGER," +
+            NULLABLE + " INTEGER," +
+            COLUMN_DEF + " VARCHAR," +
+            SQL_DATA_TYPE + " INTEGER," +
+            SQL_DATETIME_SUB + " INTEGER," +
+            CHAR_OCTET_LENGTH + " INTEGER," +
+            ORDINAL_POSITION + " INTEGER," +
+            IS_NULLABLE + " VARCHAR," +
+            SCOPE_CATALOG + " VARCHAR," +
+            SCOPE_SCHEMA + " VARCHAR," +
+            SCOPE_TABLE + " VARCHAR," +
+            SOURCE_DATA_TYPE + " INTEGER," + // supposed to be SHORT
+            IS_AUTOINCREMENT + " VARCHAR," +
+            // Columns added in 1.2.1
+            COLUMN_MODIFIER + " INTEGER," +
+            SALT_BUCKETS + " INTEGER," +
+            // Columns added in 2.0.0
+            DATA_TABLE_NAME + " VARCHAR," +
+            INDEX_STATE + " CHAR(1),\n" +
+            IMMUTABLE_ROWS + " BOOLEAN,\n" +
+            // Columns added in 3.0.0
+            VIEW_STATEMENT + " VARCHAR,\n" +
+            DEFAULT_COLUMN_FAMILY_NAME + " VARCHAR,\n" +
+            DISABLE_WAL + " BOOLEAN,\n" +
+            MULTI_TENANT + " BOOLEAN,\n" +
+            VIEW_TYPE + " UNSIGNED_TINYINT,\n" +
+            LINK_TYPE + " UNSIGNED_TINYINT,\n" +
+            ARRAY_SIZE + " INTEGER,\n" +
+            "CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (" + TENANT_ID + ","
+            + TABLE_SCHEM_NAME + "," + TABLE_NAME_NAME + "," + COLUMN_NAME + "," + TABLE_CAT_NAME + "))\n" +
+            HConstants.VERSIONS + "=" + MetaDataProtocol.DEFAULT_MAX_META_DATA_VERSIONS + ",\n" +
+            DEFAULT_COLUMN_FAMILY_NAME + "=" + "'_0'" + ",\n" + // Use original default for b/w compat
+            HTableDescriptor.SPLIT_POLICY + "='" + MetaDataSplitPolicy.class.getName() + "'\n";
+    
+    public static final String CREATE_SEQUENCE_METADATA =
+            "CREATE TABLE IF NOT EXISTS " + TYPE_SCHEMA + ".\"" + TYPE_SEQUENCE + "\"(\n" +                                    
+            TENANT_ID + " VARCHAR NULL," +
+    		SEQUENCE_SCHEMA + " VARCHAR NULL, \n" + 
+            SEQUENCE_NAME +  " VARCHAR NOT NULL, \n" +
+            START_WITH + " BIGINT NOT NULL, \n" + 
+    		CURRENT_VALUE + " BIGINT NOT NULL, \n" + 
+            INCREMENT_BY  + " BIGINT NOT NULL, \n" + 
+            CACHE_SIZE  + " INTEGER NOT NULL \n" + 
+    		" CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (" + TENANT_ID + "," + SEQUENCE_SCHEMA + "," + SEQUENCE_NAME + "))\n" + 
+    		HConstants.VERSIONS + "=" + MetaDataProtocol.DEFAULT_MAX_META_DATA_VERSIONS + "\n";
+	
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..320bac0
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
@@ -0,0 +1,119 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.util.concurrent.ExecutorService;
+
+import org.apache.http.annotation.Immutable;
+
+import org.apache.phoenix.iterate.SpoolTooBigToDiskException;
+import org.apache.phoenix.memory.MemoryManager;
+import org.apache.phoenix.optimize.QueryOptimizer;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.SQLCloseable;
+
+
+
+/**
+ * 
+ * Interface to group together services needed during querying.  The
+ * parameters that may be set in {@link org.apache.hadoop.conf.Configuration}
+ * are documented here: https://github.com/forcedotcom/phoenix/wiki/Tuning
+ *     
+ * @author jtaylor
+ * @since 0.1
+ */
+@Immutable
+public interface QueryServices extends SQLCloseable {
+    public static final String KEEP_ALIVE_MS_ATTRIB = "phoenix.query.keepAliveMs";
+    public static final String THREAD_POOL_SIZE_ATTRIB = "phoenix.query.threadPoolSize";
+    public static final String QUEUE_SIZE_ATTRIB = "phoenix.query.queueSize";
+    public static final String THREAD_TIMEOUT_MS_ATTRIB = "phoenix.query.timeoutMs";
+    public static final String SPOOL_THRESHOLD_BYTES_ATTRIB = "phoenix.query.spoolThresholdBytes";
+    
+    /**
+	 * max size to spool the the result into
+	 * ${java.io.tmpdir}/ResultSpoolerXXX.bin if
+	 * {@link QueryServices#SPOOL_THRESHOLD_BYTES_ATTRIB } is reached.
+	 * <p>
+	 * default is unlimited(-1)
+	 * <p>
+	 * if the threshold is reached, a {@link SpoolTooBigToDiskException } will be thrown 
+	 */
+	public static final String MAX_SPOOL_TO_DISK_BYTES_ATTRIB = "phoenix.query.maxSpoolToDiskBytes";
+    
+    public static final String MAX_MEMORY_PERC_ATTRIB = "phoenix.query.maxGlobalMemoryPercentage";
+    public static final String MAX_MEMORY_WAIT_MS_ATTRIB = "phoenix.query.maxGlobalMemoryWaitMs";
+    public static final String MAX_TENANT_MEMORY_PERC_ATTRIB = "phoenix.query.maxTenantMemoryPercentage";
+    public static final String MAX_SERVER_CACHE_SIZE_ATTRIB = "phoenix.query.maxServerCacheBytes";
+    public static final String TARGET_QUERY_CONCURRENCY_ATTRIB = "phoenix.query.targetConcurrency";
+    public static final String MAX_QUERY_CONCURRENCY_ATTRIB = "phoenix.query.maxConcurrency";
+    public static final String DATE_FORMAT_ATTRIB = "phoenix.query.dateFormat";
+    public static final String NUMBER_FORMAT_ATTRIB = "phoenix.query.numberFormat";
+    public static final String STATS_UPDATE_FREQ_MS_ATTRIB = "phoenix.query.statsUpdateFrequency";
+    public static final String MAX_STATS_AGE_MS_ATTRIB = "phoenix.query.maxStatsAge";
+    public static final String CALL_QUEUE_ROUND_ROBIN_ATTRIB = "ipc.server.callqueue.roundrobin";
+    public static final String SCAN_CACHE_SIZE_ATTRIB = "hbase.client.scanner.caching";
+    public static final String MAX_MUTATION_SIZE_ATTRIB = "phoenix.mutate.maxSize";
+    public static final String MUTATE_BATCH_SIZE_ATTRIB = "phoenix.mutate.batchSize";
+    public static final String MAX_SERVER_CACHE_TIME_TO_LIVE_MS = "phoenix.coprocessor.maxServerCacheTimeToLiveMs";
+    public static final String MAX_INTRA_REGION_PARALLELIZATION_ATTRIB  = "phoenix.query.maxIntraRegionParallelization";
+    public static final String ROW_KEY_ORDER_SALTED_TABLE_ATTRIB  = "phoenix.query.rowKeyOrderSaltedTable";
+    public static final String USE_INDEXES_ATTRIB  = "phoenix.query.useIndexes";
+    public static final String IMMUTABLE_ROWS_ATTRIB  = "phoenix.mutate.immutableRows";
+    public static final String INDEX_MUTATE_BATCH_SIZE_THRESHOLD_ATTRIB  = "phoenix.index.mutableBatchSizeThreshold";
+    public static final String DROP_METADATA_ATTRIB  = "phoenix.schema.dropMetaData";
+    public static final String GROUPBY_SPILLABLE_ATTRIB  = "phoenix.groupby.spillable";
+    public static final String GROUPBY_SPILL_FILES_ATTRIB = "phoenix.groupby.spillFiles";
+    public static final String GROUPBY_MAX_CACHE_SIZE_ATTRIB = "phoenix.groupby.maxCacheSize";
+
+    public static final String CALL_QUEUE_PRODUCER_ATTRIB_NAME = "CALL_QUEUE_PRODUCER";
+    
+    public static final String MASTER_INFO_PORT_ATTRIB = "hbase.master.info.port";
+    public static final String REGIONSERVER_INFO_PORT_ATTRIB = "hbase.regionserver.info.port";
+    public static final String REGIONSERVER_LEASE_PERIOD_ATTRIB = "hbase.regionserver.lease.period";
+    public static final String RPC_TIMEOUT_ATTRIB = "hbase.rpc.timeout";
+    public static final String ZOOKEEPER_QUARUM_ATTRIB = "hbase.zookeeper.quorum";
+    public static final String ZOOKEEPER_PORT_ATTRIB = "hbase.zookeeper.property.clientPort";
+    public static final String ZOOKEEPER_ROOT_NODE_ATTRIB = "zookeeper.znode.parent";
+    public static final String DISTINCT_VALUE_COMPRESS_THRESHOLD_ATTRIB = "phoenix.distinct.value.compress.threshold";
+    public static final String SEQUENCE_CACHE_SIZE_ATTRIB = "phoenix.sequence.cacheSize";
+
+    
+    /**
+     * Get executor service used for parallel scans
+     */
+    public ExecutorService getExecutor();
+    /**
+     * Get the memory manager used to track memory usage
+     */
+    public MemoryManager getMemoryManager();
+    
+    /**
+     * Get the properties from the HBase configuration in a
+     * read-only structure that avoids any synchronization
+     */
+    public ReadOnlyProps getProps();
+    
+    /**
+     * Get query optimizer used to choose the best query plan
+     */
+    public QueryOptimizer getOptimizer();
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesImpl.java
new file mode 100644
index 0000000..5ebde59
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesImpl.java
@@ -0,0 +1,38 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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;
+
+
+
+
+
+/**
+ * 
+ * Real implementation of QueryServices for use in runtime and perf testing
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public final class QueryServicesImpl extends BaseQueryServicesImpl {
+    
+    public QueryServicesImpl() {
+        super(QueryServicesOptions.withDefaults());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..7f3025c
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
@@ -0,0 +1,410 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.query.QueryServices.CALL_QUEUE_PRODUCER_ATTRIB_NAME;
+import static org.apache.phoenix.query.QueryServices.CALL_QUEUE_ROUND_ROBIN_ATTRIB;
+import static org.apache.phoenix.query.QueryServices.DATE_FORMAT_ATTRIB;
+import static org.apache.phoenix.query.QueryServices.DROP_METADATA_ATTRIB;
+import static org.apache.phoenix.query.QueryServices.GROUPBY_MAX_CACHE_SIZE_ATTRIB;
+import static org.apache.phoenix.query.QueryServices.GROUPBY_SPILL_FILES_ATTRIB;
+import static org.apache.phoenix.query.QueryServices.GROUPBY_SPILLABLE_ATTRIB;
+import static org.apache.phoenix.query.QueryServices.IMMUTABLE_ROWS_ATTRIB;
+import static org.apache.phoenix.query.QueryServices.INDEX_MUTATE_BATCH_SIZE_THRESHOLD_ATTRIB;
+import static org.apache.phoenix.query.QueryServices.KEEP_ALIVE_MS_ATTRIB;
+import static org.apache.phoenix.query.QueryServices.MASTER_INFO_PORT_ATTRIB;
+import static org.apache.phoenix.query.QueryServices.MAX_INTRA_REGION_PARALLELIZATION_ATTRIB;
+import static org.apache.phoenix.query.QueryServices.MAX_MEMORY_PERC_ATTRIB;
+import static org.apache.phoenix.query.QueryServices.MAX_MEMORY_WAIT_MS_ATTRIB;
+import static org.apache.phoenix.query.QueryServices.MAX_MUTATION_SIZE_ATTRIB;
+import static org.apache.phoenix.query.QueryServices.MAX_QUERY_CONCURRENCY_ATTRIB;
+import static org.apache.phoenix.query.QueryServices.MAX_SERVER_CACHE_SIZE_ATTRIB;
+import static org.apache.phoenix.query.QueryServices.MAX_SERVER_CACHE_TIME_TO_LIVE_MS;
+import static org.apache.phoenix.query.QueryServices.MAX_SPOOL_TO_DISK_BYTES_ATTRIB;
+import static org.apache.phoenix.query.QueryServices.MAX_TENANT_MEMORY_PERC_ATTRIB;
+import static org.apache.phoenix.query.QueryServices.MUTATE_BATCH_SIZE_ATTRIB;
+import static org.apache.phoenix.query.QueryServices.QUEUE_SIZE_ATTRIB;
+import static org.apache.phoenix.query.QueryServices.REGIONSERVER_INFO_PORT_ATTRIB;
+import static org.apache.phoenix.query.QueryServices.REGIONSERVER_LEASE_PERIOD_ATTRIB;
+import static org.apache.phoenix.query.QueryServices.ROW_KEY_ORDER_SALTED_TABLE_ATTRIB;
+import static org.apache.phoenix.query.QueryServices.RPC_TIMEOUT_ATTRIB;
+import static org.apache.phoenix.query.QueryServices.SCAN_CACHE_SIZE_ATTRIB;
+import static org.apache.phoenix.query.QueryServices.SEQUENCE_CACHE_SIZE_ATTRIB;
+import static org.apache.phoenix.query.QueryServices.SPOOL_THRESHOLD_BYTES_ATTRIB;
+import static org.apache.phoenix.query.QueryServices.STATS_UPDATE_FREQ_MS_ATTRIB;
+import static org.apache.phoenix.query.QueryServices.TARGET_QUERY_CONCURRENCY_ATTRIB;
+import static org.apache.phoenix.query.QueryServices.THREAD_POOL_SIZE_ATTRIB;
+import static org.apache.phoenix.query.QueryServices.THREAD_TIMEOUT_MS_ATTRIB;
+import static org.apache.phoenix.query.QueryServices.USE_INDEXES_ATTRIB;
+
+import java.util.Map.Entry;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.regionserver.wal.WALEditCodec;
+
+import org.apache.phoenix.util.DateUtil;
+import org.apache.phoenix.util.ReadOnlyProps;
+
+
+/**
+ * Options for {@link QueryServices}.
+ * 
+ * @author syyang
+ * @since 0.1
+ */
+public class QueryServicesOptions {
+	public static final int DEFAULT_KEEP_ALIVE_MS = 60000;
+	public static final int DEFAULT_THREAD_POOL_SIZE = 128;
+	public static final int DEFAULT_QUEUE_SIZE = 500;
+	public static final int DEFAULT_THREAD_TIMEOUT_MS = 600000; // 10min
+	public static final int DEFAULT_SPOOL_THRESHOLD_BYTES = 1024 * 1024 * 20; // 20m
+	public static final int DEFAULT_MAX_MEMORY_PERC = 50; // 50% of heap
+	public static final int DEFAULT_MAX_MEMORY_WAIT_MS = 10000;
+	public static final int DEFAULT_MAX_TENANT_MEMORY_PERC = 100;
+	public static final long DEFAULT_MAX_SERVER_CACHE_SIZE = 1024*1024*100;  // 100 Mb
+    public static final int DEFAULT_TARGET_QUERY_CONCURRENCY = 32;
+    public static final int DEFAULT_MAX_QUERY_CONCURRENCY = 64;
+    public static final String DEFAULT_DATE_FORMAT = DateUtil.DEFAULT_DATE_FORMAT;
+    public static final int DEFAULT_STATS_UPDATE_FREQ_MS = 15 * 60000; // 15min
+    public static final int DEFAULT_MAX_STATS_AGE_MS = 24 * 60 * 60000; // 1 day
+    public static final boolean DEFAULT_CALL_QUEUE_ROUND_ROBIN = true; 
+    public static final int DEFAULT_MAX_MUTATION_SIZE = 500000;
+    public static final boolean DEFAULT_ROW_KEY_ORDER_SALTED_TABLE = true; // Merge sort on client to ensure salted tables are row key ordered
+    public static final boolean DEFAULT_USE_INDEXES = true; // Use indexes
+    public static final boolean DEFAULT_IMMUTABLE_ROWS = false; // Tables rows may be updated
+    public static final boolean DEFAULT_DROP_METADATA = true; // Drop meta data also.
+    
+    public final static int DEFAULT_MUTATE_BATCH_SIZE = 1000; // Batch size for UPSERT SELECT and DELETE
+	// The only downside of it being out-of-sync is that the parallelization of the scan won't be as balanced as it could be.
+    public static final int DEFAULT_MAX_SERVER_CACHE_TIME_TO_LIVE_MS = 30000; // 30 sec (with no activity)
+    public static final int DEFAULT_SCAN_CACHE_SIZE = 1000;
+    public static final int DEFAULT_MAX_INTRA_REGION_PARALLELIZATION = DEFAULT_MAX_QUERY_CONCURRENCY;
+    public static final int DEFAULT_DISTINCT_VALUE_COMPRESS_THRESHOLD = 1024 * 1024 * 1; // 1 Mb
+    public static final int DEFAULT_INDEX_MUTATE_BATCH_SIZE_THRESHOLD = 5;
+    public static final long DEFAULT_MAX_SPOOL_TO_DISK_BYTES = 1024000000;
+    
+    // 
+    // Spillable GroupBy - SPGBY prefix
+    //
+    // Enable / disable spillable group by
+    public static boolean DEFAULT_GROUPBY_SPILLABLE = true;
+    // Number of spill files / partitions the keys are distributed to
+    // Each spill file fits 2GB of data
+    public static final int DEFAULT_GROUPBY_SPILL_FILES = 2;
+    // Max size of 1st level main memory cache in bytes --> upper bound
+    public static final long DEFAULT_GROUPBY_MAX_CACHE_MAX = 1024L*1024L*100L;  // 100 Mb
+    
+    public static final int DEFAULT_SEQUENCE_CACHE_SIZE = 100;  // reserve 100 sequences at a time
+    
+    
+    private final Configuration config;
+    
+    private QueryServicesOptions(Configuration config) {
+        this.config = config;
+    }
+    
+    public ReadOnlyProps getProps() {
+        // Ensure that HBase RPC time out value is at least as large as our thread time out for query. 
+        int threadTimeOutMS = config.getInt(THREAD_TIMEOUT_MS_ATTRIB, DEFAULT_THREAD_TIMEOUT_MS);
+        int hbaseRPCTimeOut = config.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
+        if (threadTimeOutMS > hbaseRPCTimeOut) {
+            config.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, threadTimeOutMS);
+        }
+        return new ReadOnlyProps(config.iterator());
+    }
+    
+    public QueryServicesOptions setAll(ReadOnlyProps props) {
+        for (Entry<String,String> entry : props) {
+            config.set(entry.getKey(), entry.getValue());
+        }
+        return this;
+    }
+
+    public static QueryServicesOptions withDefaults() {
+        Configuration config = HBaseFactoryProvider.getConfigurationFactory().getConfiguration();
+        QueryServicesOptions options = new QueryServicesOptions(config)
+            .setIfUnset(KEEP_ALIVE_MS_ATTRIB, DEFAULT_KEEP_ALIVE_MS)
+            .setIfUnset(THREAD_POOL_SIZE_ATTRIB, DEFAULT_THREAD_POOL_SIZE)
+            .setIfUnset(QUEUE_SIZE_ATTRIB, DEFAULT_QUEUE_SIZE)
+            .setIfUnset(THREAD_TIMEOUT_MS_ATTRIB, DEFAULT_THREAD_TIMEOUT_MS)
+            .setIfUnset(SPOOL_THRESHOLD_BYTES_ATTRIB, DEFAULT_SPOOL_THRESHOLD_BYTES)
+            .setIfUnset(MAX_MEMORY_PERC_ATTRIB, DEFAULT_MAX_MEMORY_PERC)
+            .setIfUnset(MAX_MEMORY_WAIT_MS_ATTRIB, DEFAULT_MAX_MEMORY_WAIT_MS)
+            .setIfUnset(MAX_TENANT_MEMORY_PERC_ATTRIB, DEFAULT_MAX_TENANT_MEMORY_PERC)
+            .setIfUnset(MAX_SERVER_CACHE_SIZE_ATTRIB, DEFAULT_MAX_SERVER_CACHE_SIZE)
+            .setIfUnset(SCAN_CACHE_SIZE_ATTRIB, DEFAULT_SCAN_CACHE_SIZE)
+            .setIfUnset(TARGET_QUERY_CONCURRENCY_ATTRIB, DEFAULT_TARGET_QUERY_CONCURRENCY)
+            .setIfUnset(MAX_QUERY_CONCURRENCY_ATTRIB, DEFAULT_MAX_QUERY_CONCURRENCY)
+            .setIfUnset(DATE_FORMAT_ATTRIB, DEFAULT_DATE_FORMAT)
+            .setIfUnset(STATS_UPDATE_FREQ_MS_ATTRIB, DEFAULT_STATS_UPDATE_FREQ_MS)
+            .setIfUnset(CALL_QUEUE_ROUND_ROBIN_ATTRIB, DEFAULT_CALL_QUEUE_ROUND_ROBIN)
+            .setIfUnset(MAX_MUTATION_SIZE_ATTRIB, DEFAULT_MAX_MUTATION_SIZE)
+            .setIfUnset(MAX_INTRA_REGION_PARALLELIZATION_ATTRIB, DEFAULT_MAX_INTRA_REGION_PARALLELIZATION)
+            .setIfUnset(ROW_KEY_ORDER_SALTED_TABLE_ATTRIB, DEFAULT_ROW_KEY_ORDER_SALTED_TABLE)
+            .setIfUnset(USE_INDEXES_ATTRIB, DEFAULT_USE_INDEXES)
+            .setIfUnset(IMMUTABLE_ROWS_ATTRIB, DEFAULT_IMMUTABLE_ROWS)
+            .setIfUnset(INDEX_MUTATE_BATCH_SIZE_THRESHOLD_ATTRIB, DEFAULT_INDEX_MUTATE_BATCH_SIZE_THRESHOLD)
+            .setIfUnset(MAX_SPOOL_TO_DISK_BYTES_ATTRIB, DEFAULT_MAX_SPOOL_TO_DISK_BYTES)
+            .setIfUnset(DROP_METADATA_ATTRIB, DEFAULT_DROP_METADATA)
+            .setIfUnset(GROUPBY_SPILLABLE_ATTRIB, DEFAULT_GROUPBY_SPILLABLE)
+            .setIfUnset(GROUPBY_MAX_CACHE_SIZE_ATTRIB, DEFAULT_GROUPBY_MAX_CACHE_MAX)
+            .setIfUnset(GROUPBY_SPILL_FILES_ATTRIB, DEFAULT_GROUPBY_SPILL_FILES)
+            .setIfUnset(SEQUENCE_CACHE_SIZE_ATTRIB, DEFAULT_SEQUENCE_CACHE_SIZE)
+            ;
+        // HBase sets this to 1, so we reset it to something more appropriate.
+        // Hopefully HBase will change this, because we can't know if a user set
+        // it to 1, so we'll change it.
+        int scanCaching = config.getInt(SCAN_CACHE_SIZE_ATTRIB, 0);
+        if (scanCaching == 1) {
+            config.setInt(SCAN_CACHE_SIZE_ATTRIB, DEFAULT_SCAN_CACHE_SIZE);
+        } else if (scanCaching <= 0) { // Provides the user with a way of setting it to 1
+            config.setInt(SCAN_CACHE_SIZE_ATTRIB, 1);
+        }
+        return options;
+    }
+    
+    public Configuration getConfiguration() {
+        return config;
+    }
+
+    private QueryServicesOptions setIfUnset(String name, int value) {
+        config.setIfUnset(name, Integer.toString(value));
+        return this;
+    }
+    
+    private QueryServicesOptions setIfUnset(String name, boolean value) {
+        config.setIfUnset(name, Boolean.toString(value));
+        return this;
+    }
+    
+    private QueryServicesOptions setIfUnset(String name, long value) {
+        config.setIfUnset(name, Long.toString(value));
+        return this;
+    }
+    
+    private QueryServicesOptions setIfUnset(String name, String value) {
+        config.setIfUnset(name, value);
+        return this;
+    }
+    
+    public QueryServicesOptions setKeepAliveMs(int keepAliveMs) {
+        return set(KEEP_ALIVE_MS_ATTRIB, keepAliveMs);
+    }
+    
+    public QueryServicesOptions setThreadPoolSize(int threadPoolSize) {
+        return set(THREAD_POOL_SIZE_ATTRIB, threadPoolSize);
+    }
+    
+    public QueryServicesOptions setQueueSize(int queueSize) {
+        config.setInt(QUEUE_SIZE_ATTRIB, queueSize);
+        return this;
+    }
+    
+    public QueryServicesOptions setThreadTimeoutMs(int threadTimeoutMs) {
+        return set(THREAD_TIMEOUT_MS_ATTRIB, threadTimeoutMs);
+    }
+    
+    public QueryServicesOptions setSpoolThresholdBytes(int spoolThresholdBytes) {
+        return set(SPOOL_THRESHOLD_BYTES_ATTRIB, spoolThresholdBytes);
+    }
+    
+    public QueryServicesOptions setMaxMemoryPerc(int maxMemoryPerc) {
+        return set(MAX_MEMORY_PERC_ATTRIB, maxMemoryPerc);
+    }
+    
+    public QueryServicesOptions setMaxMemoryWaitMs(int maxMemoryWaitMs) {
+        return set(MAX_MEMORY_WAIT_MS_ATTRIB, maxMemoryWaitMs);
+    }
+    
+    public QueryServicesOptions setMaxTenantMemoryPerc(int maxTenantMemoryPerc) {
+        return set(MAX_TENANT_MEMORY_PERC_ATTRIB, maxTenantMemoryPerc);
+    }
+    
+    public QueryServicesOptions setMaxServerCacheSize(long maxServerCacheSize) {
+        return set(MAX_SERVER_CACHE_SIZE_ATTRIB, maxServerCacheSize);
+    }
+
+    public QueryServicesOptions setScanFetchSize(int scanFetchSize) {
+        return set(SCAN_CACHE_SIZE_ATTRIB, scanFetchSize);
+    }
+    
+    public QueryServicesOptions setMaxQueryConcurrency(int maxQueryConcurrency) {
+        return set(MAX_QUERY_CONCURRENCY_ATTRIB, maxQueryConcurrency);
+    }
+
+    public QueryServicesOptions setTargetQueryConcurrency(int targetQueryConcurrency) {
+        return set(TARGET_QUERY_CONCURRENCY_ATTRIB, targetQueryConcurrency);
+    }
+    
+    public QueryServicesOptions setDateFormat(String dateFormat) {
+        return set(DATE_FORMAT_ATTRIB, dateFormat);
+    }
+    
+    public QueryServicesOptions setStatsUpdateFrequencyMs(int frequencyMs) {
+        return set(STATS_UPDATE_FREQ_MS_ATTRIB, frequencyMs);
+    }
+    
+    public QueryServicesOptions setCallQueueRoundRobin(boolean isRoundRobin) {
+        return set(CALL_QUEUE_PRODUCER_ATTRIB_NAME, isRoundRobin);
+    }
+    
+    public QueryServicesOptions setMaxMutateSize(int maxMutateSize) {
+        return set(MAX_MUTATION_SIZE_ATTRIB, maxMutateSize);
+    }
+    
+    public QueryServicesOptions setMutateBatchSize(int mutateBatchSize) {
+        return set(MUTATE_BATCH_SIZE_ATTRIB, mutateBatchSize);
+    }
+    
+    public QueryServicesOptions setMaxIntraRegionParallelization(int maxIntraRegionParallelization) {
+        return set(MAX_INTRA_REGION_PARALLELIZATION_ATTRIB, maxIntraRegionParallelization);
+    }
+    
+    public QueryServicesOptions setRowKeyOrderSaltedTable(boolean rowKeyOrderSaltedTable) {
+        return set(ROW_KEY_ORDER_SALTED_TABLE_ATTRIB, rowKeyOrderSaltedTable);
+    }
+    
+    public QueryServicesOptions setDropMetaData(boolean dropMetadata) {
+        return set(DROP_METADATA_ATTRIB, dropMetadata);
+    }
+    
+    public QueryServicesOptions setSPGBYEnabled(boolean enabled) {
+        return set(GROUPBY_SPILLABLE_ATTRIB, enabled);
+    }
+
+    public QueryServicesOptions setSPGBYMaxCacheSize(long size) {
+        return set(GROUPBY_MAX_CACHE_SIZE_ATTRIB, size);
+    }
+    
+    public QueryServicesOptions setSPGBYNumSpillFiles(long num) {
+        return set(GROUPBY_SPILL_FILES_ATTRIB, num);
+    }
+
+    
+    private QueryServicesOptions set(String name, boolean value) {
+        config.set(name, Boolean.toString(value));
+        return this;
+    }
+    
+    private QueryServicesOptions set(String name, int value) {
+        config.set(name, Integer.toString(value));
+        return this;
+    }
+    
+    private QueryServicesOptions set(String name, String value) {
+        config.set(name, value);
+        return this;
+    }
+    
+    private QueryServicesOptions set(String name, long value) {
+        config.set(name, Long.toString(value));
+        return this;
+    }
+
+    public int getKeepAliveMs() {
+        return config.getInt(KEEP_ALIVE_MS_ATTRIB, DEFAULT_KEEP_ALIVE_MS);
+    }
+    
+    public int getThreadPoolSize() {
+        return config.getInt(THREAD_POOL_SIZE_ATTRIB, DEFAULT_THREAD_POOL_SIZE);
+    }
+    
+    public int getQueueSize() {
+        return config.getInt(QUEUE_SIZE_ATTRIB, DEFAULT_QUEUE_SIZE);
+    }
+    
+    public int getMaxMemoryPerc() {
+        return config.getInt(MAX_MEMORY_PERC_ATTRIB, DEFAULT_MAX_MEMORY_PERC);
+    }
+    
+    public int getMaxMemoryWaitMs() {
+        return config.getInt(MAX_MEMORY_WAIT_MS_ATTRIB, DEFAULT_MAX_MEMORY_WAIT_MS);
+    }
+
+    public int getMaxMutateSize() {
+        return config.getInt(MAX_MUTATION_SIZE_ATTRIB, DEFAULT_MAX_MUTATION_SIZE);
+    }
+
+    public int getMutateBatchSize() {
+        return config.getInt(MUTATE_BATCH_SIZE_ATTRIB, DEFAULT_MUTATE_BATCH_SIZE);
+    }
+    
+    public int getMaxIntraRegionParallelization() {
+        return config.getInt(MAX_INTRA_REGION_PARALLELIZATION_ATTRIB, DEFAULT_MAX_INTRA_REGION_PARALLELIZATION);
+    }
+    
+    public boolean isUseIndexes() {
+        return config.getBoolean(USE_INDEXES_ATTRIB, DEFAULT_USE_INDEXES);
+    }
+
+    public boolean isImmutableRows() {
+        return config.getBoolean(IMMUTABLE_ROWS_ATTRIB, DEFAULT_IMMUTABLE_ROWS);
+    }
+    
+    public boolean isDropMetaData() {
+        return config.getBoolean(DROP_METADATA_ATTRIB, DEFAULT_DROP_METADATA);
+    }
+    
+    public boolean isSpillableGroupByEnabled() {
+        return config.getBoolean(GROUPBY_SPILLABLE_ATTRIB, DEFAULT_GROUPBY_SPILLABLE);
+    }
+    
+    public long getSpillableGroupByMaxCacheSize() {
+        return config.getLong(GROUPBY_MAX_CACHE_SIZE_ATTRIB, DEFAULT_GROUPBY_MAX_CACHE_MAX);
+    }
+    
+    public int getSpillableGroupByNumSpillFiles() {
+        return config.getInt(GROUPBY_SPILL_FILES_ATTRIB, DEFAULT_GROUPBY_SPILL_FILES);
+    }
+
+    public QueryServicesOptions setMaxServerCacheTTLMs(int ttl) {
+        return set(MAX_SERVER_CACHE_TIME_TO_LIVE_MS, ttl);
+    }
+    
+    public QueryServicesOptions setMasterInfoPort(int port) {
+        return set(MASTER_INFO_PORT_ATTRIB, port);
+    }
+    
+    public QueryServicesOptions setRegionServerInfoPort(int port) {
+        return set(REGIONSERVER_INFO_PORT_ATTRIB, port);
+    }
+    
+    public QueryServicesOptions setRegionServerLeasePeriodMs(int period) {
+        return set(REGIONSERVER_LEASE_PERIOD_ATTRIB, period);
+    }
+    
+    public QueryServicesOptions setRpcTimeoutMs(int timeout) {
+        return set(RPC_TIMEOUT_ATTRIB, timeout);
+    }
+    
+    public QueryServicesOptions setUseIndexes(boolean useIndexes) {
+        return set(USE_INDEXES_ATTRIB, useIndexes);
+    }
+    
+    public QueryServicesOptions setImmutableRows(boolean isImmutableRows) {
+        return set(IMMUTABLE_ROWS_ATTRIB, isImmutableRows);
+    }
+
+    public QueryServicesOptions setWALEditCodec(String walEditCodec) {
+        return set(WALEditCodec.WAL_EDIT_CODEC_CLASS_KEY, walEditCodec);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/query/StatsManager.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/StatsManager.java b/phoenix-core/src/main/java/org/apache/phoenix/query/StatsManager.java
new file mode 100644
index 0000000..7d02e02
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/StatsManager.java
@@ -0,0 +1,59 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.sql.SQLException;
+
+import org.apache.phoenix.schema.TableRef;
+
+
+/**
+ * 
+ * Interface for managing and caching table statistics.
+ * The frequency of updating the table statistics are controlled
+ * by {@link org.apache.phoenix.query.QueryServices#STATS_UPDATE_FREQ_MS_ATTRIB}.
+ * Table stats may also be manually updated through {@link #updateStats(TableRef)}.
+ * 
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public interface StatsManager {
+    /**
+     * Get the minimum key for the given table
+     * @param table the table
+     * @return the minimum key or null if unknown
+     */
+    byte[] getMinKey(TableRef table);
+    
+    /**
+     * Get the maximum key for the given table
+     * @param table the table
+     * @return the maximum key or null if unknown
+     */
+    byte[] getMaxKey(TableRef table);
+    
+    /**
+     * Manually update the cached table statistics
+     * @param table the table
+     * @throws SQLException
+     */
+    void updateStats(TableRef table) throws SQLException;
+}


[02/51] [partial] Initial commit of master branch from github

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/site/markdown/Phoenix-in-15-minutes-or-less.md
----------------------------------------------------------------------
diff --git a/phoenix-core/src/site/markdown/Phoenix-in-15-minutes-or-less.md b/phoenix-core/src/site/markdown/Phoenix-in-15-minutes-or-less.md
new file mode 100644
index 0000000..bc01b18
--- /dev/null
+++ b/phoenix-core/src/site/markdown/Phoenix-in-15-minutes-or-less.md
@@ -0,0 +1,80 @@
+# Phoenix in 15 minutes or less
+
+*<strong>What is this new [Phoenix](index.html) thing I've been hearing about?</strong>*<br/>
+Phoenix is an open source SQL skin for HBase. You use the standard JDBC APIs instead of the regular HBase client APIs to create tables, insert data, and query your HBase data.
+
+*<strong>Doesn't putting an extra layer between my application and HBase just slow things down?</strong>*<br/>
+Actually, no. Phoenix achieves as good or likely better [performance](performance.html) than if you hand-coded it yourself (not to mention with a heck of a lot less code) by:
+* compiling your SQL queries to native HBase scans
+* determining the optimal start and stop for your scan key
+* orchestrating the parallel execution of your scans
+* bringing the computation to the data by
+  * pushing the predicates in your where clause to a server-side filter
+  * executing aggregate queries through server-side hooks (called co-processors)
+
+In addition to these items, we've got some interesting enhancements in the works to further optimize performance:
+* secondary indexes to improve performance for queries on non row key columns 
+* stats gathering to improve parallelization and guide choices between optimizations 
+* skip scan filter to optimize IN, LIKE, and OR queries
+* optional salting of row keys to evenly distribute write load
+
+*<strong>Ok, so it's fast. But why SQL? It's so 1970s</strong>*<br/>
+Well, that's kind of the point: give folks something with which they're already familiar. What better way to spur the adoption of HBase? On top of that, using JDBC and SQL:
+* Reduces the amount of code users need to write
+* Allows for performance optimizations transparent to the user
+* Opens the door for leveraging and integrating lots of existing tooling
+
+*<strong>But how can SQL support my favorite HBase technique of x,y,z</strong>*<br/>
+Didn't make it to the last HBase Meetup did you? SQL is just a way of expressing *<strong>what you want to get</strong>* not *<strong>how you want to get it</strong>*. Check out my [presentation](http://files.meetup.com/1350427/IntelPhoenixHBaseMeetup.ppt) for various existing and to-be-done Phoenix features to support your favorite HBase trick. Have ideas of your own? We'd love to hear about them: file an [issue](issues.html) for us and/or join our [mailing list](mailing_list.html).
+
+*<strong>Blah, blah, blah - I just want to get started!</strong>*<br/>
+Ok, great! Just follow our [install instructions](download.html#Installation):
+* [download](download.html) and expand our installation tar
+* copy the phoenix jar into the HBase lib directory of every region server
+* restart the region servers
+* add the phoenix client jar to the classpath of your HBase client
+* download and [setup SQuirrel](download.html#SQL-Client) as your SQL client so you can issue adhoc SQL against your HBase cluster
+
+*<strong>I don't want to download and setup anything else!</strong>*<br/>
+Ok, fair enough - you can create your own SQL scripts and execute them using our command line tool instead. Let's walk through an example now. In the bin directory of your install location:
+* Create us_population.sql file
+<pre><code>CREATE TABLE IF NOT EXISTS us_population (
+      state CHAR(2) NOT NULL,
+      city VARCHAR NOT NULL,
+      population BIGINT
+      CONSTRAINT my_pk PRIMARY KEY (state, city));</code></pre>
+* Create us_population.csv file
+<pre><code>NY,New York,8143197
+CA,Los Angeles,3844829
+IL,Chicago,2842518
+TX,Houston,2016582
+PA,Philadelphia,1463281
+AZ,Phoenix,1461575
+TX,San Antonio,1256509
+CA,San Diego,1255540
+TX,Dallas,1213825
+CA,San Jose,912332
+</code></pre>
+* Create us_population_queries.sql file
+<pre><code>SELECT state as "State",count(city) as "City Count",sum(population) as "Population Sum"
+FROM us_population
+GROUP BY state
+ORDER BY sum(population) DESC;
+</code></pre>
+* Execute the following command from a command terminal
+<pre><code>./psql.sh &lt;your_zookeeper_quorum&gt; us_population.sql us_population.csv us_population_queries.sql
+</code></pre>
+
+Congratulations! You've just created your first Phoenix table, inserted data into it, and executed an aggregate query with just a few lines of code in 15 minutes or less! 
+
+*<strong>Big deal - 10 rows! What else you got?</strong>*<br/>
+Ok, ok - tough crowd. Check out our <code>bin/performance.sh</code> script to create as many rows as you want, for any schema you come up with, and run timed queries against it.
+
+*<strong>Why is it called Phoenix anyway? Did some other project crash and burn and this is the next generation?</strong>*<br/>
+I'm sorry, but we're out of time and space, so we'll have to answer that next time!
+
+Thanks for your time,<br/>
+James Taylor<br/>
+http://phoenix-hbase.blogspot.com/
+<br/>
+@JamesPlusPlus<br/>

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/site/markdown/building.md
----------------------------------------------------------------------
diff --git a/phoenix-core/src/site/markdown/building.md b/phoenix-core/src/site/markdown/building.md
new file mode 100644
index 0000000..92f9a6f
--- /dev/null
+++ b/phoenix-core/src/site/markdown/building.md
@@ -0,0 +1,20 @@
+# Building Phoenix Project
+
+Phoenix is a fully mavenized project. That means you can build simply by doing:
+```
+ $ mvn package
+```
+
+builds, test and package Phoenix and put the resulting jars (phoenix-[version].jar and phoenix-[version]-client.jar) in the generated target/ directory.
+
+To build, but skip running the tests, you can do:
+```
+ $ mvn package -DskipTests
+```
+To only build the generated parser (i.e. <code>PhoenixSQLLexer</code> and <code>PhoenixSQLParser</code>), you can do:
+```
+ $ mvn process-sources
+```
+
+To build an Eclipse project, install the m2e plugin and do an File->Import...->Import Existing Maven Projects selecting the root directory of Phoenix.
+

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/site/markdown/download.md
----------------------------------------------------------------------
diff --git a/phoenix-core/src/site/markdown/download.md b/phoenix-core/src/site/markdown/download.md
new file mode 100644
index 0000000..147bc78
--- /dev/null
+++ b/phoenix-core/src/site/markdown/download.md
@@ -0,0 +1,84 @@
+## Available Phoenix Downloads
+
+### Download link will be available soon.
+
+<br/>
+
+### Installation ###
+To install a pre-built phoenix, use these directions:
+
+* Download and expand the latest phoenix-[version]-install.tar
+* Add the phoenix-[version].jar to the classpath of every HBase region server. An easy way to do this is to copy it into the HBase lib directory.
+* Restart all region servers.
+* Add the phoenix-[version]-client.jar to the classpath of any Phoenix client.
+
+### Getting Started ###
+Wanted to get started quickly? Take a look at our [FAQs](faq.html) and take our quick start guide [here](Phoenix-in-15-minutes-or-less.html).
+
+<h4>Command Line</h4>
+
+A terminal interface to execute SQL from the command line is now bundled with Phoenix. To start it, execute the following from the bin directory:
+
+	$ sqlline.sh localhost
+
+To execute SQL scripts from the command line, you can include a SQL file argument like this:
+
+	$ sqlline.sh localhost ../examples/stock_symbol.sql
+
+![sqlline](images/sqlline.png)
+
+For more information, see the [manual](http://www.hydromatic.net/sqlline/manual.html).
+
+<h5>Loading Data</h5>
+
+In addition, you can use the bin/psql.sh to load CSV data or execute SQL scripts. For example:
+
+        $ psql.sh localhost ../examples/web_stat.sql ../examples/web_stat.csv ../examples/web_stat_queries.sql
+
+Other alternatives include:
+* Using our [map-reduce based CSV loader](mr_dataload.html) for bigger data sets
+* [Mapping an existing HBase table to a Phoenix table](index.html#Mapping-to-an-Existing-HBase-Table) and using the [UPSERT SELECT](language/index.html#upsert_select) command to populate a new table.
+* Populating the table through our [UPSERT VALUES](language/index.html#upsert_values) command.
+
+<h4>SQL Client</h4>
+
+If you'd rather use a client GUI to interact with Phoenix, download and install [SQuirrel](http://squirrel-sql.sourceforge.net/). Since Phoenix is a JDBC driver, integration with tools such as this are seamless. Here are the setup steps necessary:
+
+1. Remove prior phoenix-[version]-client.jar from the lib directory of SQuirrel
+2. Copy the phoenix-[version]-client.jar into the lib directory of SQuirrel (Note that on a Mac, this is the *internal* lib directory).
+3. Start SQuirrel and add new driver to SQuirrel (Drivers -> New Driver)
+4. In Add Driver dialog box, set Name to Phoenix
+5. Press List Drivers button and org.apache.phoenix.jdbc.PhoenixDriver should be automatically populated in the Class Name textbox. Press OK to close this dialog.
+6. Switch to Alias tab and create the new Alias (Aliases -> New Aliases)
+7. In the dialog box, Name: _any name_, Driver: Phoenix, User Name: _anything_, Password: _anything_
+8. Construct URL as follows: jdbc:phoenix: _zookeeper quorum server_. For example, to connect to a local HBase use: jdbc:phoenix:localhost
+9. Press Test (which should succeed if everything is setup correctly) and press OK to close.
+10. Now double click on your newly created Phoenix alias and click Connect. Now you are ready to run SQL queries against Phoenix.
+
+Through SQuirrel, you can issue SQL statements in the SQL tab (create tables, insert data, run queries), and inspect table metadata in the Object tab (i.e. list tables, their columns, primary keys, and types).
+
+![squirrel](images/squirrel.png)
+
+### Samples ###
+The best place to see samples are in our unit tests under src/test/java. The ones in the endToEnd package are tests demonstrating how to use all aspects of the Phoenix JDBC driver. We also have some examples in the examples directory.
+
+### Phoenix Client - Server Compatibility
+
+Major and minor version should match between client and server (patch version can mismatch). Following is the list of compatible client and server version(s). It is recommended that same client and server version are used. 
+
+Phoenix Client Version | Compatible Server Versions
+-----------------------|---
+1.0.0 | 1.0.0
+1.1.0 | 1.1.0
+1.2.0 | 1.2.0, 1.2.1
+1.2.1 | 1.2.0, 1.2.1
+2.0.0 | 2.0.0, 2.0.1, 2.0.2
+2.0.1 | 2.0.0, 2.0.1, 2.0.2
+2.0.2 | 2.0.0, 2.0.1, 2.0.2
+2.1.0 | 2.1.0, 2.1.1, 2.1.2
+2.1.1 | 2.1.0, 2.1.1, 2.1.2
+2.1.2 | 2.1.0, 2.1.1, 2.1.2
+2.2.0 | 2.2.0, 2.2.1
+2.2.1 | 2.2.0, 2.2.1
+
+[![githalytics.com alpha](https://cruel-carlota.pagodabox.com/33878dc7c0522eed32d2d54db9c59f78 "githalytics.com")](http://githalytics.com/forcedotcom/phoenix.git)

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/site/markdown/dynamic_columns.md
----------------------------------------------------------------------
diff --git a/phoenix-core/src/site/markdown/dynamic_columns.md b/phoenix-core/src/site/markdown/dynamic_columns.md
new file mode 100644
index 0000000..0c7d9ce
--- /dev/null
+++ b/phoenix-core/src/site/markdown/dynamic_columns.md
@@ -0,0 +1,17 @@
+# Dynamic Columns
+
+Sometimes defining a static schema up front is not feasible. Instead, a subset of columns may be specified at table [create](language/index.html#create) time while the rest would be specified at [query](language/index.html#select) time. As of Phoenix 1.2, specifying columns dynamically is now supported by allowing column definitions to included in parenthesis after the table in the <code>FROM</code> clause on a <code>SELECT</code> statement. Although this is not standard SQL, it is useful to surface this type of functionality to leverage the late binding ability of HBase.
+
+For example:
+
+    SELECT eventTime, lastGCTime, usedMemory, maxMemory
+    FROM EventLog(lastGCTime TIME, usedMemory BIGINT, maxMemory BIGINT)
+    WHERE eventType = 'OOM' AND lastGCTime < eventTime - 1
+
+Where you may have defined only a subset of your event columns at create time, since each event type may have different properties:
+
+    CREATE TABLE EventLog (
+        eventId BIGINT NOT NULL,
+        eventTime TIME NOT NULL,
+        eventType CHAR(3) NOT NULL
+        CONSTRAINT pk PRIMARY KEY (eventId, eventTime))

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/site/markdown/faq.md
----------------------------------------------------------------------
diff --git a/phoenix-core/src/site/markdown/faq.md b/phoenix-core/src/site/markdown/faq.md
new file mode 100644
index 0000000..cbcfc0a
--- /dev/null
+++ b/phoenix-core/src/site/markdown/faq.md
@@ -0,0 +1,279 @@
+# F.A.Q.
+
+* [I want to get started. Is there a Phoenix Hello World?](#I_want_to_get_started_Is_there_a_Phoenix_Hello_World)
+* [Is there a way to bulk load in Phoenix?](#Is_there_a_way_to_bulk_load_in_Phoenix)
+* [How do I create a VIEW in Phoenix? What's the difference between a VIEW and a TABLE?](#How_I_create_Views_in_Phoenix_Whatnulls_the_difference_between_ViewsTables)
+* [Are there any tips for optimizing Phoenix?](#Are_there_any_tips_for_optimizing_Phoenix)
+* [How do I create Secondary Index on a table?](#How_do_I_create_Secondary_Index_on_a_table)
+* [Why isn't my secondary index being used?](#Why_isnnullt_my_secondary_index_being_used)
+* [How fast is Phoenix? Why is it so fast?](#How_fast_is_Phoenix_Why_is_it_so_fast)
+* [How do I connect to secure HBase cluster?](#How_do_I_connect_to_secure_HBase_cluster)
+* [How do I connect with HBase running on Hadoop-2?](#How_do_I_connect_with_HBase_running_on_Hadoop-2)
+* [Can phoenix work on tables with arbitrary timestamp as flexible as HBase API?](#Can_phoenix_work_on_tables_with_arbitrary_timestamp_as_flexible_as_HBase_API)
+* [Why isn't my query doing a RANGE SCAN?](#Why_isnnullt_my_query_doing_a_RANGE_SCAN)
+
+
+### I want to get started. Is there a Phoenix _Hello World_?
+
+*Pre-requisite:* Download latest Phoenix from [here](download.html)
+and copy phoenix-*.jar to HBase lib folder and restart HBase.
+
+**1. Using console**
+
+1. Start Sqlline: `$ sqlline.sh [zookeeper]`
+2. Execute the following statements when Sqlline connects: 
+
+```
+create table test (mykey integer not null primary key, mycolumn varchar);
+upsert into test values (1,'Hello');
+upsert into test values (2,'World!');
+select * from test;
+```
+
+3. You should get the following output
+
+``` 
++-------+------------+
+| MYKEY |  MYCOLUMN  |
++-------+------------+
+| 1     | Hello      |
+| 2     | World!     |
++-------+------------+
+``` 
+
+
+**2. Using java**
+
+Create test.java file with the following content:
+
+``` 
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.PreparedStatement;
+import java.sql.Statement;
+
+public class test {
+
+	public static void main(String[] args) throws SQLException {
+		Statement stmt = null;
+		ResultSet rset = null;
+		
+		Connection con = DriverManager.getConnection("jdbc:phoenix:[zookeeper]");
+		stmt = con.createStatement();
+		
+		stmt.executeUpdate("create table test (mykey integer not null primary key, mycolumn varchar)");
+		stmt.executeUpdate("upsert into test values (1,'Hello')");
+		stmt.executeUpdate("upsert into test values (2,'World!')");
+		con.commit();
+		
+		PreparedStatement statement = con.prepareStatement("select * from test");
+		rset = statement.executeQuery();
+		while (rset.next()) {
+			System.out.println(rset.getString("mycolumn"));
+		}
+		statement.close();
+		con.close();
+	}
+}
+``` 
+Compile and execute on command line
+
+`$ javac test.java`
+
+`$ java -cp "../phoenix-[version]-client.jar:." test`
+
+
+You should get the following output
+
+`Hello`
+`World!`
+
+
+
+### Is there a way to bulk load in Phoenix?
+
+**Map Reduce**
+
+See the example [here](mr_dataload.html) Credit: Arun Singh
+
+**CSV**
+
+CSV data can be bulk loaded with built in utility named psql. Typical upsert rates are 20K - 50K rows per second (depends on how wide are the rows).
+
+Usage example:  
+Create table using psql
+`$ psql.sh [zookeeper] ../examples/web_stat.sql`  
+
+Upsert CSV bulk data
+`$ psql.sh [zookeeper] ../examples/web_stat.csv`
+
+
+
+### How I create Views in Phoenix? What's the difference between Views/Tables?
+
+You can create both a Phoenix table or view through the CREATE TABLE/CREATE VIEW DDL statement on a pre-existing HBase table. In both cases, we'll leave the HBase metadata as-is, except for with a TABLE we turn KEEP_DELETED_CELLS on. For CREATE TABLE, we'll create any metadata (table, column families) that doesn't already exist. We'll also add an empty key value for each row so that queries behave as expected (without requiring all columns to be projected during scans).
+
+The other caveat is that the way the bytes were serialized must match the way the bytes are serialized by Phoenix. For VARCHAR,CHAR, and UNSIGNED_* types, we use the HBase Bytes methods. The CHAR type expects only single-byte characters and the UNSIGNED types expect values greater than or equal to zero.
+
+Our composite row keys are formed by simply concatenating the values together, with a zero byte character used as a separator after a variable length type.
+
+If you create an HBase table like this:
+
+`create 't1', {NAME => 'f1', VERSIONS => 5}`
+
+then you have an HBase table with a name of 't1' and a column family with a name of 'f1'. Remember, in HBase, you don't model the possible KeyValues or the structure of the row key. This is the information you specify in Phoenix above and beyond the table and column family.
+
+So in Phoenix, you'd create a view like this:
+
+`CREATE VIEW "t1" ( pk VARCHAR PRIMARY KEY, "f1".val VARCHAR )`
+
+The "pk" column declares that your row key is a VARCHAR (i.e. a string) while the "f1".val column declares that your HBase table will contain KeyValues with a column family and column qualifier of "f1":VAL and that their value will be a VARCHAR.
+
+Note that you don't need the double quotes if you create your HBase table with all caps names (since this is how Phoenix normalizes strings, by upper casing them). For example, with:
+
+`create 'T1', {NAME => 'F1', VERSIONS => 5}`
+
+you could create this Phoenix view:
+
+`CREATE VIEW t1 ( pk VARCHAR PRIMARY KEY, f1.val VARCHAR )`
+
+Or if you're creating new HBase tables, just let Phoenix do everything for you like this (No need to use the HBase shell at all.):
+
+`CREATE TABLE t1 ( pk VARCHAR PRIMARY KEY, val VARCHAR )`
+
+
+
+### Are there any tips for optimizing Phoenix?
+
+* Use **Salting** to increase read/write performance
+Salting can significantly increase read/write performance by pre-splitting the data into multiple regions. Although Salting will yield better performance in most scenarios. 
+
+Example:
+
+` CREATE TABLE TEST (HOST VARCHAR NOT NULL PRIMARY KEY, DESCRIPTION VARCHAR) SALT_BUCKETS=16`
+
+Note: Ideally for a 16 region server cluster with quad-core CPUs, choose salt buckets between 32-64 for optimal performance.
+
+* **Per-split** table
+Salting does automatic table splitting but in case you want to exactly control where table split occurs with out adding extra byte or change row key order then you can pre-split a table. 
+
+Example: 
+
+` CREATE TABLE TEST (HOST VARCHAR NOT NULL PRIMARY KEY, DESCRIPTION VARCHAR) SPLIT ON ('CS','EU','NA')`
+
+* Use **multiple column families**
+
+Column family contains related data in separate files. If you query use selected columns then it make sense to group those columns together in a column family to improve read performance.
+
+Example:
+
+Following create table DDL will create two column familes A and B.
+
+` CREATE TABLE TEST (MYKEY VARCHAR NOT NULL PRIMARY KEY, A.COL1 VARCHAR, A.COL2 VARCHAR, B.COL3 VARCHAR)`
+
+* Use **compression**
+On disk compression improves performance on large tables
+
+Example: 
+
+` CREATE TABLE TEST (HOST VARCHAR NOT NULL PRIMARY KEY, DESCRIPTION VARCHAR) COMPRESSION='GZ'`
+
+* Create **indexes**
+See [faq.html#/How_do_I_create_Secondary_Index_on_a_table](faq.html#/How_do_I_create_Secondary_Index_on_a_table)
+
+* **Optimize cluster** parameters
+See http://hbase.apache.org/book/performance.html
+
+* **Optimize Phoenix** parameters
+See [tuning.html](tuning.html)
+
+
+
+### How do I create Secondary Index on a table?
+
+Starting with Phoenix version 2.1, Phoenix supports index over mutable and immutable data. Note that Phoenix 2.0.x only supports Index over immutable data. Index write performance index with immutable table is slightly faster than mutable table however data in immutable table cannot be updated.
+
+Example
+
+* Create table
+
+Immutable table: `create table test (mykey varchar primary key, col1 varchar, col2 varchar) IMMUTABLE_ROWS=true;`
+
+Mutable table: `create table test (mykey varchar primary key, col1 varchar, col2 varchar);`
+
+* Creating index on col2
+
+`create index idx on test (col2)`
+
+* Creating index on col1 and a covered index on col2
+
+`create index idx on test (col1) include (col2)`
+
+Upsert rows in this test table and Phoenix query optimizer will choose correct index to use. You can see in [explain plan](language/index.html#explain) if Phoenix is using the index table. You can also give a [hint](language/index.html#hint) in Phoenix query to use a specific index.
+
+
+
+### Why isn't my secondary index being used?
+
+The secondary index won't be used unless all columns used in the query are in it ( as indexed or covered columns). All columns making up the primary key of the data table will automatically be included in the index.
+
+Example: DDL `create table usertable (id varchar primary key, firstname varchar, lastname varchar); create index idx_name on usertable (firstname);`
+
+Query: DDL `select id, firstname, lastname from usertable where firstname = 'foo';`
+
+Index would not be used in this case as lastname is not part of indexed or covered column. This can be verified by looking at the explain plan. To fix this create index that has either lastname part of index or covered column. Example: `create idx_name on usertable (firstname) include (lastname);`
+
+
+### How fast is Phoenix? Why is it so fast?
+
+Phoenix is fast. Full table scan of 100M rows usually completes in 20 seconds (narrow table on a medium sized cluster). This time come down to few milliseconds if query contains filter on key columns. For filters on non-key columns or non-leading key columns, you can add index on these columns which leads to performance equivalent to filtering on key column by making copy of table with indexed column(s) part of key.
+
+Why is Phoenix fast even when doing full scan:
+
+1. Phoenix chunks up your query using the region boundaries and runs them in parallel on the client using a configurable number of threads 
+2. The aggregation will be done in a coprocessor on the server-side, collapsing the amount of data that gets returned back to the client rather than returning it all. 
+
+
+
+### How do I connect to secure HBase cluster?
+Check out excellent post by Anil Gupta 
+http://bigdatanoob.blogspot.com/2013/09/connect-phoenix-to-secure-hbase-cluster.html
+
+
+
+### How do I connect with HBase running on Hadoop-2?
+Hadoop-2 profile exists in Phoenix pom.xml. 
+
+
+### Can phoenix work on tables with arbitrary timestamp as flexible as HBase API?
+By default, Phoenix let's HBase manage the timestamps and just shows you the latest values for everything. However, Phoenix also allows arbitrary timestamps to be supplied by the user. To do that you'd specify a "CurrentSCN" (or PhoenixRuntime.CURRENT_SCN_ATTRIB if you want to use our constant) at connection time, like this:
+
+    Properties props = new Properties();
+    props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts));
+    Connection conn = DriverManager.connect(myUrl, props);
+
+    conn.createStatement().execute("UPSERT INTO myTable VALUES ('a')");
+    conn.commit();
+The above is equivalent to doing this with the HBase API:
+
+    myTable.put(Bytes.toBytes('a'),ts);
+By specifying a CurrentSCN, you're telling Phoenix that you want everything for that connection to be done at that timestamp. Note that this applies to queries done on the connection as well - for example, a query over myTable above would not see the data it just upserted, since it only sees data that was created before its CurrentSCN property. This provides a way of doing snapshot, flashback, or point-in-time queries.
+
+Keep in mind that creating a new connection is *not* an expensive operation. The same underlying HConnection is used for all connections to the same cluster, so it's more or less like instantiating a few objects.
+
+
+### Why isn't my query doing a RANGE SCAN?
+
+`DDL: CREATE TABLE TEST (pk1 char(1) not null, pk2 char(1) not null, pk3 char(1) not null, non-pk varchar CONSTRAINT PK PRIMARY KEY(pk1, pk2, pk3));`
+
+RANGE SCAN means that only a subset of the rows in your table will be scanned over. This occurs if you use one or more leading columns from your primary key constraint. Query that is not filtering on leading PK columns ex. `select * from test where pk2='x' and pk3='y';` will result in full scan whereas the following query will result in range scan `select * from test where pk1='x' and pk2='y';`. Note that you can add a secondary index on your "pk2" and "pk3" columns and that would cause a range scan to be done for the first query (over the index table).
+
+DEGENERATE SCAN means that a query can't possibly return any rows. If we can determine that at compile time, then we don't bother to even run the scan.
+
+FULL SCAN means that all rows of the table will be scanned over (potentially with a filter applied if you have a WHERE clause)
+
+SKIP SCAN means that either a subset or all rows in your table will be scanned over, however it will skip large groups of rows depending on the conditions in your filter. See this blog for more detail. We don't do a SKIP SCAN if you have no filter on the leading primary key columns, but you can force a SKIP SCAN by using the /*+ SKIP_SCAN */ hint. Under some conditions, namely when the cardinality of your leading primary key columns is low, it will be more efficient than a FULL SCAN.
+
+

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/site/markdown/flume.md
----------------------------------------------------------------------
diff --git a/phoenix-core/src/site/markdown/flume.md b/phoenix-core/src/site/markdown/flume.md
new file mode 100644
index 0000000..6cc9251
--- /dev/null
+++ b/phoenix-core/src/site/markdown/flume.md
@@ -0,0 +1,42 @@
+# Apache Flume Plugin
+
+The plugin enables us to reliably and efficiently stream large amounts of data/logs onto HBase using the Phoenix API. The necessary configuration of the custom Phoenix sink and the Event Serializer has to be configured in the Flume configuration file for the Agent. Currently, the only supported Event serializer is a RegexEventSerializer which primarily breaks the Flume Event body based on the regex specified in the configuration file.   
+
+#### Prerequisites:
+
+* Phoenix v 3.0.0 SNAPSHOT +
+* Flume 1.4.0 +
+
+#### Installation & Setup:
+
+1. Download and build Phoenix v 0.3.0 SNAPSHOT
+2. Follow the instructions as specified [here](building.html) to build the project as the Flume plugin is still under beta
+3. Create a directory plugins.d within $FLUME_HOME directory. Within that, create a sub-directories phoenix-sink/lib 
+4. Copy the generated phoenix-3.0.0-SNAPSHOT-client.jar onto $FLUME_HOME/plugins.d/phoenix-sink/lib
+
+#### Configuration:
+  
+Property Name             |Default| Description
+--------------------------|-------|---
+type                      |       |org.apache.phoenix.flume.sink.PhoenixSink
+batchSize                 |100    |Default number of events per transaction 
+zookeeperQuorum           |       |Zookeeper quorum of the HBase cluster
+table                     |       |The name of the table in HBase to write to.
+ddl                       |       |The CREATE TABLE query for the HBase table where the events will be                                                    upserted to. If specified, the query will be executed. Recommended to include the IF NOT EXISTS clause in the ddl.
+serializer                |regex  |Event serializers for processing the Flume Event . Currently , only regex is supported.
+serializer.regex          |(.*)   |The regular expression for parsing the event. 
+serializer.columns        |       |The columns that will be extracted from the Flume event for inserting         into HBase. 
+serializer.headers        |       |Headers of the Flume Events that go as part of the UPSERT query. The  data type for these columns are VARCHAR by default.
+serializer.rowkeyType     |     |A custom row key generator . Can be one of timestamp,date,uuid,random and     nanotimestamp. This should be configured in cases  where we need a custom row key value to be auto generated and set for the primary key column.
+
+
+For an example configuration for ingesting Apache access logs onto Phoenix, see [this](https://github.com/forcedotcom/phoenix/blob/master/src/main/config/apache-access-logs.properties) property file. Here we are using UUID as a row key generator for the primary key.	
+		   	
+#### Starting the agent:
+       $ bin/flume-ng agent -f conf/flume-conf.properties -c ./conf -n agent
+
+#### Monitoring:
+   For monitoring the agent and the sink process , enable JMX via flume-env.sh($FLUME_HOME/conf/flume-env.sh) script. Ensure you have the following line uncommented.
+   
+    JAVA_OPTS="-Xms1g -Xmx1g -Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.port=3141 -Dcom.sun.management.jmxremote.authenticate=false -Dcom.sun.management.jmxremote.ssl=false"   	
+	

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/site/markdown/index.md
----------------------------------------------------------------------
diff --git a/phoenix-core/src/site/markdown/index.md b/phoenix-core/src/site/markdown/index.md
new file mode 100644
index 0000000..8b9f0b0
--- /dev/null
+++ b/phoenix-core/src/site/markdown/index.md
@@ -0,0 +1,69 @@
+# Overview
+
+Apache Phoenix is a SQL skin over HBase delivered as a client-embedded JDBC driver targeting low latency queries over HBase data. Apache Phoenix takes your SQL query, compiles it into a series of HBase scans, and orchestrates the running of those scans to produce regular JDBC result sets. The table metadata is stored in an HBase table and versioned, such that snapshot queries over prior versions will automatically use the correct schema. Direct use of the HBase API, along with coprocessors and custom filters, results in [performance](performance.html) on the order of milliseconds for small queries, or seconds for tens of millions of rows. 
+
+## Mission
+Become the standard means of accessing HBase data through a well-defined, industry standard API.
+
+## Quick Start
+Tired of reading already and just want to get started? Take a look at our [FAQs](faq.html), listen to the Apache Phoenix talks from [Hadoop Summit 2013](http://www.youtube.com/watch?v=YHsHdQ08trg) and [HBaseConn 2013](http://www.cloudera.com/content/cloudera/en/resources/library/hbasecon/hbasecon-2013--how-and-why-phoenix-puts-the-sql-back-into-nosql-video.html), and jump over to our quick start guide [here](Phoenix-in-15-minutes-or-less.html).
+
+##SQL Support##
+To see what's supported, go to our [language reference](language/index.html). It includes all typical SQL query statement clauses, including `SELECT`, `FROM`, `WHERE`, `GROUP BY`, `HAVING`, `ORDER BY`, etc. It also supports a full set of DML commands as well as table creation and versioned incremental alterations through our DDL commands. We try to follow the SQL standards wherever possible.
+
+<a id="connStr"></a>Use JDBC to get a connection to an HBase cluster like this:
+
+        Connection conn = DriverManager.getConnection("jdbc:phoenix:server1,server2:3333");
+where the connection string is composed of:
+<code><small>jdbc:phoenix</small></code> [ <code><small>:&lt;zookeeper quorum&gt;</small></code> [ <code><small>:&lt;port number&gt;</small></code> ] [ <code><small>:&lt;root node&gt;</small></code> ] ]
+
+For any omitted part, the relevant property value, hbase.zookeeper.quorum, hbase.zookeeper.property.clientPort, and zookeeper.znode.parent will be used from hbase-site.xml configuration file.
+
+Here's a list of what is currently **not** supported:
+
+* **Full Transaction Support**. Although we allow client-side batching and rollback as described [here](#transactions), we do not provide transaction semantics above and beyond what HBase gives you out-of-the-box.
+* **Derived tables**. Nested queries are coming soon.
+* **Relational operators**. Union, Intersect, Minus.
+* **Miscellaneous built-in functions**. These are easy to add - read this [blog](http://phoenix-hbase.blogspot.com/2013/04/how-to-add-your-own-built-in-function.html) for step by step instructions.
+
+##<a id="schema"></a>Schema##
+
+Apache Phoenix supports table creation and versioned incremental alterations through DDL commands. The table metadata is stored in an HBase table.
+
+A Phoenix table is created through the [CREATE TABLE](language/index.html#create) DDL command and can either be:
+
+1. **built from scratch**, in which case the HBase table and column families will be created automatically.
+2. **mapped to an existing HBase table**, by creating either a read-write TABLE or a read-only VIEW, with the caveat that the binary representation of the row key and key values must match that of the Phoenix data types (see [Data Types reference](datatypes.html) for the detail on the binary representation).
+    * For a read-write TABLE, column families will be created automatically if they don't already exist. An empty key value will be added to the first column family of each existing row to minimize the size of the projection for queries.
+    * For a read-only VIEW, all column families must already exist. The only change made to the HBase table will be the addition of the Phoenix coprocessors used for query processing. The primary use case for a VIEW is to transfer existing data into a Phoenix table, since data modification are not allowed on a VIEW and query performance will likely be less than as with a TABLE.
+
+All schema is versioned, and prior versions are stored forever. Thus, snapshot queries over older data will pick up and use the correct schema for each row.
+
+####Salting
+A table could also be declared as salted to prevent HBase region hot spotting. You just need to declare how many salt buckets your table has, and Phoenix will transparently manage the salting for you. You'll find more detail on this feature [here](salted.html), along with a nice comparison on write throughput between salted and unsalted tables [here](performance.htm#salting).
+
+####Schema at Read-time
+Another schema-related feature allows columns to be defined dynamically at query time. This is useful in situations where you don't know in advance all of the columns at create time. You'll find more details on this feature [here](dynamic_columns.html).
+
+####<a id="mapping"></a>Mapping to an Existing HBase Table
+Apache Phoenix supports mapping to an existing HBase table through the [CREATE TABLE](language/index.html#create) and [CREATE VIEW](language/index.html#create) DDL statements. In both cases, the HBase metadata is left as-is, except for with CREATE TABLE the [KEEP_DELETED_CELLS](http://hbase.apache.org/book/cf.keep.deleted.html) option is enabled to allow for flashback queries to work correctly. For CREATE TABLE, any HBase metadata (table, column families) that doesn't already exist will be created. Note that the table and column family names are case sensitive, with Phoenix upper-casing all names. To make a name case sensitive in the DDL statement, surround it with double quotes as shown below:
+      <pre><code>CREATE VIEW "MyTable" ("a".ID VARCHAR PRIMARY KEY)</code></pre>
+
+For CREATE TABLE, an empty key value will also be added for each row so that queries behave as expected (without requiring all columns to be projected during scans). For CREATE VIEW, this will not be done, nor will any HBase metadata be created. Instead the existing HBase metadata must match the metadata specified in the DDL statement or a <code>ERROR 505 (42000): Table is read only</code> will be thrown.
+
+The other caveat is that the way the bytes were serialized in HBase must match the way the bytes are expected to be serialized by Phoenix. For VARCHAR,CHAR, and UNSIGNED_* types, Phoenix uses the HBase Bytes utility methods to perform serialization. The CHAR type expects only single-byte characters and the UNSIGNED types expect values greater than or equal to zero.
+
+Our composite row keys are formed by simply concatenating the values together, with a zero byte character used as a separator after a variable length type. For more information on our type system, see the [Data Type](datatypes.html).
+
+##<a id="transactions"></a>Transactions##
+The DML commands of Apache Phoenix, [UPSERT VALUES](language/index.html#upsert_values), [UPSERT SELECT](language/index.html#upsert_select) and [DELETE](language/index.html#delete), batch pending changes to HBase tables on the client side. The changes are sent to the server when the transaction is committed and discarded when the transaction is rolled back. The only transaction isolation level we support is TRANSACTION_READ_COMMITTED. This includes not being able to see your own uncommitted data as well. Phoenix does not providing any additional transactional semantics beyond what HBase supports when a batch of mutations is submitted to the server. If auto commit is turned on for a connection, then Phoenix will, whenever possible, execute the entire DML command through a coprocessor on the server-side, so performance will improve.
+
+Most commonly, an application will let HBase manage timestamps. However, under some circumstances, an application needs to control the timestamps itself. In this case, a long-valued "CurrentSCN" property may be specified at connection time to control timestamps for any DDL, DML, or query. This capability may be used to run snapshot queries against prior row values, since Phoenix uses the value of this connection property as the max timestamp of scans.
+
+## Metadata ##
+The catalog of tables, their columns, primary keys, and types may be retrieved via the java.sql metadata interfaces: `DatabaseMetaData`, `ParameterMetaData`, and `ResultSetMetaData`. For retrieving schemas, tables, and columns through the DatabaseMetaData interface, the schema pattern, table pattern, and column pattern are specified as in a LIKE expression (i.e. % and _ are wildcards escaped through the \ character). The table catalog argument to the metadata APIs deviates from a more standard relational database model, and instead is used to specify a column family name (in particular to see all columns in a given column family).
+
+<hr/>
+## Disclaimer ##
+Apache Phoenix is an effort undergoing incubation at The Apache Software Foundation (ASF), sponsored by the [Apache Incubator PMC](http://incubator.apache.org/). Incubation is required of all newly accepted projects until a further review indicates that the infrastructure, communications, and decision making process have stabilized in a manner consistent with other successful ASF projects. While incubation status is not necessarily a reflection of the completeness or stability of the code, it does indicate that the project has yet to be fully endorsed by the ASF.
+<br/><br/><img src="http://incubator.apache.org/images/apache-incubator-logo.png"/>

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/site/markdown/issues.md
----------------------------------------------------------------------
diff --git a/phoenix-core/src/site/markdown/issues.md b/phoenix-core/src/site/markdown/issues.md
new file mode 100644
index 0000000..64ea3ca
--- /dev/null
+++ b/phoenix-core/src/site/markdown/issues.md
@@ -0,0 +1,9 @@
+# Issue Tracking
+
+This project uses JIRA issue tracking and project management application. Issues, bugs, and feature requests should be submitted to the following:
+
+<hr/>
+
+https://issues.apache.org/jira/browse/PHOENIX
+
+<hr/>

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/site/markdown/mailing_list.md
----------------------------------------------------------------------
diff --git a/phoenix-core/src/site/markdown/mailing_list.md b/phoenix-core/src/site/markdown/mailing_list.md
new file mode 100644
index 0000000..fbe9e43
--- /dev/null
+++ b/phoenix-core/src/site/markdown/mailing_list.md
@@ -0,0 +1,14 @@
+# Mailing Lists
+
+These are the mailing lists that have been established for this project. For each list, there is a subscribe, unsubscribe and post link.
+
+<hr/>
+
+Name| Subscribe| Unsubscribe| Post
+--------------------------|----|----|----
+User List | [Subscribe](mailto:user-subscribe@phoenix.incubator.apache.org) | [Unsubscribe](mailto:user-unsubscribe@phoenix.incubator.apache.org) | [Post](mailto:user@phoenix.incubator.apache.org)
+Developer List | [Subscribe](mailto:dev-subscribe@phoenix.incubator.apache.org) | [Unsubscribe](mailto:dev-unsubscribe@phoenix.incubator.apache.org) | [Post](mailto:dev@phoenix.incubator.apache.org)
+Private List | [Subscribe](mailto:private-subscribe@phoenix.incubator.apache.org) | [Unsubscribe](mailto:private-unsubscribe@phoenix.incubator.apache.org) | [Post](mailto:private@phoenix.incubator.apache.org)
+Commits List | [Subscribe](mailto:commits-subscribe@phoenix.incubator.apache.org) | [Unsubscribe](mailto:commits-unsubscribe@phoenix.incubator.apache.org) | [Post](mailto:commits@phoenix.incubator.apache.org)
+
+<hr/>

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/site/markdown/mr_dataload.md
----------------------------------------------------------------------
diff --git a/phoenix-core/src/site/markdown/mr_dataload.md b/phoenix-core/src/site/markdown/mr_dataload.md
new file mode 100644
index 0000000..b0053ac
--- /dev/null
+++ b/phoenix-core/src/site/markdown/mr_dataload.md
@@ -0,0 +1,63 @@
+# Bulk CSV Data Load using Map-Reduce
+
+Phoenix v 2.1 provides support for loading CSV data into a new/existing Phoenix table using Hadoop Map-Reduce. This provides a means of bulk loading CSV data in parallel through map-reduce, yielding better performance in comparison with the existing [psql csv loader](download.html#Loading-Data).
+
+####Sample input CSV data:
+
+```
+12345, John, Doe
+67890, Mary, Poppins
+```
+
+####Compatible Phoenix schema to hold the above CSV data:
+
+     CREATE TABLE ns.example (
+        my_pk bigint not null,
+        m.first_name varchar(50),
+        m.last_name varchar(50) 
+        CONSTRAINT pk PRIMARY KEY (my_pk))
+
+<table>
+<tr><td>Row Key</td><td colspan="2" bgcolor="#00FF00"><center>Column Family (m)</center></td></tr>
+<tr><td><strong>my_pk</strong> BIGINT</td><td><strong>first_name</strong> VARCHAR(50)</td><td><strong>last_name</strong> VARCHAR(50)</td></tr>
+<tr><td>12345</td><td>John</td><td>Doe</td></tr>
+<tr><td>67890</td><td>Mary</td><td>Poppins</td></tr>
+</table>
+
+
+####How to run?
+
+1- Please make sure that Hadoop cluster is working correctly and you are able to run any job like [this](http://wiki.apache.org/hadoop/WordCount). 
+
+2- Copy latest phoenix-[version].jar to hadoop/lib folder on each node or add it to Hadoop classpath.
+
+3- Run the bulk loader job using the script /bin/csv-bulk-loader.sh as below:
+
+```
+./csv-bulk-loader.sh <option value>
+
+<option>  <value>
+-i        CSV data file path in hdfs (mandatory)
+-s        Phoenix schema name (mandatory if not default)
+-t        Phoenix table name (mandatory)
+-sql      Phoenix create table sql file path (mandatory)
+-zk       Zookeeper IP:<port> (mandatory)
+-mr       MapReduce Job Tracker IP:<port> (mandatory)
+-hd       HDFS NameNode IP:<port> (mandatory)
+-o        Output directory path in hdfs (optional)
+-idx      Phoenix index table name (optional, not yet supported)
+-error    Ignore error while reading rows from CSV ? (1-YES | 0-NO, default-1) (optional)
+-help     Print all options (optional)
+```
+Example
+
+```
+./csv-bulk-loader.sh -i hdfs://server:9000/mydir/data.csv -s ns -t example -sql ~/Documents/createTable.sql -zk server:2181 -hd hdfs://server:9000 -mr server:9001
+```
+
+This would create the phoenix table "ns.example" as specified in createTable.sql and will then load the CSV data from the file "data.csv" located in HDFS into the table.
+
+##### Notes
+1. You must provide an explicit column family name in your CREATE TABLE statement for your non primary key columns, as the default column family used by Phoenix is treated specially by HBase because it starts with an underscore.
+2. The current bulk loader does not support the migration of index related data yet. So, if you have created your phoenix table with index, please use the [psql CSV loader](download.html#Loading-Data). 
+3. In case you want to further optimize the map-reduce performance, please refer to the current map-reduce optimization params in the file "src/main/config/csv-bulk-load-config.properties". In case you modify this list, please re-build the phoenix jar and re-run the job as described above.

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/site/markdown/paged.md
----------------------------------------------------------------------
diff --git a/phoenix-core/src/site/markdown/paged.md b/phoenix-core/src/site/markdown/paged.md
new file mode 100644
index 0000000..f01aedb
--- /dev/null
+++ b/phoenix-core/src/site/markdown/paged.md
@@ -0,0 +1,31 @@
+# Paged Queries
+
+Phoenix v 2.1 supports the use in queries of row value constructors, a standard SQL construct to enable paged queries. A row value constructor is an ordered sequence of values delimited by parentheses. For example:
+
+    (4, 'foo', 3.5)
+    ('Doe', 'Jane')
+    (my_col1, my_col2, 'bar')
+
+Just like with regular values, row value constructors may be used in comparison expression like this:
+
+    WHERE (x,y,z) >= ('foo','bar')
+    WHERE (last_name,first_name) = ('Jane','Doe')
+
+Row value constructors are compared by conceptually concatenating the values together and comparing them against each other, with the leftmost part being most significant. Section 8.2 (comparison predicates) of the SQL-92 standard explains this in detail, but here are a few examples of predicates that would evaluate to true:
+
+    (9, 5, 3) > (8, 8)
+    ('foo', 'bar') < 'g'
+    (1,2) = (1,2)
+Row value constructors may also be used in an IN list expression to efficiently query for a set of rows given the composite primary key columns. For example, the following would be optimized to be a point get of three rows:
+
+    WHERE (x,y) IN ((1,2),(3,4),(5,6))
+Another primary use case for row value constructors is to support query-more type functionality by enabling an ordered set of rows to be incrementally stepped through. For example, the following query would step through a set of rows, 20 rows at a time:
+
+    SELECT title, author, isbn, description 
+    FROM library 
+    WHERE published_date > 2010
+    AND (title, author, isbn) > (?, ?, ?)
+    ORDER BY title, author, isbn
+    LIMIT 20
+
+Assuming that the client binds the three bind variables to the values of the last row processed, the next invocation would find the next 20 rows that match the query. If the columns you supply in your row value constructor match in order the columns from your primary key (or from a secondary index), then Phoenix will be able to turn the row value constructor expression into the start row of your scan. This enables a very efficient mechanism to locate _at or after_ a row.

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/site/markdown/performance.md
----------------------------------------------------------------------
diff --git a/phoenix-core/src/site/markdown/performance.md b/phoenix-core/src/site/markdown/performance.md
new file mode 100644
index 0000000..2f6c054
--- /dev/null
+++ b/phoenix-core/src/site/markdown/performance.md
@@ -0,0 +1,86 @@
+# Performance
+
+Phoenix follows the philosophy of **bringing the computation to the data** by using:
+* **coprocessors** to perform operations on the server-side thus minimizing client/server data transfer
+* **custom filters** to prune data as close to the source as possible
+In addition, to minimize any startup costs, Phoenix uses native HBase APIs rather than going through the map/reduce framework.
+## Phoenix vs related products
+Below are charts showing relative performance between Phoenix and some other related products.
+
+### Phoenix vs Hive (running over HDFS and HBase)
+![Phoenix vs Hive](images/PhoenixVsHive.png)
+
+Query: select count(1) from table over 10M and 100M rows. Data is 5 narrow columns. Number of Region 
+Servers: 4 (HBase heap: 10GB, Processor: 6 cores @ 3.3GHz Xeon)
+
+### Phoenix vs Impala (running over HBase)
+![Phoenix vs Impala](images/PhoenixVsImpala.png)
+
+Query: select count(1) from table over 1M and 5M rows. Data is 3 narrow columns. Number of Region Server: 1 (Virtual Machine, HBase heap: 2GB, Processor: 2 cores @ 3.3GHz Xeon)
+
+***
+## Latest Automated Performance Run
+
+[Latest Automated Performance Run](http://phoenix-bin.github.io/client/performance/latest.htm) | 
+[Automated Performance Runs History](http://phoenix-bin.github.io/client/performance/)
+
+***
+
+## Performance improvements in Phoenix 1.2
+
+### Essential Column Family
+Phoenix 1.2 query filter leverages [HBase Filter Essential Column Family](http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.html#isFamilyEssential(byte[]) feature which leads to improved performance when Phoenix query filters on data that is split in multiple column families (cf) by only loading essential cf. In second pass, all cf are are loaded as needed.
+
+Consider the following schema in which data is split in two cf
+`create table t (k varchar not null primary key, a.c1 integer, b.c2 varchar, b.c3 varchar, b.c4 varchar)`. 
+
+Running a query similar to the following shows significant performance when a subset of rows match filter
+`select count(c2) from t where c1 = ?` 
+
+Following chart shows query in-memory performance of running the above query with 10M rows on 4 region servers when 10% of the rows matches the filter. Note: cf-a is approx 8 bytes and cf-b is approx 400 bytes wide.
+
+![Ess. CF](images/perf-esscf.png)
+
+
+### Skip Scan
+
+Skip Scan Filter leverages [SEEK_NEXT_USING_HINT](http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/Filter.ReturnCode.html#SEEK_NEXT_USING_HINT) of HBase Filter. It significantly improves point queries over key columns.
+
+Consider the following schema in which data is split in two cf
+`create table t (k varchar not null primary key, a.c1 integer, b.c2 varchar, b.c3 varchar)`. 
+
+Running a query similar to the following shows significant performance when a subset of rows match filter
+`select count(c1) from t where k in (1% random k's)` 
+
+Following chart shows query in-memory performance of running the above query with 10M rows on 4 region servers when 1% random keys over the entire range passed in query `IN` clause. Note: all varchar columns are approx 15 bytes.
+
+![SkipScan](images/perf-skipscan.png)
+
+
+### Salting
+Salting in Phoenix 1.2 leads to both improved read and write performance by adding an extra hash byte at start of key and pre-splitting data in number of regions. This eliminates hot-spotting of single or few regions servers. Read more about this feature [here](salted.html).
+
+Consider the following schema
+
+`CREATE TABLE T (HOST CHAR(2) NOT NULL,DOMAIN VARCHAR NOT NULL,`
+`FEATURE VARCHAR NOT NULL,DATE DATE NOT NULL,USAGE.CORE BIGINT,USAGE.DB BIGINT,STATS.ACTIVE_VISITOR`
+`INTEGER CONSTRAINT PK PRIMARY KEY (HOST, DOMAIN, FEATURE, DATE)) SALT_BUCKETS = 4`. 
+
+Following chart shows write performance with and without the use of Salting which splits table in 4 regions running on 4 region server cluster (Note: For optimal performance, number of salt buckets should match number of region servers).
+
+![Salted-Write](images/perf-salted-write.png)
+
+Following chart shows in-memory query performance for 10M row table where `host='NA'` filter matches 3.3M rows
+
+`select count(1) from t where host='NA'`
+
+![Salted-Read](images/perf-salted-read.png)
+
+
+### Top-N 
+
+Following chart shows in-memory query time of running the Top-N query over 10M rows using Phoenix 1.2 and Hive over HBase
+
+`select core from t order by core desc limit 10`
+
+![Phoenix vs Hive](images/perf-topn.png)

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/site/markdown/phoenix_on_emr.md
----------------------------------------------------------------------
diff --git a/phoenix-core/src/site/markdown/phoenix_on_emr.md b/phoenix-core/src/site/markdown/phoenix_on_emr.md
new file mode 100644
index 0000000..27ca2bc
--- /dev/null
+++ b/phoenix-core/src/site/markdown/phoenix_on_emr.md
@@ -0,0 +1,43 @@
+# Phoenix on Amazon EMR
+
+Follow these steps to deploy HBase with Phoenix on Amazon's Elastic MapReduce (EMR).
+
+### 1. Amazon EMR Configuration
+
+1. Create a free/paid [EMR account](https://portal.aws.amazon.com/gp/aws/developer/registration/index.html) 
+2. [Download](http://aws.amazon.com/developertools/2264) the latest CLI from and follow the setup instructions [here](http://docs.aws.amazon.com/ElasticMapReduce/latest/DeveloperGuide/emr-cli-install.html).
+
+_Note: Step 2 is a multi-step process in which you would install ruby, gem, create credentials file and create a S3 bucket._
+
+### 2. Deploy HBase with Phoenix on EMR using Web Console
+
+##### Using Web Console
+
+Go to _Elastic MapReduce > Create Cluster_ and follow the steps listed below:
+
+![EMR Create Cluster Page](http://phoenix-bin.github.io/client/images/EMR2.png)
+
+1. Type your cluster name
+2. Set _AMI version_ to _3.0.1_
+3. From _Additional Application_ drop down, select HBase and add.
+4. In _Core_ text box, enter the number of HBase region server(s) you want configured for your cluster
+5. Add a _custom action bootstrap_ from dropdown and specify S3 location: ``s3://beta.elasticmapreduce/bootstrap-actions/phoenix/install-phoenix-bootstrap.sh`` and click add.
+6. Click _Create Cluster_
+
+##### Using CLI
+
+Instead of using _Web Console_, you may use following _CLI_ command to deploy _HBase_ with _Phoenix_:
+
+ ```
+./elastic-mapreduce --create --instance-type c1.xlarge --name 
+PHOENIX_2.2_install --ami-version 3.0.1 --hbase --alive --bootstrap-action 
+"s3://beta.elasticmapreduce/bootstrap-actions/phoenix/install-phoenix-bootstrap.sh"
+```
+
+### 3. Usage
+
+_SSH_ to the _EMR Master_ and CD to _/home/hadoop/hbase/lib/phoenix/bin_
+
+Create test data: ``./performance localhost 1000000 ``
+
+SQL CLI: ``./sqlline localhost``

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/site/markdown/recent.md
----------------------------------------------------------------------
diff --git a/phoenix-core/src/site/markdown/recent.md b/phoenix-core/src/site/markdown/recent.md
new file mode 100644
index 0000000..02bd39b
--- /dev/null
+++ b/phoenix-core/src/site/markdown/recent.md
@@ -0,0 +1,18 @@
+# New Features
+
+As items are implemented from our road map, they are moved here to track the progress we've made:
+
+1. **Joins**. Join support through hash joins (where one side of the query is small enough to fit into memory) is now available in our master branch.
+2. **[Sequences](http://phoenix.incubator.apache.org/sequences.html)**. Support for CREATE/DROP SEQUENCE, NEXT VALUE FOR, and CURRENT VALUE FOR has been implemented and is now available in our master branch.
+2. **Multi-tenancy**. Support for creating multi-tenant tables is now available in our master branch.
+1. **[Secondary Indexes](secondary_indexing.html)**. Allows users to create indexes over mutable or immutable data through a new `CREATE INDEX` DDL command. Behind the scenes, Phoenix creates a separate HBase table with a different row key for the index. At query time, Phoenix takes care of choosing the best table to use based on how much of the row key can be formed. We support getting at the uncommitted <code>List&lt;KeyValue&gt;</code> for both the data and the index tables to allow an HFile to be built without needing an HBase connection using the "connectionless" of our JDBC driver.
+2. **Row Value Constructors**. A standard SQL construct to efficiently locate the row at or after a composite key value. Enables a query-more capability to efficiently step through your data and optimizes IN list of composite key values to be point gets.
+3. **[Map-reduce-based CSV Bulk Loader](mr_dataload.html)** Builds Phoenix-compliant HFiles and load them into HBase.
+2. **Aggregation Enhancements**. <code>COUNT DISTINCT</code>, <code>PERCENTILE</code>, and <code>STDDEV</code> are now supported.
+4. **Type Additions**. The <code>FLOAT</code>, <code>DOUBLE</code>, <code>TINYINT</code>, and <code>SMALLINT</code> are now supported.
+2. **IN/OR/LIKE Optimizations**. When an IN (or the equivalent OR) and a LIKE appears in a query using the leading row key columns, compile it into a skip scanning filter to more efficiently retrieve the query results.
+3. **Support ASC/DESC declaration of primary key columns**. Allow a primary key column to be declared as ascending (the default) or descending such that the row key order can match the desired sort order (thus preventing an extra sort).
+3. **Salting Row Key**. To prevent hot spotting on writes, the row key may be *"salted"* by inserting a leading byte into the row key which is a mod over N buckets of the hash of the entire row key. This ensures even distribution of writes when the row key is a monotonically increasing value (often a timestamp representing the current time).
+4. **TopN Queries**. Support a query that returns the top N rows, through support for ORDER BY when used in conjunction with TopN.
+6. **Dynamic Columns**. For some use cases, it's difficult to model a schema up front. You may have columns that you'd like to specify only at query time. This is possible in HBase, in that every row (and column family) contains a map of values with keys that can be specified at run time. So, we'd like to support that.
+7. **Phoenix package for the Apache Bigtop distribution**. See [BIGTOP-993](http://issues.apache.org/jira/browse/BIGTOP-993) for more information.

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/site/markdown/roadmap.md
----------------------------------------------------------------------
diff --git a/phoenix-core/src/site/markdown/roadmap.md b/phoenix-core/src/site/markdown/roadmap.md
new file mode 100644
index 0000000..201c3dc
--- /dev/null
+++ b/phoenix-core/src/site/markdown/roadmap.md
@@ -0,0 +1,35 @@
+# Roadmap
+
+#### Note: Issues on Github would be converted to JIRAs soon.
+
+Our roadmap is driven by our user community. Below, in prioritized order, is the current plan for Phoenix:
+
+1. **Hash Joins**. Provide the ability to join together multiple tables, through a phased approach:
+   <br/>**[Equi-join](https://github.com/forcedotcom/phoenix/issues/20)**. Support left, right, inner, outer equi-joins where one side of the join is small enough to fit into memory. **Available in master branch**
+   <br/>**[Semi/anti-join](https://github.com/forcedotcom/phoenix/issues/36)**. Support correlated sub queries for exists and in where one side of the join is small enough to fit into memory.
+2. **[Multi-tenant Tables](https://github.com/forcedotcom/phoenix/issues/296)**. Allows the creation of multiple tables from a base tables on the same physical HBase table. **Available in master branch**
+3. **[Sequences](http://phoenix.incubator.apache.org/sequences.html)**. Support the atomic increment of sequence values through the CREATE SEQUENCE and the NEXT VALUE FOR statements.
+4. **Type Enhancements**. Additional work includes support for [DEFAULT declaration](https://github.com/forcedotcom/phoenix/issues/345) when creating a table, for [ARRAY](https://github.com/forcedotcom/phoenix/issues/178) (**available in master branch**), [STRUCT](https://github.com/forcedotcom/phoenix/issues/346), and [JSON](https://github.com/forcedotcom/phoenix/issues/497) data types.
+5. **Third Party Integration**. There are a number of open source projects with which interop with Phoenix could be added or improved:
+  <br/> **[Flume sink](https://github.com/forcedotcom/phoenix/issues/225)**. Support a Flume sink that writes Phoenix-compliant HBase data. **Available in master branch**
+  <br/> **[Hue integration](https://github.com/forcedotcom/phoenix/issues/224)**. Add Phoenix as an HBase service layer in Hue.
+  <br/> **[Pentaho Mondrian support](https://github.com/forcedotcom/phoenix/issues/512)**. Allow Phoenix to be used as the JDBC driver for Pentaho Mondrian. **This effort is pretty far along already, with the Pentaho FoodMart demo running through Phoenix now**
+  <br/> **[Cleanup Pig support](https://github.com/forcedotcom/phoenix/issues/499)** . Commonize the functions we use across Map-reduce and Pig processing. We should also upgrade our pom to reference the 0.12 version of Pig and map our DECIMAL type to their new decimal type.
+  <br/> **[Improve Map-reduce integration](https://github.com/forcedotcom/phoenix/issues/556)**. It's possible that we could provide a processing model where the map and reduce functions can invoke Phoenix queries (though this needs some more thought).
+5. **[Derived Tables](https://github.com/forcedotcom/phoenix/issues/5)**. Allow a <code>SELECT</code> clause to be used in the FROM clause to define a derived table. This would include support for pipelining queries when necessary.
+3. **[Functional Indexes](https://github.com/forcedotcom/phoenix/issues/383)**. Enables an index to contain the evaluation of an expression as opposed to just a column value.
+2. **[Monitoring and Management](https://github.com/forcedotcom/phoenix/issues/46)**. Provide visibility into cpu, physical io, logical io, wait time, blocking time, and transmission time spent for each thread of execution across the HBase cluster, within coprocessors, and within the client-side thread pools for each query. On top of this, we should exposing things like active sessions and currently running queries. The [EXPLAIN PLAN](http://forcedotcom.github.io/phoenix/#explain) gives an idea of how a query will be executed, but we need more information to help users debug and tune their queries.
+9. **[Parent/child Join](https://github.com/forcedotcom/phoenix/issues/19)**. Unlike with standard relational databases, HBase allows you the flexibility of dynamically creating as many key values in a row as you'd like. Phoenix could leverage this by providing a way to model child rows inside of a parent row. The child row would be comprised of the set of key values whose column qualifier is prefixed with a known name and appended with the primary key of the child row. Phoenix could hide all this complexity, and allow querying over the nested children through joining to the parent row. Essentially, this would be an optimization of the general join case, but could support cases where both sides of the join are bigger than would fit into memory.
+  <br/> **[Intra-row secondary indexes](https://github.com/forcedotcom/phoenix/issues/585)**. Once we support putting multiple "virtual" rows inside of a HBase row as prefixed KeyValues, we can support intra-row secondary indexes without much more effort.
+6. **[Port to HBase 0.96](https://github.com/forcedotcom/phoenix/issues/349)**. Currently Phoenix only works on the 0.94 branch of HBase. The latest branch of HBase is now 0.96, which has many breaking, non backward compatible changes (for example requiring that EndPoint coprocessors use protobufs). Ideally, we should create a shim that'll allow Phoenix to work with both 0.94 and 0.96, but barring that, we should have a branch of Phoenix that works under 0.96. Additional work includes replacing our type system with the new HBase type system in 0.96, but that would be significantly more work.
+5. **Security Features**. A number of existing HBase security features in 0.94 could be leverage and new security features being added to 0.98 could be leveraged in the future.
+  <br/> **[Support GRANT and REVOKE](https://github.com/forcedotcom/phoenix/issues/541)**. Support the standard GRANT and REVOKE SQL commands through an HBase AccessController.
+  <br/> **[Surface support for encryption] (https://github.com/forcedotcom/phoenix/issues/542)**. In HBase 0.98, transparent encryption will be possible. We should surface this in Phoenix.
+  <br/> **[Support Cell-level security](https://github.com/forcedotcom/phoenix/issues/553)**. In HBase 0.98, it will be possible to apply labels at a per-cell granularity. We should surface this in Phoenix.
+7. **Cost-based Optimizer**. Once secondary indexing and joins are implemented, we'll need to [collect and maintains stats](https://github.com/forcedotcom/phoenix/issues/64) and [drive query optimization decisions based on them](https://github.com/forcedotcom/phoenix/issues/49) to produce the most efficient query plan.
+8. **[Query over Multiple Row Versions](https://github.com/forcedotcom/phoenix/issues/459)**. Expose the time dimension of rows through a built-in function to allow aggregation and trending over multiple row versions.
+8. **[OLAP Extensions](https://github.com/forcedotcom/phoenix/issues/23)**. Support the `WINDOW`, `PARTITION OVER`, `RANK`, etc. functionality.
+10. **[Table Sampling](https://github.com/forcedotcom/phoenix/issues/22)**. Support the <code>TABLESAMPLE</code> clause by implementing a filter that uses the guideposts established by stats gathering to only return n rows per region.
+13. **Nested-loop Join**. Support joins where both sides are big enough that they wouldn't fit into memory. As projects like [Apache Drill](http://incubator.apache.org/drill/) progress, the need for this may lessen, since these systems will be able to decompose the query and perform the join efficiently without Phoenix needing to as described [here](http://www.hbasecon.com/sessions/apache-drill-a-community-driven-initiative-to-deliver-ansi-sql-capabilities-for-apache-hbase-1210pm-1230pm/).
+11. **Schema Evolution**. Phoenix supports adding and removing columns through the [ALTER TABLE] (language/index.html#alter_table) DDL command, but changing the data type of, or renaming, an existing column is not yet supported.
+12. **[Transactions](https://github.com/forcedotcom/phoenix/issues/269?source=cc)**. Support transactions by integrating a system that controls time stamps like [OMID](https://github.com/yahoo/omid). For some ideas on how this might be done, see [here](https://github.com/forcedotcom/phoenix/issues/269).

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/site/markdown/salted.md
----------------------------------------------------------------------
diff --git a/phoenix-core/src/site/markdown/salted.md b/phoenix-core/src/site/markdown/salted.md
new file mode 100644
index 0000000..6eddec8
--- /dev/null
+++ b/phoenix-core/src/site/markdown/salted.md
@@ -0,0 +1,25 @@
+# Salted Tables
+
+HBase sequential write may suffer from region server hotspotting if your row key is monotonically increasing. Salting the row key provides a way to mitigate the problem. Details of the method would be found on [this link](http://blog.sematext.com/2012/04/09/hbasewd-avoid-regionserver-hotspotting-despite-writing-records-with-sequential-keys/).
+
+Phoenix provides a way to transparently salt the row key with a salting byte for a particular table. You need to specify this in table creation time by specifying a table property "SALT_BUCKETS" with a value from 1 to 256. Like this:
+
+CREATE TABLE table (a_key VARCHAR PRIMARY KEY, a_col VARCHAR) SALT_BUCKETS = 20;
+
+There are some cautions and difference in behavior you should be aware about when using a salted table.
+
+### Sequential Scan
+Since salting table would not store the data sequentially, a strict sequential scan would not return all the data in the natural sorted fashion. Clauses that currently would force a sequential scan, for example, clauses with LIMIT, would likely to return items that are different from a normal table.
+
+### Splitting
+If no split points are specified for the table, the salted table would be pre-split on salt bytes boundaries to ensure load distribution among region servers even during the initial phase of the table. If users are to provide split points manually, users need to include a salt byte in the split points they provide.
+
+### Row Key Ordering
+Pre-spliting also ensures that all entries in the region server all starts with the same salt byte, and therefore are stored in a sorted manner. When doing a parallel scan across all region servers, we can take advantage of this properties to perform a merge sort of the client side. The resulting scan would still be return sequentially as if it is from a normal table.
+
+This Rowkey order scan can be turned on by setting the <code>phoenix.query.rowKeyOrderSaltedTable</code> config property to <code>true</code> in your hbase-sites.xml. When set, we disallow user specified split points on salted table to ensure that each bucket will only contains entries with the same salt byte. When this property is turned on, the salted table would behave just like a normal table and would return items in rowkey order for scans.
+
+### Performance
+Using salted table with pre-split would help uniformly distribute write workload across all the region servers, thus improves the write performance. Our own [performance evaluation](performance.html#Salting) shows that a salted table achieves 80% write throughput increases over non-salted table.
+
+Reading from salted table can also reap benefits from the more uniform distribution of data. Our [performance evaluation](performance.html#Salting) shows much improved read performances on read queries with salted table over non-salted table when we focus our query on a subset of the data.

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/site/markdown/secondary_indexing.md
----------------------------------------------------------------------
diff --git a/phoenix-core/src/site/markdown/secondary_indexing.md b/phoenix-core/src/site/markdown/secondary_indexing.md
new file mode 100644
index 0000000..e36608c
--- /dev/null
+++ b/phoenix-core/src/site/markdown/secondary_indexing.md
@@ -0,0 +1,152 @@
+# Secondary Indexing
+
+Secondary indexes are an orthogonal way to access data from its primary access path. In HBase, you have a single index that is lexicographically sorted on 
+the primary row key. Access to records in any way other than through the primary row requires scanning over potentially all the rows in the table to test them against your filter. With secondary indexing, the columns you index form an alternate row key to allow point lookups and range scans along this new axis. Phoenix is particularly powerful in that we provide _covered_ indexes - we do not need to go back to the primary table once we have found the index entry. Instead, we bundle the data we care about right in the index rows, saving read-time overhead.
+
+Phoenix supports two main forms of indexing: mutable and immutable indexing. They are useful in different scenarios and have their own failure profiles and performance characteristics. Both indexes are 'global' indexes - they live on their own tables and are copies of primary table data, which Phoenix ensures remain in-sync.
+
+# Mutable Indexing
+
+Often, the rows you are inserting are changing - pretty much any time you are not doing time-series data. In this case, use mutable indexing to ensure that your index is properly maintained as your data changes.
+
+All the performance penalties for indexes occur at write time. We intercept the primary table updates on write ([DELETE](language/index.html#delete), [UPSERT VALUES](language/index.html#upsert_values) and [UPSERT SELECT](language/index.html#upsert_select)), build the index update and then sent any necessary updates to all interested index tables. At read time, Phoenix will select the index table to use that will produce the fastest query time and directly scan it just like any other HBase table.
+
+## Example
+
+Given the schema shown here:
+
+    CREATE TABLE my_table (k VARCHAR PRIMARY KEY, v1 VARCHAR, v2 BIGINT);
+you'd create an index on the v1 column like this:
+
+    CREATE INDEX my_index ON my_table (v1);
+A table may contain any number of indexes, but note that your write speed will drop as you add additional indexes.
+
+We can also include columns from the data table in the index apart from the indexed columns. This allows an index to be used more frequently, as it will only be used if all columns referenced in the query are contained by it.
+
+    CREATE INDEX my_index ON my_table (v1) INCLUDE (v2);
+In addition, multiple columns may be indexed and their values may be stored in ascending or descending order.
+
+    CREATE INDEX my_index ON my_table (v2 DESC, v1) INCLUDE (v3);
+Finally, just like with the <code>CREATE TABLE</code> statement, the <code>CREATE INDEX</code> statement may pass through properties to apply to the underlying HBase table, including the ability to salt it:
+
+    CREATE INDEX my_index ON my_table (v2 DESC, v1) INCLUDE (v3)
+        SALT_BUCKETS=10, DATA_BLOCK_ENCODING='NONE';
+Note that if the primary table is salted, then the index is automatically salted in the same way. In addition, the MAX_FILESIZE for the index is adjusted down, relative to the size of the primary versus index table. For more on salting see [here](salted.html).
+
+# Immutable Indexing
+
+Immutable indexing targets use cases that are _write once_, _append only_; this is common in time-series data, where you log once, but read multiple times. In this case, the indexing is managed entirely on the client - either we successfully write all the primary and index data or we return a failure to the client. Since once written, rows are never updated, no incremental index maintenance is required. This reduces the overhead of secondary indexing at write time. However, keep in mind that immutable indexing are only applicable in a limited set of use cases.
+
+## Example
+
+To use immutable indexing, supply an <code>IMMUTABLE_ROWS=true</code> property when you create your table like this:
+
+    CREATE TABLE my_table (k VARCHAR PRIMARY KEY, v VARCHAR) IMMUTABLE_ROWS=true;
+
+Other than that, all of the previous examples are identical for immutable indexing.
+
+If you have an existing table that you'd like to switch from immutable indexing to mutable indexing, use the <code>ALTER TABLE</code> command as show below:
+
+    ALTER TABLE my_table SET IMMUTABLE_ROWS=false;
+For the complete syntax, see our [Language Reference Guide](language/index.html#create_index).
+
+## Data Guarantees and Failure Management
+
+On successful return to the client, all data is guaranteed to be written to all interested indexes and the primary table. For each individual data row, updates are an all-or-nothing, with a small gap of being behind. From the perspective of a single client, it either thinks all-or-none of the update worked.
+
+We maintain index update durability by adding the index updates to the Write-Ahead-Log (WAL) entry of the primary table row. Only after the WAL entry is successfully synced to disk do we attempt to make the index/primary table updates. We write the index updates in parallel by default, leading to very high throughput. If the server crashes while we are writing the index updates, we replay the all the index updates to the index tables in the WAL recovery process and rely on the idempotence of the updates to ensure correctness. Therefore, index tables are only every a single edit ahead of the primary table.
+
+Its important to note several points:
+ * We _do not provide full transactions_ so you could see the index table out of sync with the primary table.
+  * As noted above, this is ok as we are only a very small bit ahead and out of sync for very short periods
+ * Each data row and its index row(s) are guaranteed to to be written or lost - we never see partial updates
+ * All data is first written to index tables before the primary table
+
+### Singular Write Path
+
+There is a single write path that guarantees the failure properties. All writes to the HRegion get intercepted by our coprocessor. We then build the index updates based on the pending update (or updates, in the case of the batch). These update are then appended to the WAL entry for the original update.
+
+If we get any failure up to this point, we return the failure to the client and no data is persisted or made visible to the client. 
+
+Once the WAL is written, we ensure that the index and primary table data will become visible, even in the case of a failure.
+ * If the server does _not_ crash, we just insert the index updates to their respective tables.
+ * If the server _does_ crash, we then replay the index updates with the usual WAL replay mechanism
+    ** If any of the index updates fails, we then fail the server, ensuring we get the WAL replay of the updates later.
+
+### Failure Policy
+
+In the event that the region server handling the data updates cannot write to the region server handling the index updates, the index is automatically disabled and will no longer be considered for use in queries (as it will no longer be in sync with the data table). To use it again, it must be manually rebuilt with the following command:
+
+```
+ALTER INDEX my_index ON my_table REBUILD;
+```
+
+If we cannot disable the index, then the server will be immediately aborted. If the abort fails, we call System.exit on the JVM, forcing the server to die. By killing the server, we ensure that the WAL will be replayed on recovery, replaying the index updates to their appropriate tables.
+
+**WARNING: indexing has the potential to bring down your entire cluster very quickly.**
+
+If the index tables are not setup correctly (Phoenix ensures that they are), this failure policy can cause a cascading failure as each region server attempts and fails to write the index update, subsequently killing itself to ensure the visibility concerns outlined above.
+
+## Setup
+
+Only mutable indexing requires special configuration options in the region server to run - phoenix ensures that they are setup correctly when you enable mutable indexing on the table; if the correct properties are not set, you will not be able to turn it on.
+
+You will need to add the following parameters to `hbase-site.xml`:
+```
+<property>
+  <name>hbase.regionserver.wal.codec</name>
+  <value>org.apache.hadoop.hbase.regionserver.wal.IndexedWALEditCodec</value>
+</property>
+```
+
+This enables custom WAL edits to be written, ensuring proper writing/replay of the index updates. This codec supports the usual host of WALEdit options, most notably WALEdit compression.
+
+## Tuning
+Out the box, indexing is pretty fast. However, to optimize for your particular environment and workload, there are several properties you can tune.
+
+All the following parameters must be set in `hbase-site.xml` - they are true for the entire cluster and all index tables, as well as across all regions on the same server (so, for instance, a single server would not write to too many different index tables at once).
+
+1. index.builder.threads.max
+ * Number of threads to used to build the index update from the primary table update
+ * Increasing this value overcomes the bottleneck of reading the current row state from the underlying HRegion. Tuning this value too high will just bottleneck at the HRegion as it will not be able to handle too many concurrent scan requests as well as general thread-swapping concerns.
+ * **Default: 10**
+
+2. index.builder.threads.keepalivetime
+ * Amount of time in seconds after we expire threads in the builder thread pool.
+ * Unused threads are immediately released after this amount of time and not core threads are retained (though this last is a small concern as tables are expected to sustain a fairly constant write load), but simultaneously allows us to drop threads if we are not seeing the expected load.
+ * **Default: 60**
+
+3. index.writer.threads.max
+ * Number of threads to use when writing to the target index tables.
+ * The first level of parallelization, on a per-table basis - it should roughly correspond to the number of index tables
+ * **Default: 10**
+
+4. index.writer.threads.keepalivetime
+ * Amount of time in seconds after we expire threads in the writer thread pool.
+ * Unused threads are immediately released after this amount of time and not core threads are retained (though this last is a small concern as tables are expected to sustain a fairly constant write load), but simultaneously allows us to drop threads if we are not seeing the expected load.
+ * **Default: 60**
+
+5. hbase.htable.threads.max
+ * Number of threads each index HTable can use for writes.
+ * Increasing this allows more concurrent index updates (for instance across batches), leading to high overall throughput.
+ * **Default: 2,147,483,647**
+
+6. hbase.htable.threads.keepalivetime
+ * Amount of time in seconds after we expire threads in the HTable's thread pool.
+ * Using the "direct handoff" approach, new threads will only be created if it is necessary and will grow unbounded. This could be bad but HTables  only create as many Runnables as there are region servers; therefore, it also scales when new region servers are added.
+ * **Default: 60** 
+ 
+7. index.tablefactory.cache.size
+ * Number of index HTables we should keep in cache.
+ * Increasing this number ensures that we do not need to recreate an HTable for each attempt to write to an index table. Conversely, you could see memory pressure if this value is set too high.
+ * **Default: 10**
+
+# Performance
+We track secondary index performance via our [performance framework](http://phoenix-bin.github.io/client/performance/latest.htm). This is a generic test of performance based on defaults - your results will vary based on hardware specs as well as you individual configuration.
+
+That said, we have seen secondary indexing (both immutable and mutable) go as quickly as < 2x the regular write path on a small, (3 node) desktop-based cluster. This is actually a phenomenal as we have to write to multiple tables as well as build the index update.
+
+# Presentations
+There have been several presentations given on how secondary indexing works in Phoenix that have a more indepth look at how indexing works (with pretty pictures!):
+ * [San Francisco HBase Meetup](http://files.meetup.com/1350427/PhoenixIndexing-SF-HUG_09-26-13.pptx) - Sept. 26, 2013
+ * [Los Anglees HBase Meetup](http://www.slideshare.net/jesse_yates/phoenix-secondary-indexing-la-hug-sept-9th-2013) - Sept, 4th, 2013

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/site/markdown/sequences.md
----------------------------------------------------------------------
diff --git a/phoenix-core/src/site/markdown/sequences.md b/phoenix-core/src/site/markdown/sequences.md
new file mode 100644
index 0000000..a613077
--- /dev/null
+++ b/phoenix-core/src/site/markdown/sequences.md
@@ -0,0 +1,47 @@
+# Sequences
+
+Sequences are a standard SQL feature that allow for generating monotonically increasing numbers typically used to form an ID. To create a sequence, use the following command:
+
+    CREATE SEQUENCE my_schema.my_sequence;
+
+This will start the sequence from 1, increment by 1 each time, and cache on your session the default number of sequence values based on the phoenix.sequence.cacheSize config parameter. The specification of a sequence schema is optional. Caching sequence values on your session improves performance, as we don't need to ask the server for more sequence values until we run out of cached values. The tradeoff is that you may end up with gaps in your sequence values when other sessions also use the same sequence. 
+
+All of these parameters can be overridden when the sequence is created like this:
+
+    CREATE SEQUENCE my_schema.my_sequence START WITH 100 INCREMENT BY 2 CACHE 50;
+
+Sequences are incremented using the NEXT VALUE FOR <sequence_name> expression in an UPSERT VALUES, UPSERT SELECT, or SELECT statement as shown below:
+
+    UPSERT VALUES INTO my_table(id, col1, col2) 
+    VALUES( NEXT VALUE FOR my_schema.my_sequence, 'foo', 'bar');
+
+This will allocate a BIGINT based on the next value from the sequence (beginning with the START WITH value and incrementing from there based on the INCREMENT BY amount).
+
+When used in an UPSERT SELECT or SELECT statement, each row returned by the statement would have a unique value associated with it. For example:
+
+    UPSERT INTO my_table(id, col1, col2) 
+    SELECT NEXT VALUE FOR my_schema.my_sequence, 'foo', 'bar' FROM my_other_table;
+
+would allocate a new sequence value for each row returned from the SELECT expression. A sequence is only increment once for a given statement, so multiple references to the same sequence by NEXT VALUE FOR produce the same value. For example, in the following statement, my_table.col1 and my_table.col2 would end up with the same value:
+
+    UPSERT VALUES INTO my_table(col1, col2) 
+    VALUES( NEXT VALUE FOR my_schema.my_sequence, NEXT VALUE FOR my_schema.my_sequence);
+
+You may also access the last sequence value allocated using a CURRENT VALUE FOR expression as shown below:
+
+    SELECT CURRENT VALUE FOR my_schema.my_sequence, col1, col2 FROM my_table;
+
+This would evaluate to the last sequence value allocated from the previous NEXT VALUE FOR expression for your session (i.e. all connections on the same JVM for the same client machine). If no NEXT VALUE FOR expression had been previously called, this would produce an exception. If in a given statement a CURRENT VALUE FOR expression is used together with a NEXT VALUE FOR expression for the same sequence, then both would evaluate to the value produced by the NEXT VALUE FOR expression.
+
+The NEXT VALUE FOR and CURRENT VALUE FOR expressions may only be used as SELECT expressions or in the UPSERT VALUES statement. Use in WHERE, GROUP BY, HAVING, or ORDER BY will produce an exception. In addition, they cannot be used as the argument to an aggregate function.
+
+To drop a sequence, issue the following command:
+
+    DROP SEQUENCE my_schema.my_sequence;
+
+To discover all sequences that have been created, you may query the SYSTEM.SEQUENCE table as shown here:
+
+    SELECT sequence_schema, sequence_name, start_with, increment_by, cache_size FROM SYSTEM."SEQUENCE";
+
+Note that only read-only access to the SYSTEM.SEQUENCE table is supported.
+

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/site/markdown/skip_scan.md
----------------------------------------------------------------------
diff --git a/phoenix-core/src/site/markdown/skip_scan.md b/phoenix-core/src/site/markdown/skip_scan.md
new file mode 100644
index 0000000..3bc07b8
--- /dev/null
+++ b/phoenix-core/src/site/markdown/skip_scan.md
@@ -0,0 +1,22 @@
+# Skip Scan
+
+Phoenix uses Skip Scan for intra-row scanning which allows for [significant performance improvement](performance.html#Skip-Scan) over Range Scan when rows are retrieved based on a given set of keys.
+
+The Skip Scan leverages <code>SEEK_NEXT_USING_HINT</code> of HBase Filter. It stores information about what set of keys/ranges of keys are being searched for in each column. It then takes a key (passed to it during filter evaluation), and figures out if it's in one of the combinations or range or not. If not, it figures out to which next highest key to jump.
+
+Input to the <code>SkipScanFilter</code> is a <code>List&lt;List&lt;KeyRange&gt;&gt;</code> where the top level list represents each column in the row key (i.e. each primary key part), and the inner list represents ORed together byte array boundaries.
+
+
+Consider the following query:
+
+    SELECT * from T
+    WHERE ((KEY1 >='a' AND KEY1 <= 'b') OR (KEY1 > 'c' AND KEY1 <= 'e')) 
+    AND KEY2 IN (1, 2)
+
+The <code>List&lt;List&lt;KeyRange&gt;&gt;</code> for <code>SkipScanFilter</code> for the above query would be [ [ [ a - b ], [ d - e ] ], [ 1, 2 ] ] where [ [ a - b ], [ d - e ] ] is the range for <code>KEY1</code> and [ 1, 2 ] keys for <code>KEY2</code>.
+
+The following diagram illustrates graphically how the skip scan is able to jump around the key space:
+
+![Skip Scan Example](http://4.bp.blogspot.com/-SAFH11n_bPY/UYvaWa0P4bI/AAAAAAAAAu4/rKOuKIaMwF4/s1600/SkipScan.png)
+
+


[08/51] [partial] Initial commit of master branch from github

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableType.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableType.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableType.java
new file mode 100644
index 0000000..f079a92
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableType.java
@@ -0,0 +1,111 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.util.Map;
+
+import com.google.common.collect.Maps;
+
+
+public enum PTableType {
+    SYSTEM("s", "SYSTEM TABLE"), 
+    TABLE("u", "TABLE"),
+    VIEW("v", "VIEW"),
+    INDEX("i", "INDEX"),
+    JOIN("j", "JOIN"); 
+
+    private final PName value;
+    private final String serializedValue;
+    
+    private PTableType(String serializedValue, String value) {
+        this.serializedValue = serializedValue;
+        this.value = PNameFactory.newName(value);
+    }
+    
+    public String getSerializedValue() {
+        return serializedValue;
+    }
+    
+    public PName getValue() {
+        return value;
+    }
+    
+    @Override
+    public String toString() {
+        return value.getString();
+    }
+    
+    private static final PTableType[] FROM_SERIALIZED_VALUE;
+    private static final int FROM_SERIALIZED_VALUE_OFFSET;
+    private static final Map<String,PTableType> FROM_VALUE = Maps.newHashMapWithExpectedSize(PTableType.values().length);
+    
+    static {
+        int minChar = Integer.MAX_VALUE;
+        int maxChar = Integer.MIN_VALUE;
+        for (PTableType type : PTableType.values()) {
+            char c = type.getSerializedValue().charAt(0);
+            if (c < minChar) {
+                minChar = c;
+            }
+            if (c > maxChar) {
+                maxChar = c;
+            }
+        }
+        FROM_SERIALIZED_VALUE_OFFSET = minChar;
+        FROM_SERIALIZED_VALUE = new PTableType[maxChar - minChar + 1];
+        for (PTableType type : PTableType.values()) {
+            FROM_SERIALIZED_VALUE[type.getSerializedValue().charAt(0) - minChar] = type;
+        }
+    }
+    
+    static {
+        for (PTableType type : PTableType.values()) {
+            if (FROM_VALUE.put(type.getValue().getString(),type) != null) {
+                throw new IllegalStateException("Duplicate PTableType value of " + type.getValue().getString() + " is not allowed");
+            }
+        }
+    }
+    
+    public static PTableType fromValue(String value) {
+        PTableType type = FROM_VALUE.get(value);
+        if (type == null) {
+            throw new IllegalArgumentException("Unable to PTableType enum for value of '" + value + "'");
+        }
+        return type;
+    }
+    
+    public static PTableType fromSerializedValue(String serializedValue) {
+        if (serializedValue.length() == 1) {
+            int i = serializedValue.charAt(0) - FROM_SERIALIZED_VALUE_OFFSET;
+            if (i >= 0 && i < FROM_SERIALIZED_VALUE.length && FROM_SERIALIZED_VALUE[i] != null) {
+                return FROM_SERIALIZED_VALUE[i];
+            }
+        }
+        throw new IllegalArgumentException("Unable to PTableType enum for serialized value of '" + serializedValue + "'");
+    }
+    
+    public static PTableType fromSerializedValue(byte serializedByte) {
+        int i = serializedByte - FROM_SERIALIZED_VALUE_OFFSET;
+        if (i >= 0 && i < FROM_SERIALIZED_VALUE.length && FROM_SERIALIZED_VALUE[i] != null) {
+            return FROM_SERIALIZED_VALUE[i];
+        }
+        throw new IllegalArgumentException("Unable to PTableType enum for serialized value of '" + (char)serializedByte + "'");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/PhoenixArray.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PhoenixArray.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PhoenixArray.java
new file mode 100644
index 0000000..e58cb50
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PhoenixArray.java
@@ -0,0 +1,470 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.sql.Array;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.Map;
+
+import org.apache.phoenix.util.SQLCloseable;
+
+/**
+ * java.sql.Array implementation for Phoenix
+ */
+public class PhoenixArray implements Array,SQLCloseable {
+	PDataType baseType;
+	Object array;
+	int numElements;
+	public PhoenixArray() {
+		// empty constructor
+	}
+	
+	public PhoenixArray(PDataType baseType, Object[] elements) {
+		// As we are dealing with primitive types and only the Boxed objects
+		this.baseType = baseType;
+		this.array = convertObjectArrayToPrimitiveArray(elements);
+		this.numElements = elements.length;
+	}
+	
+	public Object convertObjectArrayToPrimitiveArray(Object[] elements) {
+		return elements;
+	}
+
+	@Override
+	public void free() throws SQLException {
+	}
+
+	@Override
+	public Object getArray() throws SQLException {
+		return array;
+	}
+	
+	@Override
+	public void close() throws SQLException {
+		this.array = null;
+	}
+
+	@Override
+	public Object getArray(Map<String, Class<?>> map) throws SQLException {
+		throw new UnsupportedOperationException("Currently not supported");
+	}
+
+	@Override
+	public Object getArray(long index, int count) throws SQLException {
+		if(index < 1) {
+			throw new IllegalArgumentException("Index cannot be less than 1");
+		}
+		// Get the set of elements from the given index to the specified count
+		Object[] intArr = (Object[]) array;
+		boundaryCheck(index, count, intArr);
+		Object[] newArr = new Object[count];
+		// Add checks() here.
+		int i = 0;
+		for (int j = (int) index; j < count; j++) {
+			newArr[i] = intArr[j];
+			i++;
+		}
+		return newArr;
+	}
+
+	private void boundaryCheck(long index, int count, Object[] arr) {
+		if ((--index) + count > arr.length) {
+			throw new IllegalArgumentException("The array index is out of range of the total number of elements in the array " + arr.length);
+		}
+	}
+
+	@Override
+	public Object getArray(long index, int count, Map<String, Class<?>> map)
+			throws SQLException {
+		if(map != null && !map.isEmpty()) {
+			throw new UnsupportedOperationException("Currently not supported");
+		}
+		return null;
+	}
+
+	@Override
+	public int getBaseType() throws SQLException {
+		return baseType.getSqlType();
+	}
+
+	@Override
+	public String getBaseTypeName() throws SQLException {
+		return baseType.getSqlTypeName();
+	}
+
+	@Override
+	public ResultSet getResultSet() throws SQLException {
+		throw new UnsupportedOperationException("Currently not supported");
+	}
+
+	@Override
+	public ResultSet getResultSet(Map<String, Class<?>> arg0)
+			throws SQLException {
+		throw new UnsupportedOperationException("Currently not supported");
+	}
+
+	@Override
+	public ResultSet getResultSet(long arg0, int arg1) throws SQLException {
+		throw new UnsupportedOperationException("Currently not supported");
+	}
+
+	@Override
+	public ResultSet getResultSet(long arg0, int arg1,
+			Map<String, Class<?>> arg2) throws SQLException {
+		throw new UnsupportedOperationException("Currently not supported");
+	}
+
+	public int getDimensions() {
+		return this.numElements;
+	}
+	
+	public int estimateByteSize(int pos) {
+		return this.baseType.estimateByteSize(((Object[])array)[pos]);
+	}
+	
+	public byte[] toBytes(int pos) {
+		return this.baseType.toBytes(((Object[])array)[pos]);
+	}
+	
+	@Override
+	public boolean equals(Object obj) {
+		if (this.numElements != ((PhoenixArray) obj).numElements) {
+			return false;
+		}
+		if (this.baseType != ((PhoenixArray) obj).baseType) {
+			return false;
+		}
+		return Arrays.deepEquals((Object[]) this.array,
+				(Object[]) ((PhoenixArray) obj).array);
+	}
+
+	@Override
+	public int hashCode() {
+		// TODO : Revisit
+		final int prime = 31;
+		int result = 1;
+		result = prime * result
+				+ ((array == null) ? 0 : array.hashCode());
+		return result;
+	}
+	
+	public static class PrimitiveIntPhoenixArray extends PhoenixArray {
+		private int[] intArr;
+		public PrimitiveIntPhoenixArray(PDataType dataType, Object[] elements) {
+			super(dataType, elements);
+		}
+		@Override
+		public Object convertObjectArrayToPrimitiveArray(Object[] elements) {
+			Object object = java.lang.reflect.Array.newInstance(int.class,
+					elements.length);
+			intArr = (int[]) object;
+			int i = 0;
+			for(Object o : elements) {
+			    if (o != null) {
+			        intArr[i] = (Integer)o;
+			    }
+			    i++;
+			}
+			return intArr;
+		}
+		
+		@Override
+        public int estimateByteSize(int pos) {
+			return this.baseType.estimateByteSize(intArr[pos]);
+		}
+		
+		@Override
+        public byte[] toBytes(int pos) {
+			return this.baseType.toBytes(intArr[pos]);
+		}
+		
+		@Override
+		public boolean equals(Object obj) {
+			if (this.numElements != ((PhoenixArray) obj).numElements) {
+				return false;
+			}
+			if (this.baseType != ((PhoenixArray) obj).baseType) {
+				return false;
+			}
+			return Arrays.equals((int[]) this.array,
+					(int[]) ((PhoenixArray) obj).array);
+		}
+	}
+	
+	public static class PrimitiveShortPhoenixArray extends PhoenixArray {
+		private short[] shortArr;
+		public PrimitiveShortPhoenixArray(PDataType dataType, Object[] elements) {
+			super(dataType, elements);
+		}
+		@Override
+		public Object convertObjectArrayToPrimitiveArray(Object[] elements) {
+			Object object = java.lang.reflect.Array.newInstance(short.class,
+					elements.length);
+			shortArr = (short[]) object;
+			int i = 0;
+            for(Object o : elements) {
+                if (o != null) {
+                    shortArr[i] = (Short)o;
+                }
+                i++;
+            }
+			return shortArr;
+		}
+		
+		@Override
+        public int estimateByteSize(int pos) {
+			return this.baseType.estimateByteSize(shortArr[pos]);
+		}
+		
+		@Override
+        public byte[] toBytes(int pos) {
+			return this.baseType.toBytes(shortArr[pos]);
+		}
+		
+		@Override
+		public boolean equals(Object obj) {
+			if (this.numElements != ((PhoenixArray) obj).numElements) {
+				return false;
+			}
+			if (this.baseType != ((PhoenixArray) obj).baseType) {
+				return false;
+			}
+			return Arrays.equals((short[]) this.array,
+					(short[]) ((PhoenixArray) obj).array);
+		}
+	}
+	
+	public static class PrimitiveLongPhoenixArray extends PhoenixArray {
+		private long[] longArr;
+		public PrimitiveLongPhoenixArray(PDataType dataType, Object[] elements) {
+			super(dataType, elements);
+		}
+		@Override
+		public Object convertObjectArrayToPrimitiveArray(Object[] elements) {
+			Object object = java.lang.reflect.Array.newInstance(long.class,
+					elements.length);
+			longArr = (long[]) object;
+			int i = 0;
+            for(Object o : elements) {
+                if (o != null) {
+                    longArr[i] = (Long)o;
+                }
+                i++;
+            }
+			return longArr;
+		}
+		@Override
+        public int estimateByteSize(int pos) {
+			return this.baseType.estimateByteSize(longArr[pos]);
+		}
+		
+		@Override
+        public byte[] toBytes(int pos) {
+			return this.baseType.toBytes(longArr[pos]);
+		}
+		
+		@Override
+		public boolean equals(Object obj) {
+			if (this.numElements != ((PhoenixArray) obj).numElements) {
+				return false;
+			}
+			if (this.baseType != ((PhoenixArray) obj).baseType) {
+				return false;
+			}
+			return Arrays.equals((long[]) this.array,
+					(long[]) ((PhoenixArray) obj).array);
+		}
+
+	}
+	
+	public static class PrimitiveDoublePhoenixArray extends PhoenixArray {
+		private double[] doubleArr;
+		public PrimitiveDoublePhoenixArray(PDataType dataType, Object[] elements) {
+			super(dataType, elements);
+		}
+		@Override
+		public Object convertObjectArrayToPrimitiveArray(Object[] elements) {
+			Object object = java.lang.reflect.Array.newInstance(double.class,
+					elements.length);
+			doubleArr = (double[]) object;
+			int i = 0;
+			for (Object o : elements) {
+			    if (o != null) {
+			        doubleArr[i] = (Double) o;
+			    }
+			    i++;
+			}
+			return doubleArr;
+		}
+		
+		@Override
+        public int estimateByteSize(int pos) {
+			return this.baseType.estimateByteSize(doubleArr[pos]);
+		}
+		
+		@Override
+        public byte[] toBytes(int pos) {
+			return this.baseType.toBytes(doubleArr[pos]);
+		}
+		
+		@Override
+		public boolean equals(Object obj) {
+			if (this.numElements != ((PhoenixArray) obj).numElements) {
+				return false;
+			}
+			if (this.baseType != ((PhoenixArray) obj).baseType) {
+				return false;
+			}
+			return Arrays.equals((double[]) this.array,
+					(double[]) ((PhoenixArray) obj).array);
+		}
+	}
+	
+	public static class PrimitiveFloatPhoenixArray extends PhoenixArray {
+		private float[] floatArr;
+		public PrimitiveFloatPhoenixArray(PDataType dataType, Object[] elements) {
+			super(dataType, elements);
+		}
+		@Override
+		public Object convertObjectArrayToPrimitiveArray(Object[] elements) {
+			Object object = java.lang.reflect.Array.newInstance(float.class,
+					elements.length);
+			floatArr = (float[]) object;
+			int i = 0;
+            for(Object o : elements) {
+                if (o != null) {
+                    floatArr[i] = (Float)o;
+                }
+                i++;
+            }
+			return floatArr;
+		}
+		
+		@Override
+        public int estimateByteSize(int pos) {
+			return this.baseType.estimateByteSize(floatArr[pos]);
+		}
+		
+		@Override
+        public byte[] toBytes(int pos) {
+			return this.baseType.toBytes(floatArr[pos]);
+		}
+		
+		@Override
+		public boolean equals(Object obj) {
+			if (this.numElements != ((PhoenixArray) obj).numElements) {
+				return false;
+			}
+			if (this.baseType != ((PhoenixArray) obj).baseType) {
+				return false;
+			}
+			return Arrays.equals((float[]) this.array,
+					(float[]) ((PhoenixArray) obj).array);
+		}
+	}
+	
+	public static class PrimitiveBytePhoenixArray extends PhoenixArray {
+		private byte[] byteArr;
+		public PrimitiveBytePhoenixArray(PDataType dataType, Object[] elements) {
+			super(dataType, elements);
+		}
+		@Override
+		public Object convertObjectArrayToPrimitiveArray(Object[] elements) {
+			Object object = java.lang.reflect.Array.newInstance(byte.class,
+					elements.length);
+			byteArr = (byte[]) object;
+			int i = 0;
+            for(Object o : elements) {
+                if (o != null) {
+                    byteArr[i] = (Byte)o;
+                }
+                i++;
+            }
+			return byteArr;
+		}
+		
+		@Override
+        public int estimateByteSize(int pos) {
+			return this.baseType.estimateByteSize(byteArr[pos]);
+		}
+		
+		@Override
+        public byte[] toBytes(int pos) {
+			return this.baseType.toBytes(byteArr[pos]);
+		}
+		
+		@Override
+		public boolean equals(Object obj) {
+			if (this.numElements != ((PhoenixArray) obj).numElements) {
+				return false;
+			}
+			if (this.baseType != ((PhoenixArray) obj).baseType) {
+				return false;
+			}
+			return Arrays.equals((byte[]) this.array,
+					(byte[]) ((PhoenixArray) obj).array);
+		}
+	}
+	
+	public static class PrimitiveBooleanPhoenixArray extends PhoenixArray {
+		private boolean[] booleanArr;
+		public PrimitiveBooleanPhoenixArray(PDataType dataType, Object[] elements) {
+			super(dataType, elements);
+		}
+		@Override
+		public Object convertObjectArrayToPrimitiveArray(Object[] elements) {
+			Object object = java.lang.reflect.Array.newInstance(boolean.class,
+					elements.length);
+			booleanArr = (boolean[]) object;
+			int i = 0;
+            for(Object o : elements) {
+                if (o != null) {
+                    booleanArr[i] = (Boolean)o;
+                }
+                i++;
+            }
+			return booleanArr;
+		}
+		
+		@Override
+        public int estimateByteSize(int pos) {
+			return this.baseType.estimateByteSize(booleanArr[pos]);
+		}
+		
+		@Override
+        public byte[] toBytes(int pos) {
+			return this.baseType.toBytes(booleanArr[pos]);
+		}
+		
+		@Override
+		public boolean equals(Object obj) {
+			if (this.numElements != ((PhoenixArray) obj).numElements) {
+				return false;
+			}
+			if (this.baseType != ((PhoenixArray) obj).baseType) {
+				return false;
+			}
+			return Arrays.equals((boolean[]) this.array,
+					(boolean[]) ((PhoenixArray) obj).array);
+		}
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/ReadOnlyTableException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/ReadOnlyTableException.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/ReadOnlyTableException.java
new file mode 100644
index 0000000..8672f8f
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/ReadOnlyTableException.java
@@ -0,0 +1,57 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.sql.SQLException;
+
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+
+/**
+ * 
+ * Exception thrown when an attempt is made to modify or write to a read-only table.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class ReadOnlyTableException extends SQLException {
+    private static final long serialVersionUID = 1L;
+    private static SQLExceptionCode code = SQLExceptionCode.READ_ONLY_TABLE;
+
+    public ReadOnlyTableException(String schemaName, String tableName) {
+        super(new SQLExceptionInfo.Builder(code).setSchemaName(schemaName).setTableName(tableName).toString(),
+                code.getSQLState(), code.getErrorCode());
+    }
+
+    public ReadOnlyTableException(String message, String schemaName, String tableName) {
+        super(new SQLExceptionInfo.Builder(code).setSchemaName(schemaName).setTableName(tableName).setMessage(message).toString(),
+                code.getSQLState(), code.getErrorCode());
+    }
+
+    public ReadOnlyTableException(String message, String schemaName, String tableName, Throwable cause) {
+        super(new SQLExceptionInfo.Builder(code).setSchemaName(schemaName).setTableName(tableName).setRootCause(cause).setMessage(message).toString(),
+                code.getSQLState(), code.getErrorCode(), cause);
+    }
+
+    public ReadOnlyTableException(String message, String schemaName, String tableName, String familyName) {
+        super(new SQLExceptionInfo.Builder(code).setSchemaName(schemaName).setTableName(tableName).setFamilyName(familyName).setMessage(message).toString(),
+                code.getSQLState(), code.getErrorCode());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/RowKeySchema.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/RowKeySchema.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/RowKeySchema.java
new file mode 100644
index 0000000..ec5fd1b
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/RowKeySchema.java
@@ -0,0 +1,243 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.apache.phoenix.query.QueryConstants.SEPARATOR_BYTE;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.query.QueryConstants;
+
+
+/**
+ * 
+ * Schema for the bytes in a RowKey. For the RowKey, we use a null byte
+ * to terminate a variable length type, while for KeyValue bytes we
+ * write the length of the var char preceding the value. We can't do
+ * that for a RowKey because it would affect the sort order.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class RowKeySchema extends ValueSchema {
+    public static final RowKeySchema EMPTY_SCHEMA = new RowKeySchema(0,Collections.<Field>emptyList())
+    ;
+    
+    public RowKeySchema() {
+    }
+    
+    protected RowKeySchema(int minNullable, List<Field> fields) {
+        super(minNullable, fields);
+    }
+
+    public static class RowKeySchemaBuilder extends ValueSchemaBuilder {
+        public RowKeySchemaBuilder(int maxFields) {
+            super(maxFields);
+            setMaxFields(maxFields);
+        }
+        
+        @Override
+        public RowKeySchemaBuilder addField(PDatum datum, boolean isNullable, ColumnModifier columnModifier) {
+            super.addField(datum, isNullable, columnModifier);
+            return this;
+        }
+
+        @Override
+        public RowKeySchema build() {
+            List<Field> condensedFields = buildFields();
+            return new RowKeySchema(this.minNullable, condensedFields);
+        }
+    }
+
+    public int getMaxFields() {
+        return this.getMinNullable();
+    }
+
+    @edu.umd.cs.findbugs.annotations.SuppressWarnings(
+            value="NP_BOOLEAN_RETURN_NULL", 
+            justification="Designed to return null.")
+    public Boolean iterator(byte[] src, int srcOffset, int srcLength, ImmutableBytesWritable ptr, int position) {
+        Boolean hasValue = null;
+        ptr.set(src, srcOffset, 0);
+        int maxOffset = srcOffset + srcLength;
+        for (int i = 0; i < position; i++) {
+            hasValue = next(ptr, i, maxOffset);
+        }
+        return hasValue;
+    }
+    
+    public Boolean iterator(ImmutableBytesWritable srcPtr, ImmutableBytesWritable ptr, int position) {
+        return iterator(srcPtr.get(), srcPtr.getOffset(), srcPtr.getLength(), ptr, position);
+    }
+    
+    public Boolean iterator(byte[] src, ImmutableBytesWritable ptr, int position) {
+        return iterator(src, 0, src.length, ptr, position);
+    }
+    
+    public int iterator(byte[] src, int srcOffset, int srcLength, ImmutableBytesWritable ptr) {
+        int maxOffset = srcOffset + srcLength;
+        iterator(src, srcOffset, srcLength, ptr, 0);
+        return maxOffset;
+    }
+    
+    public int iterator(byte[] src, ImmutableBytesWritable ptr) {
+        return iterator(src, 0, src.length, ptr);
+    }
+    
+    public int iterator(ImmutableBytesWritable ptr) {
+        return iterator(ptr.get(),ptr.getOffset(),ptr.getLength(), ptr);
+    }
+    
+    /**
+     * Move the bytes ptr to the next position relative to the current ptr
+     * @param ptr bytes pointer pointing to the value at the positional index
+     * provided.
+     * @param position zero-based index of the next field in the value schema
+     * @param maxOffset max possible offset value when iterating
+     * @return true if a value was found and ptr was set, false if the value is null and ptr was not
+     * set, and null if the value is null and there are no more values
+      */
+    @edu.umd.cs.findbugs.annotations.SuppressWarnings(
+            value="NP_BOOLEAN_RETURN_NULL", 
+            justification="Designed to return null.")
+    public Boolean next(ImmutableBytesWritable ptr, int position, int maxOffset) {
+        if (ptr.getOffset() + ptr.getLength() >= maxOffset) {
+            ptr.set(ptr.get(), maxOffset, 0);
+            return null;
+        }
+        if (position >= getFieldCount()) {
+            return null;
+        }
+        // Move the pointer past the current value and set length
+        // to 0 to ensure you never set the ptr past the end of the
+        // backing byte array.
+        ptr.set(ptr.get(), ptr.getOffset() + ptr.getLength(), 0);
+        // If positioned at SEPARATOR_BYTE, skip it.
+        if (position > 0 && !getField(position-1).getDataType().isFixedWidth()) {
+            ptr.set(ptr.get(), ptr.getOffset()+ptr.getLength()+1, 0);
+        }
+        Field field = this.getField(position);
+        if (field.getDataType().isFixedWidth()) {
+            ptr.set(ptr.get(),ptr.getOffset(), field.getByteSize());
+        } else {
+            if (position+1 == getFieldCount() ) { // Last field has no terminator
+                ptr.set(ptr.get(), ptr.getOffset(), maxOffset - ptr.getOffset());
+            } else {
+                byte[] buf = ptr.get();
+                int offset = ptr.getOffset();
+                while (offset < maxOffset && buf[offset] != SEPARATOR_BYTE) {
+                    offset++;
+                }
+                ptr.set(buf, ptr.getOffset(), offset - ptr.getOffset());
+            }
+        }
+        return ptr.getLength() > 0;
+    }
+    
+    @edu.umd.cs.findbugs.annotations.SuppressWarnings(
+            value="NP_BOOLEAN_RETURN_NULL", 
+            justification="Designed to return null.")
+    public Boolean previous(ImmutableBytesWritable ptr, int position, int minOffset) {
+        if (position < 0) {
+            return null;
+        }
+        Field field = this.getField(position);
+        if (field.getDataType().isFixedWidth()) {
+            ptr.set(ptr.get(), ptr.getOffset()-field.getByteSize(), field.getByteSize());
+            return true;
+        }
+        // If ptr has length of zero, it is assumed that we're at the end of the row key
+        int offsetAdjustment = position + 1 == this.getFieldCount() || ptr.getLength() == 0 ? 0 : 1;
+        if (position == 0) {
+            ptr.set(ptr.get(), minOffset, ptr.getOffset() - minOffset - offsetAdjustment);
+            return true;
+        }
+        field = this.getField(position-1);
+        // Field before the one we want to position at is variable length
+        // In this case, we can search backwards for our separator byte
+        // to determine the length
+        if (!field.getDataType().isFixedWidth()) {
+            byte[] buf = ptr.get();
+            int offset = ptr.getOffset()-1-offsetAdjustment;
+            while (offset > minOffset /* sanity check*/ && buf[offset] != QueryConstants.SEPARATOR_BYTE) {
+                offset--;
+            }
+            if (offset == minOffset) { // shouldn't happen
+                ptr.set(buf, minOffset, ptr.getOffset()-minOffset-1);
+            } else {
+                ptr.set(buf,offset+1,ptr.getOffset()-1-offsetAdjustment-offset); // Don't include null terminator in length
+            }
+            return true;
+        }
+        int i,fixedOffset = field.getByteSize();
+        for (i = position-2; i >= 0 && this.getField(i).getDataType().isFixedWidth(); i--) {
+            fixedOffset += this.getField(i).getByteSize();
+        }
+        // All of the previous fields are fixed width, so we can calculate the offset
+        // based on the total fixed offset
+        if (i < 0) {
+            int length = ptr.getOffset() - fixedOffset - minOffset - offsetAdjustment;
+            ptr.set(ptr.get(),minOffset+fixedOffset, length);
+            return true;
+        }
+        // Otherwise we're stuck with starting from the minOffset and working all the way forward,
+        // because we can't infer the length of the previous position.
+        return iterator(ptr.get(), minOffset, ptr.getOffset() - minOffset - offsetAdjustment, ptr, position+1);
+    }
+
+    @edu.umd.cs.findbugs.annotations.SuppressWarnings(
+            value="NP_BOOLEAN_RETURN_NULL", 
+            justification="Designed to return null.")
+    public Boolean reposition(ImmutableBytesWritable ptr, int oldPosition, int newPosition, int minOffset, int maxOffset) {
+        if (newPosition == oldPosition) {
+            return ptr.getLength() > 0;
+        }
+        Boolean hasValue = null;
+        if (newPosition > oldPosition) {
+            do {
+                hasValue = next(ptr, ++oldPosition, maxOffset);
+            }  while (hasValue != null && oldPosition < newPosition) ;
+        } else {
+            int nVarLengthFromBeginning = 0;
+            for (int i = 0; i <= newPosition; i++) {
+                if (!this.getField(i).getDataType().isFixedWidth()) {
+                    nVarLengthFromBeginning++;
+                }
+            }
+            int nVarLengthBetween = 0;
+            for (int i = oldPosition - 1; i >= newPosition; i--) {
+                if (!this.getField(i).getDataType().isFixedWidth()) {
+                    nVarLengthBetween++;
+                }
+            }
+            if (nVarLengthBetween > nVarLengthFromBeginning) {
+                return iterator(ptr.get(), minOffset, maxOffset-minOffset, ptr, newPosition+1);
+            }
+            do  {
+                hasValue = previous(ptr, --oldPosition, minOffset);
+            } while (hasValue != null && oldPosition > newPosition);
+        }
+        
+        return hasValue;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/RowKeyValueAccessor.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/RowKeyValueAccessor.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/RowKeyValueAccessor.java
new file mode 100644
index 0000000..01f51c7
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/RowKeyValueAccessor.java
@@ -0,0 +1,195 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.apache.phoenix.query.QueryConstants.SEPARATOR_BYTE;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableUtils;
+
+import org.apache.phoenix.util.ByteUtil;
+
+
+/**
+ * 
+ * Class that encapsulates accessing a value stored in the row key.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class RowKeyValueAccessor implements Writable   {
+    /**
+     * Constructor solely for use during deserialization. Should not
+     * otherwise be used.
+     */
+    public RowKeyValueAccessor() {
+    }
+    
+    /**
+     * Constructor to compile access to the value in the row key formed from
+     * a list of PData.
+     * 
+     * @param data the list of data that make up the key
+     * @param index the zero-based index of the data item to access.
+     */
+    public RowKeyValueAccessor(List<? extends PDatum> data, int index) {
+        this.index = index;
+        int[] offsets = new int[data.size()];
+        int nOffsets = 0;
+        Iterator<? extends PDatum> iterator = data.iterator();
+        PDatum datum = iterator.next();
+        int pos = 0;
+        while (pos < index) {
+            int offset = 0;
+            if (datum.getDataType().isFixedWidth()) {
+                do {
+                    offset += datum.getByteSize();
+                    datum = iterator.next();
+                    pos++;
+                } while (pos < index && datum.getDataType().isFixedWidth());
+                offsets[nOffsets++] = offset; // Encode fixed byte offset as positive
+            } else {
+                do {
+                    offset++; // Count the number of variable length columns
+                    datum = iterator.next();
+                    pos++;
+                } while (pos < index && !datum.getDataType().isFixedWidth());
+                offsets[nOffsets++] = -offset; // Encode number of variable length columns as negative
+            }
+        }
+        if (nOffsets < offsets.length) {
+            this.offsets = Arrays.copyOf(offsets, nOffsets);
+        } else {
+            this.offsets = offsets;
+        }
+        // Remember this so that we don't bother looking for the null separator byte in this case
+        this.isFixedLength = datum.getDataType().isFixedWidth();
+        this.hasSeparator = !isFixedLength && (datum != data.get(data.size()-1));
+    }
+    
+    RowKeyValueAccessor(int[] offsets, boolean isFixedLength, boolean hasSeparator) {
+        this.offsets = offsets;
+        this.isFixedLength = isFixedLength;
+        this.hasSeparator = hasSeparator;
+    }
+
+    private int index = -1; // Only available on client side
+    private int[] offsets;
+    private boolean isFixedLength;
+    private boolean hasSeparator;
+
+    public int getIndex() {
+        return index;
+    }
+    
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + (hasSeparator ? 1231 : 1237);
+        result = prime * result + (isFixedLength ? 1231 : 1237);
+        result = prime * result + Arrays.hashCode(offsets);
+        return result;
+    }
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) return true;
+        if (obj == null) return false;
+        if (getClass() != obj.getClass()) return false;
+        RowKeyValueAccessor other = (RowKeyValueAccessor)obj;
+        if (hasSeparator != other.hasSeparator) return false;
+        if (isFixedLength != other.isFixedLength) return false;
+        if (!Arrays.equals(offsets, other.offsets)) return false;
+        return true;
+    }
+
+    @Override
+    public String toString() {
+        return "RowKeyValueAccessor [offsets=" + Arrays.toString(offsets) + ", isFixedLength=" + isFixedLength
+                + ", hasSeparator=" + hasSeparator + "]";
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        // Decode hasSeparator and isFixedLength from vint storing offset array length
+        int length = WritableUtils.readVInt(input);
+        hasSeparator = (length & 0x02) != 0;
+        isFixedLength = (length & 0x01) != 0;
+        length >>= 2;
+        offsets = ByteUtil.deserializeVIntArray(input, length);
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+        // Encode hasSeparator and isFixedLength into vint storing offset array length
+        // (since there's plenty of room)
+        int length = offsets.length << 2;
+        length |= (hasSeparator ? 1 << 1 : 0) | (isFixedLength ? 1 : 0);
+        ByteUtil.serializeVIntArray(output, offsets, length);
+    }
+    
+    /**
+     * Calculate the byte offset in the row key to the start of the PK column value
+     * @param keyBuffer the byte array of the row key
+     * @param keyOffset the offset in the byte array of where the key begins
+     * @return byte offset to the start of the PK column value
+     */
+    public int getOffset(byte[] keyBuffer, int keyOffset) {
+        // Use encoded offsets to navigate through row key buffer
+        for (int offset : offsets) {
+            if (offset >= 0) { // If offset is non negative, it's a byte offset
+                keyOffset += offset;
+            } else { // Else, a negative offset is the number of variable length values to skip
+                while (offset++ < 0) {
+                    // FIXME: keyOffset < keyBuffer.length required because HBase passes bogus keys to filter to position scan (HBASE-6562)
+                    while (keyOffset < keyBuffer.length && keyBuffer[keyOffset++] != SEPARATOR_BYTE) {
+                    }
+                }
+            }
+        }
+        return keyOffset;
+    }
+    
+    /**
+     * Calculate the length of the PK column value
+     * @param keyBuffer the byte array of the row key
+     * @param keyOffset the offset in the byte array of where the key begins
+     * @param maxOffset maximum offset to use while calculating length 
+     * @return the length of the PK column value
+     */
+    public int getLength(byte[] keyBuffer, int keyOffset, int maxOffset) {
+        if (!hasSeparator) {
+            return maxOffset - keyOffset;
+        }
+        int offset = keyOffset;
+        // FIXME: offset < maxOffset required because HBase passes bogus keys to filter to position scan (HBASE-6562)
+        while (offset < maxOffset && keyBuffer[offset] != SEPARATOR_BYTE) {
+            offset++;
+        }
+        return offset - keyOffset;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..cb02b79
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/SaltingUtil.java
@@ -0,0 +1,172 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.phoenix.compile.ScanRanges;
+import org.apache.phoenix.query.KeyRange;
+import org.apache.phoenix.query.KeyRange.Bound;
+import org.apache.phoenix.util.ScanUtil;
+
+import com.google.common.collect.Lists;
+
+
+/**
+ * Utility methods related to transparent salting of row keys.
+ */
+public class SaltingUtil {
+    public static final int NUM_SALTING_BYTES = 1;
+    public static final Integer MAX_BUCKET_NUM = 256; // Unsigned byte.
+    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, PDataType.BINARY, 1, 0, false, 0, null, 0);
+
+    public static List<KeyRange> generateAllSaltingRanges(int bucketNum) {
+        List<KeyRange> allRanges = Lists.<KeyRange>newArrayListWithExpectedSize(bucketNum);
+        for (int i=0; i<bucketNum; i++) {
+            byte[] saltByte = new byte[] {(byte) i};
+            allRanges.add(SALTING_COLUMN.getDataType().getKeyRange(
+                    saltByte, true, saltByte, true));
+        }
+        return allRanges;
+    }
+
+    public static byte[][] getSalteByteSplitPoints(int saltBucketNum) {
+        byte[][] splits = new byte[saltBucketNum-1][];
+        for (int i = 1; i < saltBucketNum; i++) {
+            splits[i-1] = new byte[] {(byte) i};
+        }
+        return splits;
+    }
+
+    // Compute the hash of the key value stored in key and set its first byte as the value. The
+    // first byte of key should be left empty as a place holder for the salting byte.
+    public static byte[] getSaltedKey(ImmutableBytesWritable key, int bucketNum) {
+        byte[] keyBytes = new byte[key.getLength()];
+        byte saltByte = getSaltingByte(key.get(), key.getOffset() + 1, key.getLength() - 1, bucketNum);
+        keyBytes[0] = saltByte;
+        System.arraycopy(key.get(), key.getOffset() + 1, keyBytes, 1, key.getLength() - 1);
+        return keyBytes;
+    }
+
+    // Generate the bucket byte given a byte array and the number of buckets.
+    public static byte getSaltingByte(byte[] value, int offset, int length, int bucketNum) {
+        int hash = hashCode(value, offset, length);
+        byte bucketByte = (byte) ((Math.abs(hash) % bucketNum));
+        return bucketByte;
+    }
+
+    private static int hashCode(byte a[], int offset, int length) {
+        if (a == null)
+            return 0;
+        int result = 1;
+        for (int i = offset; i < offset + length; i++) {
+            result = 31 * result + a[i];
+        }
+        return result;
+    }
+
+    public static List<List<KeyRange>> setSaltByte(List<List<KeyRange>> ranges, int bucketNum) {
+        if (ranges == null || ranges.isEmpty()) {
+            return ScanRanges.NOTHING.getRanges();
+        }
+        for (int i = 1; i < ranges.size(); i++) {
+            List<KeyRange> range = ranges.get(i);
+            if (range != null && !range.isEmpty()) {
+                throw new IllegalStateException();
+            }
+        }
+        List<KeyRange> newRanges = Lists.newArrayListWithExpectedSize(ranges.size());
+        for (KeyRange range : ranges.get(0)) {
+            if (!range.isSingleKey()) {
+                throw new IllegalStateException();
+            }
+            byte[] key = range.getLowerRange();
+            byte saltByte = SaltingUtil.getSaltingByte(key, 0, key.length, bucketNum);
+            byte[] saltedKey = new byte[key.length + 1];
+            System.arraycopy(key, 0, saltedKey, 1, key.length);   
+            saltedKey[0] = saltByte;
+            newRanges.add(KeyRange.getKeyRange(saltedKey, true, saltedKey, true));
+        }
+        return Collections.singletonList(newRanges);
+    }
+    
+    public static List<List<KeyRange>> flattenRanges(List<List<KeyRange>> ranges, RowKeySchema schema, int bucketNum) {
+        if (ranges == null || ranges.isEmpty()) {
+            return ScanRanges.NOTHING.getRanges();
+        }
+        int count = 1;
+        // Skip salt byte range in the first position
+        for (int i = 1; i < ranges.size(); i++) {
+            count *= ranges.get(i).size();
+        }
+        KeyRange[] expandedRanges = new KeyRange[count];
+        int[] position = new int[ranges.size()];
+        int estimatedKeyLength = ScanUtil.estimateMaximumKeyLength(schema, 1, ranges);
+        int idx = 0, length;
+        byte saltByte;
+        byte[] key = new byte[estimatedKeyLength];
+        do {
+            length = ScanUtil.setKey(schema, ranges, position, Bound.LOWER, key, 1, 0, ranges.size(), 1);
+            saltByte = SaltingUtil.getSaltingByte(key, 1, length, bucketNum);
+            key[0] = saltByte;
+            byte[] saltedKey = Arrays.copyOf(key, length + 1);
+            KeyRange range = PDataType.VARBINARY.getKeyRange(saltedKey, true, saltedKey, true);
+            expandedRanges[idx++] = range;
+        } while (incrementKey(ranges, position));
+        // The comparator is imperfect, but sufficient for all single keys.
+        Arrays.sort(expandedRanges, KeyRange.COMPARATOR);
+        List<KeyRange> expandedRangesList = Arrays.asList(expandedRanges);
+        return Collections.singletonList(expandedRangesList);
+    }
+
+    private static boolean incrementKey(List<List<KeyRange>> slots, int[] position) {
+        int idx = slots.size() - 1;
+        while (idx >= 0 && (position[idx] = (position[idx] + 1) % slots.get(idx).size()) == 0) {
+            idx--;
+        }
+        return idx >= 0;
+    }
+
+    public static KeyRange addSaltByte(byte[] startKey, KeyRange minMaxRange) {
+        byte saltByte = startKey.length == 0 ? 0 : startKey[0];
+        byte[] lowerRange = minMaxRange.getLowerRange();
+        if(!minMaxRange.lowerUnbound()) {
+            byte[] newLowerRange = new byte[lowerRange.length + 1];
+            newLowerRange[0] = saltByte;
+            System.arraycopy(lowerRange, 0, newLowerRange, 1, lowerRange.length);
+            lowerRange = newLowerRange;
+        }
+        byte[] upperRange = minMaxRange.getUpperRange();
+
+        if(!minMaxRange.upperUnbound()) { 
+            byte[] newUpperRange = new byte[upperRange.length + 1];
+            newUpperRange[0] = saltByte;
+            System.arraycopy(upperRange, 0, newUpperRange, 1, upperRange.length);
+            upperRange = newUpperRange;
+        }
+        return KeyRange.getKeyRange(lowerRange, upperRange);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/Sequence.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/Sequence.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/Sequence.java
new file mode 100644
index 0000000..f88475d
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/Sequence.java
@@ -0,0 +1,391 @@
+package org.apache.phoenix.schema;
+
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CACHE_SIZE_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CURRENT_VALUE_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INCREMENT_BY_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SEQUENCE_FAMILY_BYTES;
+
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Append;
+import org.apache.hadoop.hbase.client.Increment;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import com.google.common.collect.Lists;
+import org.apache.phoenix.coprocessor.MetaDataProtocol;
+import org.apache.phoenix.coprocessor.SequenceRegionObserver;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.KeyValueUtil;
+import org.apache.phoenix.util.SchemaUtil;
+
+public class Sequence {
+    public static final int SUCCESS = 0;
+    
+    private static final Long AMOUNT = Long.valueOf(0L);
+    // Pre-compute index of sequence key values to prevent binary search
+    private static final KeyValue CURRENT_VALUE_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, SEQUENCE_FAMILY_BYTES, CURRENT_VALUE_BYTES);
+    private static final KeyValue INCREMENT_BY_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, SEQUENCE_FAMILY_BYTES, INCREMENT_BY_BYTES);
+    private static final KeyValue CACHE_SIZE_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, SEQUENCE_FAMILY_BYTES, CACHE_SIZE_BYTES);
+    private static final List<KeyValue> SEQUENCE_KV_COLUMNS = Arrays.<KeyValue>asList(
+            CURRENT_VALUE_KV,
+            INCREMENT_BY_KV,
+            CACHE_SIZE_KV
+            );
+    static {
+        Collections.sort(SEQUENCE_KV_COLUMNS, KeyValue.COMPARATOR);
+    }
+    private static final int CURRENT_VALUE_INDEX = SEQUENCE_KV_COLUMNS.indexOf(CURRENT_VALUE_KV);
+    private static final int INCREMENT_BY_INDEX = SEQUENCE_KV_COLUMNS.indexOf(INCREMENT_BY_KV);
+    private static final int CACHE_SIZE_INDEX = SEQUENCE_KV_COLUMNS.indexOf(CACHE_SIZE_KV);
+
+    private static final int SEQUENCE_KEY_VALUES = SEQUENCE_KV_COLUMNS.size();
+    private static final EmptySequenceCacheException EMPTY_SEQUENCE_CACHE_EXCEPTION = new EmptySequenceCacheException();
+    
+    private final SequenceKey key;
+    private final ReentrantLock lock;
+    private List<SequenceValue> values;
+    
+    public Sequence(SequenceKey key) {
+        if (key == null) throw new NullPointerException();
+        this.key = key;
+        this.lock = new ReentrantLock();
+    }
+
+    private void insertSequenceValue(SequenceValue value) {
+        if (values == null) {
+            values = Lists.newArrayListWithExpectedSize(1);
+            values.add(value);
+        } else {
+            int i = values.size()-1;
+            while (i >= 0 && values.get(i).timestamp > value.timestamp) {
+                i--;
+            }
+            // Don't insert another value if there's one at the same timestamp that is a delete
+            if (i >= 0 && values.get(i).timestamp == value.timestamp) {
+                if (values.get(i).isDeleted()) {
+                    throw new IllegalStateException("Unexpected delete marker at timestamp " + value.timestamp + " for "+ key);
+                }
+                values.set(i, value);
+            } else {
+                values.add(i+1, value);
+            }
+        }
+    }
+    
+    private SequenceValue findSequenceValue(long timestamp) {
+        if (values == null) {
+            return null;
+        }
+        int i = values.size()-1;
+        while (i >= 0 && values.get(i).timestamp >= timestamp) {
+            i--;
+        }
+        if (i < 0) {
+            return null;
+        }
+        SequenceValue value = values.get(i);
+        return value.isDeleted() ? null : value;
+    }
+    
+    public long incrementValue(long timestamp, int factor) throws EmptySequenceCacheException {
+        SequenceValue value = findSequenceValue(timestamp);
+        if (value == null) {
+            throw EMPTY_SEQUENCE_CACHE_EXCEPTION;
+        }
+        if (value.currentValue == value.nextValue) {
+            throw EMPTY_SEQUENCE_CACHE_EXCEPTION;
+        }
+        long returnValue = value.currentValue;
+        value.currentValue += factor * value.incrementBy;
+        return returnValue;
+    }
+
+    public List<Append> newReturns() {
+        if (values == null) {
+            return Collections.emptyList();
+        }
+        List<Append> appends = Lists.newArrayListWithExpectedSize(values.size());
+        for (SequenceValue value : values) {
+            if (value.isInitialized() && value.currentValue != value.nextValue) {
+                appends.add(newReturn(value));
+            }
+        }
+        return appends;
+    }
+    
+    public Append newReturn(long timestamp) throws EmptySequenceCacheException {
+        SequenceValue value = findSequenceValue(timestamp);
+        if (value == null) {
+            throw EMPTY_SEQUENCE_CACHE_EXCEPTION;
+        }
+        if (value.currentValue == value.nextValue) {
+            throw EMPTY_SEQUENCE_CACHE_EXCEPTION;
+        }
+        return newReturn(value);
+    }
+
+    private Append newReturn(SequenceValue value) {
+        byte[] key = SchemaUtil.getSequenceKey(this.key.getTenantId(), this.key.getSchemaName(), this.key.getSequenceName());
+        Append append = new Append(key);
+        byte[] opBuf = new byte[] {(byte)SequenceRegionObserver.Op.RETURN_SEQUENCE.ordinal()};
+        append.setAttribute(SequenceRegionObserver.OPERATION_ATTRIB, opBuf);
+        append.setAttribute(SequenceRegionObserver.CURRENT_VALUE_ATTRIB, PDataType.LONG.toBytes(value.nextValue));
+        Map<byte[], List<KeyValue>> familyMap = append.getFamilyMap();
+        familyMap.put(PhoenixDatabaseMetaData.SEQUENCE_FAMILY_BYTES, Arrays.<KeyValue>asList(
+                KeyValueUtil.newKeyValue(key, PhoenixDatabaseMetaData.SEQUENCE_FAMILY_BYTES, PhoenixDatabaseMetaData.CURRENT_VALUE_BYTES, value.timestamp, PDataType.LONG.toBytes(value.currentValue))
+                ));
+        return append;
+    }
+    
+    public long currentValue(long timestamp) throws EmptySequenceCacheException {
+        SequenceValue value = findSequenceValue(timestamp);
+        if (value == null || value.isUnitialized()) {
+            throw EMPTY_SEQUENCE_CACHE_EXCEPTION;
+        }
+        return value.currentValue - value.incrementBy;
+    }
+
+    public ReentrantLock getLock() {
+        return lock;
+    }
+
+    public SequenceKey getKey() {
+        return key;
+    }
+
+    public long incrementValue(Result result, int factor) throws SQLException {
+        // In this case, we don't definitely know the timestamp of the deleted sequence,
+        // but we know anything older is likely deleted. Worse case, we remove a sequence
+        // from the cache that we shouldn't have which will cause a gap in sequence values.
+        // In that case, we might get an error that a curr value was done on a sequence
+        // before a next val was. Not sure how to prevent that.
+        if (result.raw().length == 1) {
+            KeyValue errorKV = result.raw()[0];
+            int errorCode = PDataType.INTEGER.getCodec().decodeInt(errorKV.getBuffer(), errorKV.getValueOffset(), null);
+            SQLExceptionCode code = SQLExceptionCode.fromErrorCode(errorCode);
+            // TODO: We could have the server return the timestamps of the
+            // delete markers and we could insert them here, but this seems
+            // like overkill.
+            // if (code == SQLExceptionCode.SEQUENCE_UNDEFINED) {
+            // }
+            throw new SQLExceptionInfo.Builder(code)
+                .setSchemaName(key.getSchemaName())
+                .setTableName(key.getSequenceName())
+                .build().buildException();
+        }
+        // If we found the sequence, we update our cache with the new value
+        SequenceValue value = new SequenceValue(result);
+        insertSequenceValue(value);
+        long currentValue = value.currentValue;
+        value.currentValue += factor * value.incrementBy;
+        return currentValue;
+    }
+
+    public Increment newIncrement(long timestamp) {
+        Increment inc = new Increment(SchemaUtil.getSequenceKey(key.getTenantId(), key.getSchemaName(), key.getSequenceName()));
+        // It doesn't matter what we set the amount too - we always use the values we get
+        // from the Get we do to prevent any race conditions. All columns that get added
+        // are returned with their current value
+        try {
+            inc.setTimeRange(MetaDataProtocol.MIN_TABLE_TIMESTAMP, timestamp);
+        } catch (IOException e) {
+            throw new RuntimeException(e); // Impossible
+        }
+        for (KeyValue kv : SEQUENCE_KV_COLUMNS) {
+            // We don't care about the amount, as we'll add what gets looked up on the server-side
+            inc.addColumn(kv.getFamily(), kv.getQualifier(), AMOUNT);
+        }
+        return inc;
+    }
+    
+    public static KeyValue getCurrentValueKV(List<KeyValue> kvs) {
+        assert(kvs.size() == SEQUENCE_KEY_VALUES);
+        return kvs.get(CURRENT_VALUE_INDEX);
+    }
+    
+    public static KeyValue getIncrementByKV(List<KeyValue> kvs) {
+        assert(kvs.size() == SEQUENCE_KEY_VALUES);
+        return kvs.get(INCREMENT_BY_INDEX);
+    }
+    
+    public static KeyValue getCacheSizeKV(List<KeyValue> kvs) {
+        assert(kvs.size() == SEQUENCE_KEY_VALUES);
+        return kvs.get(CACHE_SIZE_INDEX);
+    }
+    
+    public static KeyValue getCurrentValueKV(Result r) {
+        KeyValue[] kvs = r.raw();
+        assert(kvs.length == SEQUENCE_KEY_VALUES);
+        return kvs[CURRENT_VALUE_INDEX];
+    }
+    
+    public static KeyValue getIncrementByKV(Result r) {
+        KeyValue[] kvs = r.raw();
+        assert(kvs.length == SEQUENCE_KEY_VALUES);
+        return kvs[INCREMENT_BY_INDEX];
+    }
+    
+    public static KeyValue getCacheSizeKV(Result r) {
+        KeyValue[] kvs = r.raw();
+        assert(kvs.length == SEQUENCE_KEY_VALUES);
+        return kvs[CACHE_SIZE_INDEX];
+    }
+    
+    public static Result replaceCurrentValueKV(Result r, KeyValue currentValueKV) {
+        KeyValue[] kvs = r.raw();
+        List<KeyValue> newkvs = Lists.newArrayList(kvs);
+        newkvs.set(CURRENT_VALUE_INDEX, currentValueKV);
+        return new Result(newkvs);
+    }
+    
+    private static final class SequenceValue {
+        public final long incrementBy;
+        public final int cacheSize;
+        public final long timestamp;
+        
+        public long currentValue;
+        public long nextValue;
+        
+        public SequenceValue(long timestamp) {
+            this(timestamp, false);
+        }
+        
+        public SequenceValue(long timestamp, boolean isDeleted) {
+            this.timestamp = timestamp;
+            this.incrementBy = isDeleted ? -1 : 0;
+            this.cacheSize = 0;
+        }
+        
+        public boolean isInitialized() {
+            return this.incrementBy > 0;
+        }
+        
+        public boolean isUnitialized() {
+            return this.incrementBy == 0;
+        }
+        
+        public boolean isDeleted() {
+            return this.incrementBy < 0;
+        }
+        
+        public SequenceValue(Result r) {
+            KeyValue currentValueKV = getCurrentValueKV(r);
+            KeyValue incrementByKV = getIncrementByKV(r);
+            KeyValue cacheSizeKV = getCacheSizeKV(r);
+            timestamp = currentValueKV.getTimestamp();
+            nextValue = PDataType.LONG.getCodec().decodeLong(currentValueKV.getBuffer(), currentValueKV.getValueOffset(), null);
+            incrementBy = PDataType.LONG.getCodec().decodeLong(incrementByKV.getBuffer(), incrementByKV.getValueOffset(), null);
+            cacheSize = PDataType.INTEGER.getCodec().decodeInt(cacheSizeKV.getBuffer(), cacheSizeKV.getValueOffset(), null);
+            currentValue = nextValue - incrementBy * cacheSize;
+        }
+    }
+
+    public boolean returnValue(Result result) throws SQLException {
+        KeyValue statusKV = result.raw()[0];
+        if (statusKV.getValueLength() == 0) { // No error, but unable to return sequence values
+            return false;
+        }
+        long timestamp = statusKV.getTimestamp();
+        int statusCode = PDataType.INTEGER.getCodec().decodeInt(statusKV.getBuffer(), statusKV.getValueOffset(), null);
+        if (statusCode == SUCCESS) {  // Success - update nextValue down to currentValue
+            SequenceValue value = findSequenceValue(timestamp);
+            if (value == null) {
+                throw new EmptySequenceCacheException(key.getSchemaName(),key.getSequenceName());
+            }
+            value.nextValue = value.currentValue;
+            return true;
+        }
+        SQLExceptionCode code = SQLExceptionCode.fromErrorCode(statusCode);
+        // TODO: We could have the server return the timestamps of the
+        // delete markers and we could insert them here, but this seems
+        // like overkill.
+        // if (code == SQLExceptionCode.SEQUENCE_UNDEFINED) {
+        // }
+        throw new SQLExceptionInfo.Builder(code)
+            .setSchemaName(key.getSchemaName())
+            .setTableName(key.getSequenceName())
+            .build().buildException();
+    }
+
+    public Append createSequence(long startWith, long incrementBy, int cacheSize, long timestamp) {
+        byte[] key = SchemaUtil.getSequenceKey(this.key.getTenantId(), this.key.getSchemaName(), this.key.getSequenceName());
+        Append append = new Append(key);
+        append.setAttribute(SequenceRegionObserver.OPERATION_ATTRIB, new byte[] {(byte)SequenceRegionObserver.Op.CREATE_SEQUENCE.ordinal()});
+        if (timestamp != HConstants.LATEST_TIMESTAMP) {
+            append.setAttribute(SequenceRegionObserver.MAX_TIMERANGE_ATTRIB, Bytes.toBytes(timestamp));
+        }
+        Map<byte[], List<KeyValue>> familyMap = append.getFamilyMap();
+        byte[] startWithBuf = PDataType.LONG.toBytes(startWith);
+        familyMap.put(PhoenixDatabaseMetaData.SEQUENCE_FAMILY_BYTES, Arrays.<KeyValue>asList(
+                KeyValueUtil.newKeyValue(key, PhoenixDatabaseMetaData.SEQUENCE_FAMILY_BYTES, QueryConstants.EMPTY_COLUMN_BYTES, timestamp, ByteUtil.EMPTY_BYTE_ARRAY),
+                KeyValueUtil.newKeyValue(key, PhoenixDatabaseMetaData.SEQUENCE_FAMILY_BYTES, PhoenixDatabaseMetaData.CURRENT_VALUE_BYTES, timestamp, startWithBuf),
+                KeyValueUtil.newKeyValue(key, PhoenixDatabaseMetaData.SEQUENCE_FAMILY_BYTES, PhoenixDatabaseMetaData.START_WITH_BYTES, timestamp, startWithBuf),
+                KeyValueUtil.newKeyValue(key, PhoenixDatabaseMetaData.SEQUENCE_FAMILY_BYTES, PhoenixDatabaseMetaData.INCREMENT_BY_BYTES, timestamp, PDataType.LONG.toBytes(incrementBy)),
+                KeyValueUtil.newKeyValue(key, PhoenixDatabaseMetaData.SEQUENCE_FAMILY_BYTES, PhoenixDatabaseMetaData.CACHE_SIZE_BYTES, timestamp, PDataType.INTEGER.toBytes(cacheSize))
+                ));
+        return append;
+    }
+
+    public long createSequence(Result result) throws SQLException {
+        KeyValue statusKV = result.raw()[0];
+        long timestamp = statusKV.getTimestamp();
+        int statusCode = PDataType.INTEGER.getCodec().decodeInt(statusKV.getBuffer(), statusKV.getValueOffset(), null);
+        if (statusCode == 0) {  // Success - add sequence value and return timestamp
+            SequenceValue value = new SequenceValue(timestamp);
+            insertSequenceValue(value);
+            return timestamp;
+        }
+        SQLExceptionCode code = SQLExceptionCode.fromErrorCode(statusCode);
+        throw new SQLExceptionInfo.Builder(code)
+            .setSchemaName(key.getSchemaName())
+            .setTableName(key.getSequenceName())
+            .build().buildException();
+    }
+
+    public Append dropSequence(long timestamp) {
+        byte[] key = SchemaUtil.getSequenceKey(this.key.getTenantId(), this.key.getSchemaName(), this.key.getSequenceName());
+        Append append = new Append(key);
+        append.setAttribute(SequenceRegionObserver.OPERATION_ATTRIB, new byte[] {(byte)SequenceRegionObserver.Op.DROP_SEQUENCE.ordinal()});
+        if (timestamp != HConstants.LATEST_TIMESTAMP) {
+            append.setAttribute(SequenceRegionObserver.MAX_TIMERANGE_ATTRIB, Bytes.toBytes(timestamp));
+        }
+        Map<byte[], List<KeyValue>> familyMap = append.getFamilyMap();
+        familyMap.put(PhoenixDatabaseMetaData.SEQUENCE_FAMILY_BYTES, Arrays.<KeyValue>asList(
+                KeyValueUtil.newKeyValue(key, PhoenixDatabaseMetaData.SEQUENCE_FAMILY_BYTES, QueryConstants.EMPTY_COLUMN_BYTES, timestamp, ByteUtil.EMPTY_BYTE_ARRAY)));
+        return append;
+    }
+
+    public long dropSequence(Result result) throws SQLException {
+        KeyValue statusKV = result.raw()[0];
+        long timestamp = statusKV.getTimestamp();
+        int statusCode = PDataType.INTEGER.getCodec().decodeInt(statusKV.getBuffer(), statusKV.getValueOffset(), null);
+        SQLExceptionCode code = statusCode == 0 ? null : SQLExceptionCode.fromErrorCode(statusCode);
+        if (code == null) {
+            // Insert delete marker so that point-in-time sequences work
+            insertSequenceValue(new SequenceValue(timestamp, true));
+            return timestamp;
+        }
+        // TODO: We could have the server return the timestamps of the
+        // delete markers and we could insert them here, but this seems
+        // like overkill.
+        // if (code == SQLExceptionCode.SEQUENCE_UNDEFINED) {
+        // }
+        throw new SQLExceptionInfo.Builder(code)
+            .setSchemaName(key.getSchemaName())
+            .setTableName(key.getSequenceName())
+            .build().buildException();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/SequenceAlreadyExistsException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/SequenceAlreadyExistsException.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/SequenceAlreadyExistsException.java
new file mode 100644
index 0000000..cdb7bdb
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/SequenceAlreadyExistsException.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.sql.SQLException;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+
+
+public class SequenceAlreadyExistsException extends SQLException {
+	private static final long serialVersionUID = 1L;
+	private static SQLExceptionCode code = SQLExceptionCode.SEQUENCE_ALREADY_EXIST;
+	private final String schemaName;
+	private final String sequenceName;
+
+
+	public SequenceAlreadyExistsException(String schemaName, String sequenceName) {
+		super(new SQLExceptionInfo.Builder(code).setSchemaName(schemaName).setTableName(sequenceName).build().toString(),
+				code.getSQLState(), code.getErrorCode());        
+		this.schemaName = schemaName;
+		this.sequenceName = sequenceName;
+	}
+
+	public String getSequenceName() {
+		return sequenceName;
+	}
+
+	public String getSchemaName() {
+		return schemaName;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/SequenceKey.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/SequenceKey.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/SequenceKey.java
new file mode 100644
index 0000000..2be5ace
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/SequenceKey.java
@@ -0,0 +1,76 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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;
+
+
+public class SequenceKey implements Comparable<SequenceKey> {
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + ((tenantId == null) ? 0 : tenantId.hashCode());
+        result = prime * result + ((schemaName == null) ? 0 : schemaName.hashCode());
+        result = prime * result + sequenceName.hashCode();
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) return true;
+        if (obj == null) return false;
+        if (getClass() != obj.getClass()) return false;
+        SequenceKey other = (SequenceKey)obj;
+        return this.compareTo(other) == 0;
+    }
+
+    private final String tenantId;
+    private final String schemaName;
+    private final String sequenceName;
+    
+    public SequenceKey(String tenantId, String schemaName, String sequenceName) {
+        this.tenantId = tenantId;
+        this.schemaName = schemaName;
+        this.sequenceName = sequenceName;
+    }
+
+    public String getTenantId() {
+        return tenantId;
+    }
+
+    public String getSchemaName() {
+        return schemaName;
+    }
+
+    public String getSequenceName() {
+        return sequenceName;
+    }
+
+    @Override
+    public int compareTo(SequenceKey that) {
+        int c = this.tenantId == that.getTenantId() ? 0 : this.tenantId == null ? -1 : that.getTenantId() == null ? 1 : this.tenantId.compareTo(that.getTenantId());
+        if (c == 0) {
+            c = this.schemaName == that.getSchemaName() ? 0 : this.schemaName == null ? -1 : that.getSchemaName() == null ? 1 : this.schemaName.compareTo(that.getSchemaName());
+            if (c == 0) {
+                return sequenceName.compareTo(that.getSequenceName());
+            }
+        }
+        return c;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/SequenceNotFoundException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/SequenceNotFoundException.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/SequenceNotFoundException.java
new file mode 100644
index 0000000..466a3b9
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/SequenceNotFoundException.java
@@ -0,0 +1,50 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+
+
+public class SequenceNotFoundException extends MetaDataEntityNotFoundException {
+	private static final long serialVersionUID = 1L;
+	private static SQLExceptionCode code = SQLExceptionCode.SEQUENCE_UNDEFINED;
+	private final String schemaName;
+	private final String tableName;
+
+	public SequenceNotFoundException(String tableName) {
+		this(null, tableName);
+	}
+
+	public SequenceNotFoundException(String schemaName, String tableName) {
+		super(new SQLExceptionInfo.Builder(code).setSchemaName(schemaName).setTableName(tableName).build().toString(),
+				code.getSQLState(), code.getErrorCode(), null);
+		this.tableName = tableName;
+		this.schemaName = schemaName;
+	}
+
+	public String getTableName() {
+		return tableName;
+	}
+
+	public String getSchemaName() {
+		return schemaName;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/TableAlreadyExistsException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/TableAlreadyExistsException.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/TableAlreadyExistsException.java
new file mode 100644
index 0000000..7781ba2
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/TableAlreadyExistsException.java
@@ -0,0 +1,55 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.sql.SQLException;
+
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+
+
+/**
+ * 
+ * Exception thrown when a table name could not be found in the schema
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class TableAlreadyExistsException extends SQLException {
+    private static final long serialVersionUID = 1L;
+    private static SQLExceptionCode code = SQLExceptionCode.TABLE_ALREADY_EXIST;
+    private final String schemaName;
+    private final String tableName;
+
+    public TableAlreadyExistsException(String schemaName, String tableName) {
+        super(new SQLExceptionInfo.Builder(code).setSchemaName(schemaName).setTableName(tableName).build().toString(),
+                code.getSQLState(), code.getErrorCode());
+        this.schemaName = schemaName;
+        this.tableName = tableName;
+    }
+
+    public String getTableName() {
+        return tableName;
+    }
+
+    public String getSchemaName() {
+        return schemaName;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/TableNotFoundException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/TableNotFoundException.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/TableNotFoundException.java
new file mode 100644
index 0000000..7b66603
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/TableNotFoundException.java
@@ -0,0 +1,73 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.HConstants;
+
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+
+
+/**
+ * 
+ * Exception thrown when a table name could not be found in the schema
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class TableNotFoundException extends MetaDataEntityNotFoundException {
+    private static final long serialVersionUID = 1L;
+    private static SQLExceptionCode code = SQLExceptionCode.TABLE_UNDEFINED;
+    private final String schemaName;
+    private final String tableName;
+    private final long timestamp;
+
+    public TableNotFoundException(TableNotFoundException e, long timestamp) {
+        this(e.schemaName,e.tableName, timestamp);
+    }
+
+    public TableNotFoundException(String tableName) {
+        this(null, tableName);
+    }
+
+    public TableNotFoundException(String schemaName, String tableName) {
+        this(schemaName, tableName, HConstants.LATEST_TIMESTAMP);
+    }
+    
+    public TableNotFoundException(String schemaName, String tableName, long timestamp) {
+        super(new SQLExceptionInfo.Builder(code).setSchemaName(schemaName).setTableName(tableName).build().toString(),
+                code.getSQLState(), code.getErrorCode(), null);
+        this.tableName = tableName;
+        this.schemaName = schemaName;
+        this.timestamp = timestamp;
+    }
+
+    public String getTableName() {
+        return tableName;
+    }
+
+    public String getSchemaName() {
+        return schemaName;
+    }
+
+    public long getTimeStamp() {
+        return timestamp;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/TableRef.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/TableRef.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/TableRef.java
new file mode 100644
index 0000000..14705ca
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/TableRef.java
@@ -0,0 +1,82 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.HConstants;
+
+
+
+public final class TableRef {
+    private final PTable table;
+    private final String alias;
+    private final long timeStamp;
+    private final boolean hasDynamicCols;
+
+    public TableRef(TableRef tableRef, long timeStamp) {
+        this(tableRef.alias, tableRef.table, timeStamp, tableRef.hasDynamicCols);
+    }
+    
+    public TableRef(PTable table) {
+        this(null, table, HConstants.LATEST_TIMESTAMP, false);
+    }
+
+    public TableRef(String alias, PTable table, long timeStamp, boolean hasDynamicCols) {
+        this.alias = alias;
+        this.table = table;
+        this.timeStamp = timeStamp;
+        this.hasDynamicCols = hasDynamicCols;
+    }
+    
+    public PTable getTable() {
+        return table;
+    }
+
+    public String getTableAlias() {
+        return alias;
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = alias == null ? 0 : alias.hashCode();
+        result = prime * result + this.table.getName().getString().hashCode();
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) return true;
+        if (obj == null) return false;
+        if (getClass() != obj.getClass()) return false;
+        TableRef other = (TableRef)obj;
+        if ((alias == null && other.alias != null) || (alias != null && !alias.equals(other.alias))) return false;
+        if (!table.getName().getString().equals(other.table.getName().getString())) return false;
+        return true;
+    }
+
+    public long getTimeStamp() {
+        return timeStamp;
+    }
+
+    public boolean hasDynamicCols() {
+        return hasDynamicCols;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/TypeMismatchException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/TypeMismatchException.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/TypeMismatchException.java
new file mode 100644
index 0000000..ae24873
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/TypeMismatchException.java
@@ -0,0 +1,60 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.sql.SQLException;
+
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+
+/**
+ * Exception thrown when we try to convert one type into a different incompatible type.
+ * 
+ * @author zhuang
+ * @since 1.0
+ */
+public class TypeMismatchException extends SQLException {
+    private static final long serialVersionUID = 1L;
+    private static SQLExceptionCode code = SQLExceptionCode.TYPE_MISMATCH;
+
+    public TypeMismatchException(String msg) {
+        super(new SQLExceptionInfo.Builder(code).setMessage(msg).build().toString(), code.getSQLState(), code.getErrorCode());
+    }
+
+    public static TypeMismatchException newException(PDataType lhs)  {
+        return new TypeMismatchException(getMessage(lhs,null,null));
+    }
+    
+    public static TypeMismatchException newException(PDataType lhs, String location)  {
+        return new TypeMismatchException(getMessage(lhs,null,location));
+    }
+    
+    public static TypeMismatchException newException(PDataType lhs, PDataType rhs)  {
+        return new TypeMismatchException(getMessage(lhs,rhs,null));
+    }
+    
+    public static TypeMismatchException newException(PDataType lhs, PDataType rhs, String location)  {
+        return new TypeMismatchException(getMessage(lhs,rhs,location));
+    }
+    
+    public static String getMessage(PDataType lhs, PDataType rhs, String location) {
+        return lhs + (rhs == null ? "" : " and " + rhs) + (location == null ? "" : " for " + location);
+    }
+}


[05/51] [partial] Initial commit of master branch from github

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..4ea8a11
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/ScanUtil.java
@@ -0,0 +1,447 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.io.IOException;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableSet;
+import java.util.TreeMap;
+
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import com.google.common.collect.Lists;
+import org.apache.phoenix.compile.ScanRanges;
+import org.apache.phoenix.coprocessor.MetaDataProtocol;
+import org.apache.phoenix.filter.SkipScanFilter;
+import org.apache.phoenix.query.KeyRange;
+import org.apache.phoenix.query.KeyRange.Bound;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.RowKeySchema;
+
+
+/**
+ * 
+ * Various utilities for scans
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class ScanUtil {
+
+    private ScanUtil() {
+    }
+
+    public static void setTenantId(Scan scan, byte[] tenantId) {
+        scan.setAttribute(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId);
+    }
+
+    // Use getTenantId and pass in column name to match against
+    // in as PSchema attribute. If column name matches in 
+    // KeyExpressions, set on scan as attribute
+    public static ImmutableBytesWritable getTenantId(Scan scan) {
+        // Create Scan with special aggregation column over which to aggregate
+        byte[] tenantId = scan.getAttribute(PhoenixRuntime.TENANT_ID_ATTRIB);
+        if (tenantId == null) {
+            return null;
+        }
+        return new ImmutableBytesWritable(tenantId);
+    }
+
+    public static Scan newScan(Scan scan) {
+        try {
+            Scan newScan = new Scan(scan);
+            // Clone the underlying family map instead of sharing it between
+            // the existing and cloned Scan (which is the retarded default
+            // behavior).
+            TreeMap<byte [], NavigableSet<byte []>> existingMap = (TreeMap<byte[], NavigableSet<byte[]>>)scan.getFamilyMap();
+            Map<byte [], NavigableSet<byte []>> clonedMap = new TreeMap<byte [], NavigableSet<byte []>>(existingMap);
+            newScan.setFamilyMap(clonedMap);
+            return newScan;
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+    /**
+     * Intersects the scan start/stop row with the startKey and stopKey
+     * @param scan
+     * @param startKey
+     * @param stopKey
+     * @return false if the Scan cannot possibly return rows and true otherwise
+     */
+    public static boolean intersectScanRange(Scan scan, byte[] startKey, byte[] stopKey) {
+        return intersectScanRange(scan, startKey, stopKey, false);
+    }
+
+    public static boolean intersectScanRange(Scan scan, byte[] startKey, byte[] stopKey, boolean useSkipScan) {
+        boolean mayHaveRows = false;
+        byte[] existingStartKey = scan.getStartRow();
+        byte[] existingStopKey = scan.getStopRow();
+        if (existingStartKey.length > 0) {
+            if (startKey.length == 0 || Bytes.compareTo(existingStartKey, startKey) > 0) {
+                startKey = existingStartKey;
+            }
+        } else {
+            mayHaveRows = true;
+        }
+        if (existingStopKey.length > 0) {
+            if (stopKey.length == 0 || Bytes.compareTo(existingStopKey, stopKey) < 0) {
+                stopKey = existingStopKey;
+            }
+        } else {
+            mayHaveRows = true;
+        }
+        scan.setStartRow(startKey);
+        scan.setStopRow(stopKey);
+        
+        mayHaveRows = mayHaveRows || Bytes.compareTo(scan.getStartRow(), scan.getStopRow()) < 0;
+        
+        // If the scan is using skip scan filter, intersect and replace the filter.
+        if (mayHaveRows && useSkipScan) {
+            Filter filter = scan.getFilter();
+            if (filter instanceof SkipScanFilter) {
+                SkipScanFilter oldFilter = (SkipScanFilter)filter;
+                SkipScanFilter newFilter = oldFilter.intersect(startKey, stopKey);
+                if (newFilter == null) {
+                    return false;
+                }
+                // Intersect found: replace skip scan with intersected one
+                scan.setFilter(newFilter);
+            } else if (filter instanceof FilterList) {
+                FilterList filterList = (FilterList)filter;
+                Filter firstFilter = filterList.getFilters().get(0);
+                if (firstFilter instanceof SkipScanFilter) {
+                    SkipScanFilter oldFilter = (SkipScanFilter)firstFilter;
+                    SkipScanFilter newFilter = oldFilter.intersect(startKey, stopKey);
+                    if (newFilter == null) {
+                        return false;
+                    }
+                    // Intersect found: replace skip scan with intersected one
+                    List<Filter> allFilters = new ArrayList<Filter>(filterList.getFilters().size());
+                    allFilters.addAll(filterList.getFilters());
+                    allFilters.set(0, newFilter);
+                    scan.setFilter(new FilterList(FilterList.Operator.MUST_PASS_ALL,allFilters));
+                }
+            }
+        }
+        return mayHaveRows;
+    }
+
+    public static void andFilterAtBeginning(Scan scan, Filter andWithFilter) {
+        if (andWithFilter == null) {
+            return;
+        }
+        Filter filter = scan.getFilter();
+        if (filter == null) {
+            scan.setFilter(andWithFilter); 
+        } else if (filter instanceof FilterList && ((FilterList)filter).getOperator() == FilterList.Operator.MUST_PASS_ALL) {
+            FilterList filterList = (FilterList)filter;
+            List<Filter> allFilters = new ArrayList<Filter>(filterList.getFilters().size() + 1);
+            allFilters.add(andWithFilter);
+            allFilters.addAll(filterList.getFilters());
+            scan.setFilter(new FilterList(FilterList.Operator.MUST_PASS_ALL,allFilters));
+        } else {
+            scan.setFilter(new FilterList(FilterList.Operator.MUST_PASS_ALL,Arrays.asList(andWithFilter, filter)));
+        }
+    }
+
+    public static void andFilterAtEnd(Scan scan, Filter andWithFilter) {
+        if (andWithFilter == null) {
+            return;
+        }
+        Filter filter = scan.getFilter();
+        if (filter == null) {
+            scan.setFilter(andWithFilter); 
+        } else if (filter instanceof FilterList && ((FilterList)filter).getOperator() == FilterList.Operator.MUST_PASS_ALL) {
+            FilterList filterList = (FilterList)filter;
+            List<Filter> allFilters = new ArrayList<Filter>(filterList.getFilters().size() + 1);
+            allFilters.addAll(filterList.getFilters());
+            allFilters.add(andWithFilter);
+            scan.setFilter(new FilterList(FilterList.Operator.MUST_PASS_ALL,allFilters));
+        } else {
+            scan.setFilter(new FilterList(FilterList.Operator.MUST_PASS_ALL,Arrays.asList(filter, andWithFilter)));
+        }
+    }
+
+    public static void setTimeRange(Scan scan, long ts) {
+        try {
+            scan.setTimeRange(MetaDataProtocol.MIN_TABLE_TIMESTAMP, ts);
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public static byte[] getMinKey(RowKeySchema schema, List<List<KeyRange>> slots) {
+        return getKey(schema, slots, Bound.LOWER);
+    }
+
+    public static byte[] getMaxKey(RowKeySchema schema, List<List<KeyRange>> slots) {
+        return getKey(schema, slots, Bound.UPPER);
+    }
+
+    private static byte[] getKey(RowKeySchema schema, List<List<KeyRange>> slots, Bound bound) {
+        if (slots.isEmpty()) {
+            return null;
+        }
+        int[] position = new int[slots.size()];
+        int maxLength = 0;
+        for (int i = 0; i < position.length; i++) {
+            position[i] = bound == Bound.LOWER ? 0 : slots.get(i).size()-1;
+            KeyRange range = slots.get(i).get(position[i]);
+            maxLength += range.getRange(bound).length + (schema.getField(i).getDataType().isFixedWidth() ? 0 : 1);
+        }
+        byte[] key = new byte[maxLength];
+        int length = setKey(schema, slots, position, bound, key, 0, 0, position.length);
+        if (length == 0) {
+            return null;
+        }
+        if (length == maxLength) {
+            return key;
+        }
+        byte[] keyCopy = new byte[length];
+        System.arraycopy(key, 0, keyCopy, 0, length);
+        return keyCopy;
+    }
+
+    public static int estimateMaximumKeyLength(RowKeySchema schema, int schemaStartIndex, List<List<KeyRange>> slots) {
+        int maxLowerKeyLength = 0, maxUpperKeyLength = 0;
+        for (int i = 0; i < slots.size(); i++) {
+            int maxLowerRangeLength = 0, maxUpperRangeLength = 0;
+            for (KeyRange range: slots.get(i)) {
+                maxLowerRangeLength = Math.max(maxLowerRangeLength, range.getLowerRange().length); 
+                maxUpperRangeLength = Math.max(maxUpperRangeLength, range.getUpperRange().length);
+            }
+            int trailingByte = (schema.getField(schemaStartIndex).getDataType().isFixedWidth() ||
+                    schemaStartIndex == schema.getFieldCount() - 1 ? 0 : 1);
+            maxLowerKeyLength += maxLowerRangeLength + trailingByte;
+            maxUpperKeyLength += maxUpperKeyLength + trailingByte;
+            schemaStartIndex++;
+        }
+        return Math.max(maxLowerKeyLength, maxUpperKeyLength);
+    }
+
+    /*
+     * Set the key by appending the keyRanges inside slots at positions as specified by the position array.
+     * 
+     * We need to increment part of the key range, or increment the whole key at the end, depending on the
+     * bound we are setting and whether the key range is inclusive or exclusive. The logic for determining
+     * whether to increment or not is:
+     * range/single    boundary       bound      increment
+     *  range          inclusive      lower         no
+     *  range          inclusive      upper         yes, at the end if occurs at any slots.
+     *  range          exclusive      lower         yes
+     *  range          exclusive      upper         no
+     *  single         inclusive      lower         no
+     *  single         inclusive      upper         yes, at the end if it is the last slots.
+     */
+    public static int setKey(RowKeySchema schema, List<List<KeyRange>> slots, int[] position, Bound bound,
+            byte[] key, int byteOffset, int slotStartIndex, int slotEndIndex) {
+        return setKey(schema, slots, position, bound, key, byteOffset, slotStartIndex, slotEndIndex, slotStartIndex);
+    }
+
+    public static int setKey(RowKeySchema schema, List<List<KeyRange>> slots, int[] position, Bound bound,
+            byte[] key, int byteOffset, int slotStartIndex, int slotEndIndex, int schemaStartIndex) {
+        int offset = byteOffset;
+        boolean lastInclusiveUpperSingleKey = false;
+        boolean anyInclusiveUpperRangeKey = false;
+        for (int i = slotStartIndex; i < slotEndIndex; i++) {
+            // Build up the key by appending the bound of each key range
+            // from the current position of each slot. 
+            KeyRange range = slots.get(i).get(position[i]);
+            boolean isFixedWidth = schema.getField(schemaStartIndex++).getDataType().isFixedWidth();
+            /*
+             * If the current slot is unbound then stop if:
+             * 1) setting the upper bound. There's no value in
+             *    continuing because nothing will be filtered.
+             * 2) setting the lower bound when the type is fixed length
+             *    for the same reason. However, if the type is variable width
+             *    continue building the key because null values will be filtered
+             *    since our separator byte will be appended and incremented.
+             */
+            if (  range.isUnbound(bound) &&
+                ( bound == Bound.UPPER || isFixedWidth) ){
+                break;
+            }
+            byte[] bytes = range.getRange(bound);
+            System.arraycopy(bytes, 0, key, offset, bytes.length);
+            offset += bytes.length;
+            /*
+             * We must add a terminator to a variable length key even for the last PK column if
+             * the lower key is non inclusive or the upper key is inclusive. Otherwise, we'd be
+             * incrementing the key value itself, and thus bumping it up too much.
+             */
+            boolean inclusiveUpper = range.isInclusive(bound) && bound == Bound.UPPER;
+            boolean exclusiveLower = !range.isInclusive(bound) && bound == Bound.LOWER;
+            if (!isFixedWidth && ( i < schema.getMaxFields()-1 || inclusiveUpper || exclusiveLower)) {
+                key[offset++] = QueryConstants.SEPARATOR_BYTE;
+            }
+            // If we are setting the upper bound of using inclusive single key, we remember 
+            // to increment the key if we exit the loop after this iteration.
+            // 
+            // We remember to increment the last slot if we are setting the upper bound with an
+            // inclusive range key.
+            //
+            // We cannot combine the two flags together in case for single-inclusive key followed
+            // by the range-exclusive key. In that case, we do not need to increment the end at the
+            // end. But if we combine the two flag, the single inclusive key in the middle of the
+            // key slots would cause the flag to become true.
+            lastInclusiveUpperSingleKey = range.isSingleKey() && inclusiveUpper;
+            anyInclusiveUpperRangeKey |= !range.isSingleKey() && inclusiveUpper;
+            // If we are setting the lower bound with an exclusive range key, we need to bump the
+            // slot up for each key part. For an upper bound, we bump up an inclusive key, but
+            // only after the last key part.
+            if (!range.isSingleKey() && exclusiveLower) {
+                if (!ByteUtil.nextKey(key, offset)) {
+                    // Special case for not being able to increment.
+                    // In this case we return a negative byteOffset to
+                    // remove this part from the key being formed. Since the
+                    // key has overflowed, this means that we should not
+                    // have an end key specified.
+                    return -byteOffset;
+                }
+            }
+        }
+        if (lastInclusiveUpperSingleKey || anyInclusiveUpperRangeKey) {
+            if (!ByteUtil.nextKey(key, offset)) {
+                // Special case for not being able to increment.
+                // In this case we return a negative byteOffset to
+                // remove this part from the key being formed. Since the
+                // key has overflowed, this means that we should not
+                // have an end key specified.
+                return -byteOffset;
+            }
+        }
+        // Remove trailing separator bytes, since the columns may have been added
+        // after the table has data, in which case there won't be a separator
+        // byte.
+        if (bound == Bound.LOWER) {
+            while (schemaStartIndex > 0 && offset > byteOffset && 
+                    !schema.getField(--schemaStartIndex).getDataType().isFixedWidth() && 
+                    key[offset-1] == QueryConstants.SEPARATOR_BYTE) {
+                offset--;
+            }
+        }
+        return offset - byteOffset;
+    }
+
+    public static boolean isAllSingleRowScan(List<List<KeyRange>> ranges, RowKeySchema schema) {
+        if (ranges.size() < schema.getMaxFields()) {
+            return false;
+        }
+        for (int i = 0; i < ranges.size(); i++) {
+            List<KeyRange> orRanges = ranges.get(i);
+            for (KeyRange range: orRanges) {
+                if (!range.isSingleKey()) {
+                    return false;
+                }
+            }
+        }
+        return true;
+    }
+
+    /**
+     * Perform a binary lookup on the list of KeyRange for the tightest slot such that the slotBound
+     * of the current slot is higher or equal than the slotBound of our range. 
+     * @return  the index of the slot whose slot bound equals or are the tightest one that is 
+     *          smaller than rangeBound of range, or slots.length if no bound can be found.
+     */
+    public static int searchClosestKeyRangeWithUpperHigherThanPtr(List<KeyRange> slots, ImmutableBytesWritable ptr, int lower) {
+        int upper = slots.size() - 1;
+        int mid;
+        while (lower <= upper) {
+            mid = (lower + upper) / 2;
+            int cmp = slots.get(mid).compareUpperToLowerBound(ptr, true);
+            if (cmp < 0) {
+                lower = mid + 1;
+            } else if (cmp > 0) {
+                upper = mid - 1;
+            } else {
+                return mid;
+            }
+        }
+        mid = (lower + upper) / 2;
+        if (mid == 0 && slots.get(mid).compareUpperToLowerBound(ptr, true) > 0) {
+            return mid;
+        } else {
+            return ++mid;
+        }
+    }
+    
+    public static ScanRanges newScanRanges(List<Mutation> mutations) throws SQLException {
+        List<KeyRange> keys = Lists.newArrayListWithExpectedSize(mutations.size());
+        for (Mutation m : mutations) {
+            keys.add(PDataType.VARBINARY.getKeyRange(m.getRow()));
+        }
+        ScanRanges keyRanges = ScanRanges.create(Collections.singletonList(keys), SchemaUtil.VAR_BINARY_SCHEMA);
+        return keyRanges;
+    }
+
+    public static byte[] nextKey(byte[] key, PTable table, ImmutableBytesWritable ptr) {
+        int pos = 0;
+        RowKeySchema schema = table.getRowKeySchema();
+        int maxOffset = schema.iterator(key, ptr);
+        while (schema.next(ptr, pos, maxOffset) != null) {
+            pos++;
+        }
+        if (!schema.getField(pos-1).getDataType().isFixedWidth()) {
+            byte[] newLowerRange = new byte[key.length + 1];
+            System.arraycopy(key, 0, newLowerRange, 0, key.length);
+            newLowerRange[key.length] = QueryConstants.SEPARATOR_BYTE;
+            key = newLowerRange;
+        } else {
+            key = Arrays.copyOf(key, key.length);
+        }
+        ByteUtil.nextKey(key, key.length);
+        return key;
+    }
+
+    private static final String REVERSED_ATTR = "_reversed_";
+    
+    public static void setReversed(Scan scan) {
+        // TODO: set attribute dynamically here to prevent dependency on newer HBase release
+        scan.setAttribute(REVERSED_ATTR, PDataType.TRUE_BYTES);
+    }
+
+    // Start/stop row must be swapped if scan is being done in reverse
+    public static void swapStartStopRowIfReversed(Scan scan) {
+        if (isReversed(scan)) {
+            byte[] startRow = scan.getStartRow();
+            byte[] stopRow = scan.getStopRow();
+            scan.setStartRow(stopRow);
+            scan.setStopRow(startRow);
+        }
+    }
+
+    public static boolean isReversed(Scan scan) {
+        byte[] reversed = scan.getAttribute(REVERSED_ATTR);
+        return (PDataType.TRUE_BYTES.equals(reversed));
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..7b070f7
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
@@ -0,0 +1,507 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.jdbc.PhoenixDatabaseMetaData.TYPE_TABLE_NAME_BYTES;
+
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Properties;
+
+import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.AmbiguousColumnException;
+import org.apache.phoenix.schema.ColumnFamilyNotFoundException;
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.ColumnNotFoundException;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PColumnFamily;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.PDatum;
+import org.apache.phoenix.schema.PMetaData;
+import org.apache.phoenix.schema.PName;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.RowKeySchema;
+import org.apache.phoenix.schema.RowKeySchema.RowKeySchemaBuilder;
+import org.apache.phoenix.schema.SaltingUtil;
+
+
+
+/**
+ * 
+ * Static class for various schema-related utilities
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class SchemaUtil {
+    private static final int VAR_LENGTH_ESTIMATE = 10;
+    
+    public static final DataBlockEncoding DEFAULT_DATA_BLOCK_ENCODING = DataBlockEncoding.FAST_DIFF;
+    public static final RowKeySchema VAR_BINARY_SCHEMA = new RowKeySchemaBuilder(1).addField(new PDatum() {
+    
+        @Override
+        public boolean isNullable() {
+            return false;
+        }
+    
+        @Override
+        public PDataType getDataType() {
+            return PDataType.VARBINARY;
+        }
+    
+        @Override
+        public Integer getByteSize() {
+            return null;
+        }
+    
+        @Override
+        public Integer getMaxLength() {
+            return null;
+        }
+    
+        @Override
+        public Integer getScale() {
+            return null;
+        }
+    
+        @Override
+        public ColumnModifier getColumnModifier() {
+            return null;
+        }
+        
+    }, false, null).build();
+    
+    /**
+     * May not be instantiated
+     */
+    private SchemaUtil() {
+    }
+
+    public static boolean isPKColumn(PColumn column) {
+        return column.getFamilyName() == null;
+    }
+    
+    /**
+     * Estimate the max key length in bytes of the PK for a given table
+     * @param table the table
+     * @return the max PK length
+     */
+    public static int estimateKeyLength(PTable table) {
+        int maxKeyLength = 0;
+        // Calculate the max length of a key (each part must currently be of a fixed width)
+        int i = 0;
+        List<PColumn> columns = table.getPKColumns();
+        while (i < columns.size()) {
+            PColumn keyColumn = columns.get(i++);
+            Integer byteSize = keyColumn.getByteSize();
+            maxKeyLength += (byteSize == null) ? VAR_LENGTH_ESTIMATE : byteSize;
+        }
+        return maxKeyLength;
+    }
+
+    /**
+     * Normalize an identifier. If name is surrounded by double quotes,
+     * it is used as-is, otherwise the name is upper caased.
+     * @param name the parsed identifier
+     * @return the normalized identifier
+     */
+    public static String normalizeIdentifier(String name) {
+        if (name == null) {
+            return name;
+        }
+        if (isCaseSensitive(name)) {
+            // Don't upper case if in quotes
+            return name.substring(1, name.length()-1);
+        }
+        return name.toUpperCase();
+    }
+
+    public static boolean isCaseSensitive(String name) {
+        return name.length() > 0 && name.charAt(0)=='"';
+    }
+    
+    public static <T> List<T> concat(List<T> l1, List<T> l2) {
+        int size1 = l1.size();
+        if (size1 == 0) {
+            return l2;
+        }
+        int size2 = l2.size();
+        if (size2 == 0) {
+            return l1;
+        }
+        List<T> l3 = new ArrayList<T>(size1 + size2);
+        l3.addAll(l1);
+        l3.addAll(l2);
+        return l3;
+    }
+
+    public static byte[] getSequenceKey(byte[] tenantId, byte[] schemaName, byte[] sequenceName) {
+        return getTableKey(tenantId, schemaName, sequenceName);
+    }
+
+    public static byte[] getSequenceKey(String tenantId, String schemaName, String sequenceName) {
+        return getTableKey(tenantId, schemaName, sequenceName);
+    }
+
+    /**
+     * Get the key used in the Phoenix metadata row for a table definition
+     * @param schemaName
+     * @param tableName
+     */
+    public static byte[] getTableKey(byte[] tenantId, byte[] schemaName, byte[] tableName) {
+        return ByteUtil.concat(tenantId, QueryConstants.SEPARATOR_BYTE_ARRAY, schemaName, QueryConstants.SEPARATOR_BYTE_ARRAY, tableName);
+    }
+
+    public static byte[] getTableKey(String tenantId, String schemaName, String tableName) {
+        return ByteUtil.concat(tenantId == null  ? ByteUtil.EMPTY_BYTE_ARRAY : Bytes.toBytes(tenantId), QueryConstants.SEPARATOR_BYTE_ARRAY, schemaName == null ? ByteUtil.EMPTY_BYTE_ARRAY : Bytes.toBytes(schemaName), QueryConstants.SEPARATOR_BYTE_ARRAY, Bytes.toBytes(tableName));
+    }
+
+    public static String getTableName(String schemaName, String tableName) {
+        return getName(schemaName,tableName);
+    }
+
+    private static String getName(String optionalQualifier, String name) {
+        if (optionalQualifier == null || optionalQualifier.isEmpty()) {
+            return name;
+        }
+        return optionalQualifier + QueryConstants.NAME_SEPARATOR + name;
+    }
+
+    public static String getTableName(byte[] schemaName, byte[] tableName) {
+        return getName(schemaName, tableName);
+    }
+
+    public static String getColumnDisplayName(byte[] cf, byte[] cq) {
+        return getName(cf == null || cf.length == 0 || Bytes.compareTo(cf, QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES) == 0 ? ByteUtil.EMPTY_BYTE_ARRAY : cf, cq);
+    }
+
+    public static String getColumnDisplayName(String cf, String cq) {
+        return getName(cf == null || cf.isEmpty() || QueryConstants.DEFAULT_COLUMN_FAMILY.equals(cf) ? null : cf, cq);
+    }
+
+    public static String getMetaDataEntityName(String schemaName, String tableName, String familyName, String columnName) {
+        if ((schemaName == null || schemaName.isEmpty()) && (tableName == null || tableName.isEmpty())) {
+            return getName(familyName, columnName);
+        }
+        if ((familyName == null || familyName.isEmpty()) && (columnName == null || columnName.isEmpty())) {
+            return getName(schemaName, tableName);
+        }
+        return getName(getName(schemaName, tableName), getName(familyName, columnName));
+    }
+
+    public static String getColumnName(String familyName, String columnName) {
+        return getName(familyName, columnName);
+    }
+
+    public static byte[] getTableNameAsBytes(String schemaName, String tableName) {
+        if (schemaName == null || schemaName.length() == 0) {
+            return StringUtil.toBytes(tableName);
+        }
+        return getTableNameAsBytes(StringUtil.toBytes(schemaName),StringUtil.toBytes(tableName));
+    }
+
+    public static byte[] getTableNameAsBytes(byte[] schemaName, byte[] tableName) {
+        return getNameAsBytes(schemaName, tableName);
+    }
+
+    private static byte[] getNameAsBytes(byte[] nameOne, byte[] nameTwo) {
+        if (nameOne == null || nameOne.length == 0) {
+            return nameTwo;
+        } else if ((nameTwo == null || nameTwo.length == 0)) {
+            return nameOne;
+        } else {
+            return ByteUtil.concat(nameOne, QueryConstants.NAME_SEPARATOR_BYTES, nameTwo);
+        }
+    }
+
+    public static String getName(byte[] nameOne, byte[] nameTwo) {
+        return Bytes.toString(getNameAsBytes(nameOne,nameTwo));
+    }
+
+    public static int getVarCharLength(byte[] buf, int keyOffset, int maxLength) {
+        return getVarCharLength(buf, keyOffset, maxLength, 1);
+    }
+
+    public static int getVarCharLength(byte[] buf, int keyOffset, int maxLength, int skipCount) {
+        int length = 0;
+        for (int i=0; i<skipCount; i++) {
+            while (length < maxLength && buf[keyOffset+length] != QueryConstants.SEPARATOR_BYTE) {
+                length++;
+            }
+            if (i != skipCount-1) { // skip over the separator if it's not the last one.
+                length++;
+            }
+        }
+        return length;
+    }
+
+    public static int getVarChars(byte[] rowKey, byte[][] rowKeyMetadata) {
+        return getVarChars(rowKey, 0, rowKey.length, 0, rowKeyMetadata);
+    }
+    
+    public static int getVarChars(byte[] rowKey, int colMetaDataLength, byte[][] colMetaData) {
+        return getVarChars(rowKey, 0, rowKey.length, 0, colMetaDataLength, colMetaData);
+    }
+    
+    public static int getVarChars(byte[] rowKey, int keyOffset, int keyLength, int colMetaDataOffset, byte[][] colMetaData) {
+        return getVarChars(rowKey, keyOffset, keyLength, colMetaDataOffset, colMetaData.length, colMetaData);
+    }
+    
+    public static int getVarChars(byte[] rowKey, int keyOffset, int keyLength, int colMetaDataOffset, int colMetaDataLength, byte[][] colMetaData) {
+        int i, offset = keyOffset;
+        for (i = colMetaDataOffset; i < colMetaDataLength && keyLength > 0; i++) {
+            int length = getVarCharLength(rowKey, offset, keyLength);
+            byte[] b = new byte[length];
+            System.arraycopy(rowKey, offset, b, 0, length);
+            offset += length + 1;
+            keyLength -= length + 1;
+            colMetaData[i] = b;
+        }
+        return i;
+    }
+    
+    public static String findExistingColumn(PTable table, List<PColumn> columns) {
+        for (PColumn column : columns) {
+            PName familyName = column.getFamilyName();
+            if (familyName == null) {
+                try {
+                    return table.getPKColumn(column.getName().getString()).getName().getString();
+                } catch (ColumnNotFoundException e) {
+                    continue;
+                }
+            } else {
+                try {
+                    return table.getColumnFamily(familyName.getString()).getColumn(column.getName().getString()).getName().getString();
+                } catch (ColumnFamilyNotFoundException e) {
+                    continue; // Shouldn't happen
+                } catch (ColumnNotFoundException e) {
+                    continue;
+                }
+            }
+        }
+        return null;
+    }
+
+    public static String toString(byte[][] values) {
+        if (values == null) {
+            return "null";
+        }
+        StringBuilder buf = new StringBuilder("[");
+        for (byte[] value : values) {
+            buf.append(Bytes.toStringBinary(value));
+            buf.append(',');
+        }
+        buf.setCharAt(buf.length()-1, ']');
+        return buf.toString();
+    }
+
+    public static String toString(PDataType type, byte[] value) {
+        boolean isString = type.isCoercibleTo(PDataType.VARCHAR);
+        return isString ? ("'" + type.toObject(value).toString() + "'") : type.toObject(value).toString();
+    }
+
+    public static byte[] getEmptyColumnFamily(List<PColumnFamily> families) {
+        return families.isEmpty() ? QueryConstants.EMPTY_COLUMN_BYTES : families.get(0).getName().getBytes();
+    }
+
+    public static ImmutableBytesPtr getEmptyColumnFamilyPtr(List<PColumnFamily> families) {
+        return families.isEmpty() ? QueryConstants.EMPTY_COLUMN_BYTES_PTR : families.get(0)
+                .getName().getBytesPtr();
+    }
+
+    public static boolean isMetaTable(byte[] tableName) {
+        return Bytes.compareTo(tableName, TYPE_TABLE_NAME_BYTES) == 0;
+    }
+    
+    public static boolean isSequenceTable(byte[] tableName) {
+        return Bytes.compareTo(tableName, PhoenixDatabaseMetaData.SEQUENCE_TABLE_NAME_BYTES) == 0;
+    }
+
+    public static boolean isMetaTable(PTable table) {
+        return PhoenixDatabaseMetaData.TYPE_SCHEMA.equals(table.getSchemaName().getString()) && PhoenixDatabaseMetaData.TYPE_TABLE.equals(table.getTableName().getString());
+    }
+    
+    public static boolean isMetaTable(byte[] schemaName, byte[] tableName) {
+        return Bytes.compareTo(schemaName, PhoenixDatabaseMetaData.TYPE_SCHEMA_BYTES) == 0 && Bytes.compareTo(tableName, PhoenixDatabaseMetaData.TYPE_TABLE_BYTES) == 0;
+    }
+    
+    public static boolean isMetaTable(String schemaName, String tableName) {
+        return PhoenixDatabaseMetaData.TYPE_SCHEMA.equals(schemaName) && PhoenixDatabaseMetaData.TYPE_TABLE.equals(tableName);
+    }
+
+    // Given the splits and the rowKeySchema, find out the keys that 
+    public static byte[][] processSplits(byte[][] splits, LinkedHashSet<PColumn> pkColumns, Integer saltBucketNum, boolean defaultRowKeyOrder) throws SQLException {
+        // FIXME: shouldn't this return if splits.length == 0?
+        if (splits == null) return null;
+        // We do not accept user specified splits if the table is salted and we specify defaultRowKeyOrder. In this case,
+        // throw an exception.
+        if (splits.length > 0 && saltBucketNum != null && defaultRowKeyOrder) {
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode.NO_SPLITS_ON_SALTED_TABLE).build().buildException();
+        }
+        // If the splits are not specified and table is salted, pre-split the table. 
+        if (splits.length == 0 && saltBucketNum != null) {
+            splits = SaltingUtil.getSalteByteSplitPoints(saltBucketNum);
+        }
+        byte[][] newSplits = new byte[splits.length][];
+        for (int i=0; i<splits.length; i++) {
+            newSplits[i] = processSplit(splits[i], pkColumns); 
+        }
+        return newSplits;
+    }
+
+    // Go through each slot in the schema and try match it with the split byte array. If the split
+    // does not confer to the schema, extends its length to match the schema.
+    private static byte[] processSplit(byte[] split, LinkedHashSet<PColumn> pkColumns) {
+        int pos = 0, offset = 0, maxOffset = split.length;
+        Iterator<PColumn> iterator = pkColumns.iterator();
+        while (pos < pkColumns.size()) {
+            PColumn column = iterator.next();
+            if (column.getDataType().isFixedWidth()) { // Fixed width
+                int length = column.getByteSize();
+                if (maxOffset - offset < length) {
+                    // The split truncates the field. Fill in the rest of the part and any fields that
+                    // are missing after this field.
+                    int fillInLength = length - (maxOffset - offset);
+                    fillInLength += estimatePartLength(pos + 1, iterator);
+                    return ByteUtil.fillKey(split, split.length + fillInLength);
+                }
+                // Account for this field, move to next position;
+                offset += length;
+                pos++;
+            } else { // Variable length
+                // If we are the last slot, then we are done. Nothing needs to be filled in.
+                if (pos == pkColumns.size() - 1) {
+                    break;
+                }
+                while (offset < maxOffset && split[offset] != QueryConstants.SEPARATOR_BYTE) {
+                    offset++;
+                }
+                if (offset == maxOffset) {
+                    // The var-length field does not end with a separator and it's not the last field.
+                    int fillInLength = 1; // SEPARATOR byte for the current var-length slot.
+                    fillInLength += estimatePartLength(pos + 1, iterator);
+                    return ByteUtil.fillKey(split, split.length + fillInLength);
+                }
+                // Move to the next position;
+                offset += 1; // skip separator;
+                pos++;
+            }
+        }
+        return split;
+    }
+
+    // Estimate the key length after pos slot for schema.
+    private static int estimatePartLength(int pos, Iterator<PColumn> iterator) {
+        int length = 0;
+        while (iterator.hasNext()) {
+            PColumn column = iterator.next();
+            if (column.getDataType().isFixedWidth()) {
+                length += column.getByteSize();
+            } else {
+                length += 1; // SEPARATOR byte.
+            }
+        }
+        return length;
+    }
+    
+    public static final String UPGRADE_TO_2_0 = "UpgradeTo20";
+    public static final String UPGRADE_TO_2_1 = "UpgradeTo21";
+
+    public static String getEscapedTableName(String schemaName, String tableName) {
+        if (schemaName == null || schemaName.length() == 0) {
+            return "\"" + tableName + "\"";
+        }
+        return "\"" + schemaName + "\"." + "\"" + tableName + "\"";
+    }
+
+    protected static PhoenixConnection addMetaDataColumn(PhoenixConnection conn, long scn, String columnDef) throws SQLException {
+        String url = conn.getURL();
+        Properties props = conn.getClientInfo();
+        PMetaData metaData = conn.getPMetaData();
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(scn));
+        PhoenixConnection metaConnection = null;
+
+        Statement stmt = null;
+        try {
+            metaConnection = new PhoenixConnection(conn.getQueryServices(), url, props, metaData);
+            try {
+                stmt = metaConnection.createStatement();
+                stmt.executeUpdate("ALTER TABLE SYSTEM.\"TABLE\" ADD IF NOT EXISTS " + columnDef);
+                return metaConnection;
+            } finally {
+                if(stmt != null) {
+                    stmt.close();
+                }
+            }
+        } finally {
+            if(metaConnection != null) {
+                metaConnection.close();
+            }
+        }
+    }
+    
+    public static boolean columnExists(PTable table, String columnName) {
+        try {
+            table.getColumn(columnName);
+            return true;
+        } catch (ColumnNotFoundException e) {
+            return false;
+        } catch (AmbiguousColumnException e) {
+            return true;
+        }
+    }
+    
+    public static String getSchemaNameFromFullName(String tableName) {
+        int index = tableName.indexOf(QueryConstants.NAME_SEPARATOR);
+        if (index < 0) {
+            return ""; 
+        }
+        return tableName.substring(0, index);
+    }
+    
+    public static String getTableNameFromFullName(String tableName) {
+        int index = tableName.indexOf(QueryConstants.NAME_SEPARATOR);
+        if (index < 0) {
+            return tableName; 
+        }
+        return tableName.substring(index+1, tableName.length());
+    }
+
+    public static byte[] getTableKeyFromFullName(String fullTableName) {
+        int index = fullTableName.indexOf(QueryConstants.NAME_SEPARATOR);
+        if (index < 0) {
+            return getTableKey(null, null, fullTableName); 
+        }
+        String schemaName = fullTableName.substring(0, index);
+        String tableName = fullTableName.substring(index+1);
+        return getTableKey(null, schemaName, tableName); 
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/util/ServerUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/ServerUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/ServerUtil.java
new file mode 100644
index 0000000..819bcc4
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/ServerUtil.java
@@ -0,0 +1,123 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.io.IOException;
+import java.sql.SQLException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+
+import org.apache.phoenix.exception.PhoenixIOException;
+import org.apache.phoenix.exception.SQLExceptionCode;
+
+
+public class ServerUtil {
+
+    private static final String FORMAT = "ERROR %d (%s): %s";
+    private static final Pattern PATTERN = Pattern.compile("ERROR (\\d+) \\((\\w+)\\): (.*)");
+    private static final Map<Class<? extends Exception>, SQLExceptionCode> errorcodeMap
+        = new HashMap<Class<? extends Exception>, SQLExceptionCode>();
+    static {
+        // Map a normal exception into a corresponding SQLException.
+        errorcodeMap.put(ArithmeticException.class, SQLExceptionCode.SERVER_ARITHMETIC_ERROR);
+    }
+
+    public static void throwIOException(String msg, Throwable t) throws IOException {
+        // First unwrap SQLExceptions if it's root cause is an IOException.
+        if (t instanceof SQLException) {
+            Throwable cause = t.getCause();
+            if (cause instanceof IOException) {
+                t = cause;
+            }
+        }
+        // Throw immediately if DoNotRetryIOException
+        if (t instanceof DoNotRetryIOException) {
+            throw (DoNotRetryIOException)t;
+        } else if (t instanceof IOException) {
+            // If the IOException does not wrap any exception, then bubble it up.
+            Throwable cause = t.getCause();
+            if (cause == null || cause instanceof IOException) {
+                throw (IOException)t;
+            }
+            // Else assume it's been wrapped, so throw as DoNotRetryIOException to prevent client hanging while retrying
+            throw new DoNotRetryIOException(t.getMessage(), cause);
+        } else if (t instanceof SQLException) {
+            // If it's already an SQLException, construct an error message so we can parse and reconstruct on the client side.
+            throw new DoNotRetryIOException(constructSQLErrorMessage((SQLException) t, msg), t);
+        } else {
+            // Not a DoNotRetryIOException, IOException or SQLException. Map the exception type to a general SQLException 
+            // and construct the error message so it can be reconstruct on the client side.
+            //
+            // If no mapping exists, rethrow it as a generic exception.
+            SQLExceptionCode code = errorcodeMap.get(t.getClass());
+            if (code == null) {
+                throw new DoNotRetryIOException(msg + ": " + t.getMessage(), t);
+            } else {
+                throw new DoNotRetryIOException(constructSQLErrorMessage(code, t, msg), t);
+            }
+        }
+    }
+
+    private static String constructSQLErrorMessage(SQLExceptionCode code, Throwable e, String message) {
+        return constructSQLErrorMessage(code.getErrorCode(), code.getSQLState(), code.getMessage() + " " + e.getMessage() + " " + message);
+    }
+
+    private static String constructSQLErrorMessage(SQLException e, String message) {
+        return constructSQLErrorMessage(e.getErrorCode(), e.getSQLState(), e.getMessage() + " " + message);
+    }
+
+    private static String constructSQLErrorMessage(int errorCode, String SQLState, String message) {
+        return String.format(FORMAT, errorCode, SQLState, message);
+    }
+
+    public static SQLException parseServerException(Throwable t) {
+        SQLException e = parseServerExceptionOrNull(t);
+        if (e != null) {
+            return e;
+        }
+        return new PhoenixIOException(t);
+    }
+    
+    public static SQLException parseServerExceptionOrNull(Throwable t) {
+        while (t.getCause() != null) {
+            t = t.getCause();
+        }
+        return parseRemoteException(t);
+    }
+
+    private static SQLException parseRemoteException(Throwable t) {
+        	String message = t.getLocalizedMessage();
+        	if (message != null) {
+            // If the message matches the standard pattern, recover the SQLException and throw it.
+            Matcher matcher = PATTERN.matcher(t.getLocalizedMessage());
+            if (matcher.find()) {
+                int errorCode = Integer.parseInt(matcher.group(1));
+                String sqlState = matcher.group(2);
+                return new SQLException(matcher.group(), sqlState, errorCode, t);
+            }
+        	}
+        return null;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/util/SizedUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/SizedUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/SizedUtil.java
new file mode 100644
index 0000000..62f0816
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/SizedUtil.java
@@ -0,0 +1,63 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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;
+
+
+/**
+ * Utilities for computing an object's size.  All size measurements are in bytes.
+ * Note, all of the sizes here, but especially OBJECT_SIZE and ARRAY_SIZE are estimates and will
+ * depend on the JVM itself (and which JVM, 64bit vs. 32bit, etc).
+ * The current values are based on:
+ * Java HotSpot(TM) 64-Bit Server VM/14.2-b01
+ * 
+ * (Uncomment out and run the main w/ appropriate object to test)
+ * Also, see this link:
+ * https://sites.google.com/a/salesforce.com/development/Home/old-wiki-home-page/i-wish-i-knew#TOC-How-to-measure-the-size-of-a-Java-O
+ * For another way to measure.
+ */
+public class SizedUtil {
+    public static final int POINTER_SIZE = 8; // 64 bit jvm.
+    public static final int OBJECT_SIZE = 16; // measured, see class comment.
+    public static final int ARRAY_SIZE = 24; // measured, see class comment.
+    public static final int CHAR_SIZE = 2;
+    public static final int INT_SIZE = 4;
+    public static final int LONG_SIZE = 8;
+    
+    public static final int MAP_ENTRY_SIZE = OBJECT_SIZE + 3 * POINTER_SIZE + INT_SIZE;
+    public static final int IMMUTABLE_BYTES_WRITABLE_SIZE = OBJECT_SIZE + INT_SIZE * 2 + ARRAY_SIZE;
+    public static final int IMMUTABLE_BYTES_PTR_SIZE = IMMUTABLE_BYTES_WRITABLE_SIZE + INT_SIZE;// Extra is an int field which caches hashcode.
+    public static final int KEY_VALUE_SIZE = 2 * INT_SIZE + LONG_SIZE + 2 * ARRAY_SIZE;
+    public static final int RESULT_SIZE = OBJECT_SIZE +  3 * POINTER_SIZE + IMMUTABLE_BYTES_WRITABLE_SIZE;
+    public static final int INT_OBJECT_SIZE = INT_SIZE + OBJECT_SIZE;
+    public static final int LONG_OBJECT_SIZE = LONG_SIZE + OBJECT_SIZE;
+    public static final int BIG_DECIMAL_SIZE = 
+        OBJECT_SIZE + 2 * INT_SIZE + LONG_SIZE + 2 * POINTER_SIZE +
+        OBJECT_SIZE /* BigInteger */ + 5 * INT_SIZE + ARRAY_SIZE /*mag[]*/ + 2 * INT_SIZE /* est mag[2] */;
+
+    private SizedUtil() {
+    }
+    
+    public static int sizeOfMap(int nRows, int keySize, int valueSize) {
+        return nRows * (
+                SizedUtil.MAP_ENTRY_SIZE + // entry
+                keySize + // key size
+                valueSize); // value size
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/util/StringUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/StringUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/StringUtil.java
new file mode 100644
index 0000000..1e9d961
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/StringUtil.java
@@ -0,0 +1,246 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.io.UnsupportedEncodingException;
+import java.util.Arrays;
+
+import org.apache.hadoop.hbase.util.Bytes;
+
+import org.apache.phoenix.schema.ColumnModifier;
+
+
+public class StringUtil {
+    public static final String EMPTY_STRING = "";
+    // Masks to determine how many bytes are in each character
+    // From http://tools.ietf.org/html/rfc3629#section-3
+    public static final byte SPACE_UTF8 = 0x20;
+    private static final int BYTES_1_MASK = 0xFF << 7; // 0xxxxxxx is a single byte char
+    private static final int BYTES_2_MASK = 0xFF << 5; // 110xxxxx is a double byte char
+    private static final int BYTES_3_MASK = 0xFF << 4; // 1110xxxx is a triple byte char
+    private static final int BYTES_4_MASK = 0xFF << 3; // 11110xxx is a quadruple byte char
+    
+    public static final byte[] MOD_SPACE_UTF8 = new byte[ColumnModifier.values().length];
+    static {
+        for (ColumnModifier columnModifier : ColumnModifier.values()) {
+            MOD_SPACE_UTF8[columnModifier.ordinal()] = columnModifier.apply(new byte[] {SPACE_UTF8}, 0, new byte[1], 0, 1)[0];
+        }
+    }
+
+    public final static char SINGLE_CHAR_WILDCARD = '?';
+    public final static char SINGLE_CHAR_LIKE = '_';
+    public final static char MULTI_CHAR_WILDCARD = '*';
+    public final static char MULTI_CHAR_LIKE = '%';
+    public final static String[] LIKE_ESCAPE_SEQS = new String[]{"\\"+SINGLE_CHAR_LIKE, "\\"+MULTI_CHAR_LIKE};
+    public final static String[] LIKE_UNESCAPED_SEQS = new String[]{""+SINGLE_CHAR_LIKE, ""+MULTI_CHAR_LIKE};
+    
+
+    private StringUtil() {
+    }
+
+    /** Replace instances of character ch in String value with String replacement */
+    public static String replaceChar(String value, char ch, CharSequence replacement) {
+        if (value == null)
+            return null;
+        int i = value.indexOf(ch);
+        if (i == -1)
+            return value; // nothing to do
+
+        // we've got at least one character to replace
+        StringBuilder buf = new StringBuilder(value.length() + 16); // some extra space
+        int j = 0;
+        while (i != -1) {
+            buf.append(value, j, i).append(replacement);
+            j = i + 1;
+            i = value.indexOf(ch, j);
+        }
+        if (j < value.length())
+            buf.append(value, j, value.length());
+        return buf.toString();
+    }
+
+    /**
+     * @return the replacement of all occurrences of src[i] with target[i] in s. Src and target are not regex's so this
+     *         uses simple searching with indexOf()
+     */
+    public static String replace(String s, String[] src, String[] target) {
+        assert src != null && target != null && src.length > 0 && src.length == target.length;
+        if (src.length == 1 && src[0].length() == 1) {
+            return replaceChar(s, src[0].charAt(0), target[0]);
+        }
+        if (s == null)
+            return null;
+        StringBuilder sb = new StringBuilder(s.length());
+        int pos = 0;
+        int limit = s.length();
+        int lastMatch = 0;
+        while (pos < limit) {
+            boolean matched = false;
+            for (int i = 0; i < src.length; i++) {
+                if (s.startsWith(src[i], pos) && src[i].length() > 0) {
+                    // we found a matching pattern - append the acculumation plus the replacement
+                    sb.append(s.substring(lastMatch, pos)).append(target[i]);
+                    pos += src[i].length();
+                    lastMatch = pos;
+                    matched = true;
+                    break;
+                }
+            }
+            if (!matched) {
+                // we didn't match any patterns, so move forward 1 character
+                pos++;
+            }
+        }
+        // see if we found any matches
+        if (lastMatch == 0) {
+            // we didn't match anything, so return the source string
+            return s;
+        }
+        
+        // apppend the trailing portion
+        sb.append(s.substring(lastMatch));
+        
+        return sb.toString();
+    }
+
+    public static int getBytesInChar(byte b, ColumnModifier columnModifier) {
+        if (columnModifier != null) {
+            b = columnModifier.apply(b);
+        }
+        int c = b & 0xff;
+        if ((c & BYTES_1_MASK) == 0)
+            return 1;
+        if ((c & BYTES_2_MASK) == 0xC0)
+            return 2;
+        if ((c & BYTES_3_MASK) == 0xE0)
+            return 3;
+        if ((c & BYTES_4_MASK) == 0xF0)
+            return 4;
+        // Any thing else in the first byte is invalid
+        throw new RuntimeException("Undecodable byte: " + b);
+    }
+
+    public static int calculateUTF8Length(byte[] bytes, int offset, int length, ColumnModifier columnModifier) throws UnsupportedEncodingException {
+        int i = offset, endOffset = offset + length;
+        length = 0;
+        while (i < endOffset) {
+            int charLength = getBytesInChar(bytes[i], columnModifier);
+            i += charLength;
+            length++;
+        }
+        return length;
+    }
+
+    // Given an array of bytes containing encoding utf-8 encoded strings, the offset and a length
+    // parameter, return the actual index into the byte array which would represent a substring
+    // of <length> starting from the character at <offset>. We assume the <offset> is the start
+    // byte of an UTF-8 character.
+    public static int getByteLengthForUtf8SubStr(byte[] bytes, int offset, int length, ColumnModifier columnModifier) throws UnsupportedEncodingException {
+        int byteLength = 0;
+        while(length > 0 && offset + byteLength < bytes.length) {
+            int charLength = getBytesInChar(bytes[offset + byteLength], columnModifier);
+            byteLength += charLength;
+            length--;
+        }
+        return byteLength;
+    }
+
+    public static boolean hasMultiByteChars(String s) {
+        for (int i = 0; i < s.length(); i++) {
+            char c = s.charAt(i);
+            if (c > 0x007F) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    public static int getFirstNonBlankCharIdxFromStart(byte[] string, int offset, int length, ColumnModifier columnModifier) {
+        int i = offset;
+        byte space = columnModifier == null ? SPACE_UTF8 : MOD_SPACE_UTF8[columnModifier.ordinal()];
+        for ( ; i < offset + length; i++) {
+            if (string[i] != space) {
+                break;
+            }
+        }
+        return i;
+    }
+
+    public static int getFirstNonBlankCharIdxFromEnd(byte[] string, int offset, int length, ColumnModifier columnModifier) {
+        int i = offset + length - 1;
+        byte space = columnModifier == null ? SPACE_UTF8 : MOD_SPACE_UTF8[columnModifier.ordinal()];
+        for ( ; i >= offset; i--) {
+            if (string[i] != space) {
+                break;
+            }
+         }
+        return i;
+    }
+
+    // A toBytes function backed up HBase's utility function, but would accept null input, in which
+    // case it returns an empty byte array.
+    public static byte[] toBytes(String input) {
+        if (input == null) {
+            return ByteUtil.EMPTY_BYTE_ARRAY;
+        }
+        return Bytes.toBytes(input);
+    }
+
+    public static String escapeLike(String s) {
+        return replace(s, LIKE_UNESCAPED_SEQS, LIKE_ESCAPE_SEQS);
+    }
+
+    public static int getUnpaddedCharLength(byte[] b, int offset, int length, ColumnModifier columnModifier) {
+        return getFirstNonBlankCharIdxFromEnd(b, offset, length, columnModifier) - offset + 1;
+    }
+
+    public static byte[] padChar(byte[] value, int offset, int length, int paddedLength) {
+        byte[] key = new byte[paddedLength];
+        System.arraycopy(value,offset, key, 0, length);
+        Arrays.fill(key, length, paddedLength, SPACE_UTF8);
+        return key;
+    }
+
+    public static byte[] padChar(byte[] value, Integer byteSize) {
+        byte[] newValue = Arrays.copyOf(value, byteSize);
+        if (newValue.length > value.length) {
+            Arrays.fill(newValue, value.length, newValue.length, SPACE_UTF8);
+        }
+        return newValue;
+    }
+    
+    /**
+     * Lame - StringBuilder.equals is retarded.
+     * @param b1
+     * @param b2
+     * @return whether or not the two builders consist the same sequence of characters
+     */
+    public static boolean equals(StringBuilder b1, StringBuilder b2) {
+        if (b1.length() != b2.length()) {
+            return false;
+        }
+        for (int i = 0; i < b1.length(); i++) {
+            if (b1.charAt(i) != b2.charAt(i)) {
+                return false;
+            }
+        }
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/util/TrustedByteArrayOutputStream.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/TrustedByteArrayOutputStream.java b/phoenix-core/src/main/java/org/apache/phoenix/util/TrustedByteArrayOutputStream.java
new file mode 100644
index 0000000..f503cb3
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/TrustedByteArrayOutputStream.java
@@ -0,0 +1,56 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.io.ByteArrayOutputStream;
+import java.io.IOException;
+
+/**
+ * 
+ * Derived version of {@link java.io.ByteArrayOutputStream} that provides access
+ * to underlying byte array buffer so that it doesn't have to be copied
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class TrustedByteArrayOutputStream extends ByteArrayOutputStream {
+    public TrustedByteArrayOutputStream(int initialSize) {
+        super(initialSize);
+    }
+    public byte[] getBuffer() {
+        return buf;
+    }
+    @Override
+    public byte[] toByteArray() {
+        if (buf.length == size()) {
+            return buf;
+        }
+        return super.toByteArray();
+    }
+    @Override
+    public void write(byte[] b) {
+        try {
+            super.write(b);
+        } catch (IOException e) {
+            throw new RuntimeException(e); // Impossible
+        }
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/util/TupleUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/TupleUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/TupleUtil.java
new file mode 100644
index 0000000..9a8608a
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/TupleUtil.java
@@ -0,0 +1,155 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.query.QueryConstants.SINGLE_COLUMN;
+import static org.apache.phoenix.query.QueryConstants.SINGLE_COLUMN_FAMILY;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.WritableUtils;
+
+import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+
+/**
+ * 
+ * Utilities for Tuple
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class TupleUtil {
+    private TupleUtil() {
+    }
+    
+    public static boolean equals(Tuple t1, Tuple t2, ImmutableBytesWritable ptr) {
+        t1.getKey(ptr);
+        byte[] buf = ptr.get();
+        int offset = ptr.getOffset();
+        int length = ptr.getLength();
+        t2.getKey(ptr);
+        return Bytes.compareTo(buf, offset, length, ptr.get(), ptr.getOffset(), ptr.getLength()) == 0;
+    }
+    
+    public static int compare(Tuple t1, Tuple t2, ImmutableBytesWritable ptr) {
+        return compare(t1, t2, ptr, 0);
+    }
+    
+    public static int compare(Tuple t1, Tuple t2, ImmutableBytesWritable ptr, int keyOffset) {
+        t1.getKey(ptr);
+        byte[] buf = ptr.get();
+        int offset = ptr.getOffset() + keyOffset;
+        int length = ptr.getLength() - keyOffset;
+        t2.getKey(ptr);
+        return Bytes.compareTo(buf, offset, length, ptr.get(), ptr.getOffset() + keyOffset, ptr.getLength() - keyOffset);
+    }
+    
+    /**
+     * Set ptr to point to the value contained in the first KeyValue without
+     * exploding Result into KeyValue array.
+     * @param r
+     * @param ptr
+     */
+    public static void getAggregateValue(Tuple r, ImmutableBytesWritable ptr) {
+        if (r.size() == 1) {
+            KeyValue kv = r.getValue(0); // Just one KV for aggregation
+            if (Bytes.compareTo(SINGLE_COLUMN_FAMILY, 0, SINGLE_COLUMN_FAMILY.length, kv.getBuffer(), kv.getFamilyOffset(), kv.getFamilyLength()) == 0) {
+                if (Bytes.compareTo(SINGLE_COLUMN, 0, SINGLE_COLUMN.length, kv.getBuffer(), kv.getQualifierOffset(), kv.getQualifierLength()) == 0) {
+                    ptr.set(kv.getBuffer(), kv.getValueOffset(), kv.getValueLength());
+                    return;
+                }
+            }
+        }
+        throw new IllegalStateException("Expected single, aggregated KeyValue from coprocessor, but instead received " + r + ". Ensure aggregating coprocessors are loaded correctly on server");
+    }
+    
+    /** Concatenate results evaluated against a list of expressions
+     * 
+     * @param result the tuple for expression evaluation
+     * @param expressions
+     * @return the concatenated byte array as ImmutableBytesWritable
+     * @throws IOException
+     */
+    public static ImmutableBytesPtr getConcatenatedValue(Tuple result, List<Expression> expressions) throws IOException {
+        ImmutableBytesPtr value = new ImmutableBytesPtr(ByteUtil.EMPTY_BYTE_ARRAY);
+        Expression expression = expressions.get(0);
+        boolean evaluated = expression.evaluate(result, value);
+        
+        if (expressions.size() == 1) {
+            if (!evaluated) {
+                value.set(ByteUtil.EMPTY_BYTE_ARRAY);
+            }
+            return value;
+        } else {
+            TrustedByteArrayOutputStream output = new TrustedByteArrayOutputStream(value.getLength() * expressions.size());
+            try {
+                if (evaluated) {
+                    output.write(value.get(), value.getOffset(), value.getLength());
+                }
+                for (int i = 1; i < expressions.size(); i++) {
+                    if (!expression.getDataType().isFixedWidth()) {
+                        output.write(QueryConstants.SEPARATOR_BYTE);
+                    }
+                    expression = expressions.get(i);
+                    // TODO: should we track trailing null values and omit the separator bytes?
+                    if (expression.evaluate(result, value)) {
+                        output.write(value.get(), value.getOffset(), value.getLength());
+                    } else if (i < expressions.size()-1 && expression.getDataType().isFixedWidth()) {
+                        // This should never happen, because any non terminating nullable fixed width type (i.e. INT or LONG) is
+                        // converted to a variable length type (i.e. DECIMAL) to allow an empty byte array to represent null.
+                        throw new DoNotRetryIOException("Non terminating null value found for fixed width expression (" + expression + ") in row: " + result);
+                    }
+                }
+                byte[] outputBytes = output.getBuffer();
+                value.set(outputBytes, 0, output.size());
+                return value;
+            } finally {
+                output.close();
+            }
+        }
+    }
+    
+    public static int write(Tuple result, DataOutput out) throws IOException {
+        int size = 0;
+        for(int i = 0; i < result.size(); i++) {
+            KeyValue kv = result.getValue(i);
+            size += kv.getLength();
+            size += Bytes.SIZEOF_INT; // kv.getLength
+          }
+
+        WritableUtils.writeVInt(out, size);
+        for(int i = 0; i < result.size(); i++) {
+            KeyValue kv = result.getValue(i);
+            out.writeInt(kv.getLength());
+            out.write(kv.getBuffer(), kv.getOffset(), kv.getLength());
+          }
+        return size;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/overview.html
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/overview.html b/phoenix-core/src/main/java/overview.html
new file mode 100644
index 0000000..b58fa45
--- /dev/null
+++ b/phoenix-core/src/main/java/overview.html
@@ -0,0 +1,25 @@
+<html>
+<body>
+This document is for the JDBC implementation of Phoenix, targeting those who are interested in extending the level of SQL support.
+For users of Phoenix, the programmatic API is <a href="http://docs.oracle.com/javase/1.4.2/docs/api/java/sql/package-summary.html">JDBC</a>.
+For example, the following snippet demonstrates how to query a table:
+
+<pre>
+        Properties props = new Properties();
+        // Ensure the driver is on classpath
+        // Connect through Phoenix to the zookeeper quorum with a host name of myServer
+        Connection conn = DriverManager.getConnection("jdbc:phoenix:myServer", props);
+        try {
+            PreparedStatement statement = conn.prepareStatement("SELECT count(1) FROM product_metrics WHERE organization_id=?");
+            statement.setString(1, orgId);
+            ResultSet rs = statement.executeQuery();
+            if (rs.next()) {
+                System.out.println("Row count of orgId='" + orgId + "' is " + rs.getLong(1));
+            }
+        } finally {
+            conn.close();
+        }
+</pre>
+
+</body>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/resources/java.sql.Driver
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/resources/java.sql.Driver b/phoenix-core/src/main/resources/java.sql.Driver
new file mode 100644
index 0000000..fc51a04
--- /dev/null
+++ b/phoenix-core/src/main/resources/java.sql.Driver
@@ -0,0 +1 @@
+org.apache.phoenix.jdbc.PhoenixDriver

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/site/bin/merge.sh
----------------------------------------------------------------------
diff --git a/phoenix-core/src/site/bin/merge.sh b/phoenix-core/src/site/bin/merge.sh
new file mode 100755
index 0000000..5ac0641
--- /dev/null
+++ b/phoenix-core/src/site/bin/merge.sh
@@ -0,0 +1,10 @@
+current_dir=$(cd $(dirname $0);pwd)
+cd $current_dir
+SITE_TARGET="../../../target/site"
+java -jar merge.jar ../language_reference_source/index.html $SITE_TARGET/language/index.html
+java -jar merge.jar ../language_reference_source/functions.html $SITE_TARGET/language/functions.html
+java -jar merge.jar ../language_reference_source/datatypes.html $SITE_TARGET/language/datatypes.html
+cd $SITE_TARGET
+
+grep -rl class=\"nav-collapse\" . | xargs sed -i 's/class=\"nav-collapse\"/class=\"nav-collapse collapse\"/g';grep -rl class=\"active\" . | xargs sed -i 's/class=\"active\"/class=\"divider\"/g'
+grep -rl "dropdown active" . | xargs sed -i 's/dropdown active/dropdown/g'


[32/51] [partial] Initial commit of master branch from github

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/DoubleDivideExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/DoubleDivideExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/DoubleDivideExpression.java
new file mode 100644
index 0000000..561cf90
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/DoubleDivideExpression.java
@@ -0,0 +1,74 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+public class DoubleDivideExpression extends DivideExpression {
+
+    public DoubleDivideExpression() {
+    }
+
+    public DoubleDivideExpression(List<Expression> children) {
+        super(children);
+    }
+    
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        double result = 0.0;
+        for (int i = 0; i < children.size(); i++) {
+            Expression child = children.get(i);
+            if (!child.evaluate(tuple, ptr)) {
+                return false;
+            }
+            if (ptr.getLength() == 0) {
+                return true;
+            }
+            double childvalue = child.getDataType().getCodec()
+                    .decodeDouble(ptr, child.getColumnModifier());
+            if (!Double.isNaN(childvalue)
+                    && childvalue != Double.NEGATIVE_INFINITY
+                    && childvalue != Double.POSITIVE_INFINITY) {
+                if (i == 0) {
+                    result = childvalue;
+                } else {
+                    result /= childvalue;
+                }
+            } else {
+                return false;
+            }
+        }
+        byte[] resultPtr = new byte[getDataType().getByteSize()];
+        ptr.set(resultPtr);
+        getDataType().getCodec().encodeDouble(result, ptr);
+        return true;
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return PDataType.DOUBLE;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/DoubleMultiplyExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/DoubleMultiplyExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/DoubleMultiplyExpression.java
new file mode 100644
index 0000000..b9e8165
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/DoubleMultiplyExpression.java
@@ -0,0 +1,70 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+public class DoubleMultiplyExpression extends MultiplyExpression {
+
+    public DoubleMultiplyExpression() {
+    }
+
+    public DoubleMultiplyExpression(List<Expression> children) {
+        super(children);
+    }
+    
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        double result = 1.0;
+        for (int i = 0; i < children.size(); i++) {
+            Expression child = children.get(i);
+            if (!child.evaluate(tuple, ptr)) {
+                return false;
+            }
+            if (ptr.getLength() == 0) {
+                return true;
+            }
+            double childvalue = child.getDataType().getCodec()
+                    .decodeDouble(ptr, child.getColumnModifier());
+            if (!Double.isNaN(childvalue)
+                    && childvalue != Double.NEGATIVE_INFINITY
+                    && childvalue != Double.POSITIVE_INFINITY) {
+                result *= childvalue;
+            } else {
+                return false;
+            }
+        }
+        byte[] resultPtr = new byte[getDataType().getByteSize()];
+        ptr.set(resultPtr);
+        getDataType().getCodec().encodeDouble(result, ptr);
+        return true;
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return PDataType.DOUBLE;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/DoubleSubtractExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/DoubleSubtractExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/DoubleSubtractExpression.java
new file mode 100644
index 0000000..8752c15
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/DoubleSubtractExpression.java
@@ -0,0 +1,74 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+public class DoubleSubtractExpression extends SubtractExpression {
+
+    public DoubleSubtractExpression() {
+    }
+
+    public DoubleSubtractExpression(List<Expression> children) {
+        super(children);
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        double result = 0.0;
+        for (int i = 0; i < children.size(); i++) {
+            Expression child = children.get(i);
+            if (!child.evaluate(tuple, ptr)) {
+                return false;
+            }
+            if (ptr.getLength() == 0) {
+                return true;
+            }
+            double childvalue = child.getDataType().getCodec()
+                    .decodeDouble(ptr, child.getColumnModifier());
+            if (!Double.isNaN(childvalue)
+                    && childvalue != Double.NEGATIVE_INFINITY
+                    && childvalue != Double.POSITIVE_INFINITY) {
+                if (i == 0) {
+                    result = childvalue;
+                } else {
+                    result -= childvalue;
+                }
+            } else {
+                return false;
+            }
+        }
+        byte[] resultPtr = new byte[getDataType().getByteSize()];
+        ptr.set(resultPtr);
+        getDataType().getCodec().encodeDouble(result, ptr);
+        return true;
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return PDataType.DOUBLE;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/Expression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/Expression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/Expression.java
new file mode 100644
index 0000000..ddced6a
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/Expression.java
@@ -0,0 +1,82 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.io.Writable;
+
+import org.apache.phoenix.expression.visitor.ExpressionVisitor;
+import org.apache.phoenix.schema.PDatum;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+
+/**
+ * 
+ * Interface for general expression evaluation
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public interface Expression extends PDatum, Writable {
+    /**
+     * Access the value by setting a pointer to it (as opposed to making
+     * a copy of it which can be expensive)
+     * @param tuple Single row result during scan iteration
+     * @param ptr Pointer to byte value being accessed
+     * @return true if the expression could be evaluated (i.e. ptr was set)
+     * and false otherwise
+     */
+    boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr);
+    
+    /**
+     * Means of traversing expression tree through visitor.
+     * @param visitor
+     */
+    <T> T accept(ExpressionVisitor<T> visitor);
+    
+    /**
+     * @return the child expressions
+     */
+    List<Expression> getChildren();
+    
+    /**
+     * Resets the state of a expression back to its initial state and
+     * enables the expession to be evaluated incrementally (which
+     * occurs during filter evaluation where we see one key value at
+     * a time; it's possible to evaluate immediately rather than
+     * wait until all key values have been seen). Note that when
+     * evaluating incrementally, you must call this method before
+     * processing a new row.
+     */
+    void reset();
+    
+    /**
+     * @return true if the expression can be evaluated without
+     * requiring a row Tuple and false otherwise.
+     */
+    boolean isStateless();
+    /**
+     * @return true if the expression returns the same output every
+     * time given the same input.
+     */
+    boolean isDeterministic();
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..c462878
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
@@ -0,0 +1,192 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.util.Map;
+
+import org.apache.phoenix.expression.function.ArrayIndexFunction;
+import org.apache.phoenix.expression.function.ArrayLengthFunction;
+import org.apache.phoenix.expression.function.CeilDateExpression;
+import org.apache.phoenix.expression.function.CeilDecimalExpression;
+import org.apache.phoenix.expression.function.CeilFunction;
+import org.apache.phoenix.expression.function.CeilTimestampExpression;
+import org.apache.phoenix.expression.function.CoalesceFunction;
+import org.apache.phoenix.expression.function.CountAggregateFunction;
+import org.apache.phoenix.expression.function.DistinctCountAggregateFunction;
+import org.apache.phoenix.expression.function.FloorDateExpression;
+import org.apache.phoenix.expression.function.FloorDecimalExpression;
+import org.apache.phoenix.expression.function.FloorFunction;
+import org.apache.phoenix.expression.function.IndexStateNameFunction;
+import org.apache.phoenix.expression.function.InvertFunction;
+import org.apache.phoenix.expression.function.LTrimFunction;
+import org.apache.phoenix.expression.function.LengthFunction;
+import org.apache.phoenix.expression.function.MaxAggregateFunction;
+import org.apache.phoenix.expression.function.MinAggregateFunction;
+import org.apache.phoenix.expression.function.PercentRankAggregateFunction;
+import org.apache.phoenix.expression.function.PercentileContAggregateFunction;
+import org.apache.phoenix.expression.function.PercentileDiscAggregateFunction;
+import org.apache.phoenix.expression.function.RTrimFunction;
+import org.apache.phoenix.expression.function.RegexpReplaceFunction;
+import org.apache.phoenix.expression.function.RegexpSubstrFunction;
+import org.apache.phoenix.expression.function.ReverseFunction;
+import org.apache.phoenix.expression.function.RoundDateExpression;
+import org.apache.phoenix.expression.function.RoundDecimalExpression;
+import org.apache.phoenix.expression.function.RoundFunction;
+import org.apache.phoenix.expression.function.RoundTimestampExpression;
+import org.apache.phoenix.expression.function.SQLTableTypeFunction;
+import org.apache.phoenix.expression.function.SQLViewTypeFunction;
+import org.apache.phoenix.expression.function.SqlTypeNameFunction;
+import org.apache.phoenix.expression.function.StddevPopFunction;
+import org.apache.phoenix.expression.function.StddevSampFunction;
+import org.apache.phoenix.expression.function.SubstrFunction;
+import org.apache.phoenix.expression.function.SumAggregateFunction;
+import org.apache.phoenix.expression.function.ToCharFunction;
+import org.apache.phoenix.expression.function.ToDateFunction;
+import org.apache.phoenix.expression.function.ToNumberFunction;
+import org.apache.phoenix.expression.function.TrimFunction;
+import org.apache.phoenix.expression.function.TruncFunction;
+
+import com.google.common.collect.Maps;
+
+/**
+ * 
+ * Enumeration of all Expression types that may be evaluated on the server-side.
+ * Used during serialization and deserialization to pass Expression between client
+ * and server.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public enum ExpressionType {
+    ReverseFunction(ReverseFunction.class),
+    RowKey(RowKeyColumnExpression.class),
+    KeyValue(KeyValueColumnExpression.class),
+    LiteralValue(LiteralExpression.class),
+    RoundFunction(RoundFunction.class),
+    FloorFunction(FloorFunction.class),
+    CeilFunction(CeilFunction.class),
+    RoundDateExpression(RoundDateExpression.class),
+    FloorDateExpression(FloorDateExpression.class),
+    CeilDateExpression(CeilDateExpression.class),
+    RoundTimestampExpression(RoundTimestampExpression.class),
+    CeilTimestampExpression(CeilTimestampExpression.class),
+    RoundDecimalExpression(RoundDecimalExpression.class),
+    FloorDecimalExpression(FloorDecimalExpression.class),
+    CeilDecimalExpression(CeilDecimalExpression.class),
+    TruncFunction(TruncFunction.class),
+    ToDateFunction(ToDateFunction.class),
+    ToCharFunction(ToCharFunction.class),
+    ToNumberFunction(ToNumberFunction.class),
+    CoerceFunction(CoerceExpression.class),
+    SubstrFunction(SubstrFunction.class),
+    AndExpression(AndExpression.class),
+    OrExpression(OrExpression.class),
+    ComparisonExpression(ComparisonExpression.class),
+    CountAggregateFunction(CountAggregateFunction.class),
+    SumAggregateFunction(SumAggregateFunction.class),
+    MinAggregateFunction(MinAggregateFunction.class),
+    MaxAggregateFunction(MaxAggregateFunction.class),
+    LikeExpression(LikeExpression.class),
+    NotExpression(NotExpression.class),
+    CaseExpression(CaseExpression.class),
+    InListExpression(InListExpression.class),
+    IsNullExpression(IsNullExpression.class),
+    LongSubtractExpression(LongSubtractExpression.class),
+    DateSubtractExpression(DateSubtractExpression.class),
+    DecimalSubtractExpression(DecimalSubtractExpression.class),
+    LongAddExpression(LongAddExpression.class),
+    DecimalAddExpression(DecimalAddExpression.class),
+    DateAddExpression(DateAddExpression.class),
+    LongMultiplyExpression(LongMultiplyExpression.class),
+    DecimalMultiplyExpression(DecimalMultiplyExpression.class),
+    LongDivideExpression(LongDivideExpression.class),
+    DecimalDivideExpression(DecimalDivideExpression.class),
+    CoalesceFunction(CoalesceFunction.class),
+    RegexpReplaceFunction(RegexpReplaceFunction.class),
+    SQLTypeNameFunction(SqlTypeNameFunction.class),
+    RegexpSubstrFunction(RegexpSubstrFunction.class),
+    StringConcatExpression(StringConcatExpression.class),
+    LengthFunction(LengthFunction.class),
+    LTrimFunction(LTrimFunction.class),
+    RTrimFunction(RTrimFunction.class),
+    TrimFunction(TrimFunction.class),
+    DistinctCountAggregateFunction(DistinctCountAggregateFunction.class),
+    PercentileContAggregateFunction(PercentileContAggregateFunction.class),
+    PercentRankAggregateFunction(PercentRankAggregateFunction.class),
+    StddevPopFunction(StddevPopFunction.class),
+    StddevSampFunction(StddevSampFunction.class),
+    PercentileDiscAggregateFunction(PercentileDiscAggregateFunction.class),
+    DoubleAddExpression(DoubleAddExpression.class),
+    DoubleSubtractExpression(DoubleSubtractExpression.class),
+    DoubleMultiplyExpression(DoubleMultiplyExpression.class),
+    DoubleDivideExpression(DoubleDivideExpression.class),
+    RowValueConstructorExpression(RowValueConstructorExpression.class),
+    SQLTableTypeFunction(SQLTableTypeFunction.class),
+    IndexKeyValue(IndexKeyValueColumnExpression.class),
+    IndexStateName(IndexStateNameFunction.class),
+    InvertFunction(InvertFunction.class),
+    ProjectedColumnExpression(ProjectedColumnExpression.class),
+    TimestampAddExpression(TimestampAddExpression.class),
+    TimestampSubtractExpression(TimestampSubtractExpression.class),
+    ArrayIndexFunction(ArrayIndexFunction.class),
+    ArrayLengthFunction(ArrayLengthFunction.class),
+    ArrayConstructorExpression(ArrayConstructorExpression.class),
+    SQLViewTypeFunction(SQLViewTypeFunction.class);
+    ExpressionType(Class<? extends Expression> clazz) {
+        this.clazz = clazz;
+    }
+
+    public Class<? extends Expression> getExpressionClass() {
+        return clazz;
+    }
+
+    private final Class<? extends Expression> clazz;
+
+    private static final Map<Class<? extends Expression>,ExpressionType> classToEnumMap = Maps.newHashMapWithExpectedSize(3);
+    static {
+        for (ExpressionType type : ExpressionType.values()) {
+            classToEnumMap.put(type.clazz, type);
+        }
+    }
+
+    /**
+     * Return the ExpressionType for a given Expression instance
+     */
+    public static ExpressionType valueOf(Expression expression) {
+        ExpressionType type = classToEnumMap.get(expression.getClass());
+        if (type == null) { // FIXME: this exception gets swallowed and retries happen
+            throw new IllegalArgumentException("No ExpressionType for " + expression.getClass());
+        }
+        return type;
+    }
+
+    /**
+     * Instantiates a DataAccessor based on its DataAccessorType
+     */
+    public Expression newInstance() {
+        try {
+            return clazz.newInstance();
+        } catch (InstantiationException e) {
+            throw new RuntimeException(e);
+        } catch (IllegalAccessException e) {
+            throw new RuntimeException(e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/InListExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/InListExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/InListExpression.java
new file mode 100644
index 0000000..f6a397d
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/InListExpression.java
@@ -0,0 +1,291 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.sql.SQLException;
+import java.util.Arrays;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Set;
+
+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.io.WritableUtils;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.expression.visitor.ExpressionVisitor;
+import org.apache.phoenix.schema.ConstraintViolationException;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.ByteUtil;
+
+/*
+ * Implementation of a SQL foo IN (a,b,c) expression. Other than the first
+ * expression, child expressions must be constants.
+ *
+ */
+public class InListExpression extends BaseSingleExpression {
+    private LinkedHashSet<ImmutableBytesPtr> values;
+    private ImmutableBytesPtr minValue;
+    private ImmutableBytesPtr maxValue;
+    private int valuesByteLength;
+    private boolean containsNull;
+    private int fixedWidth = -1;
+    private ImmutableBytesPtr value = new ImmutableBytesPtr();
+    private List<Expression> keyExpressions; // client side only
+
+    public static Expression create (List<Expression> children, ImmutableBytesWritable ptr) throws SQLException {
+        Expression firstChild = children.get(0);
+        PDataType firstChildType = firstChild.getDataType();
+        
+        boolean addedNull = false;
+        List<Expression> keys = Lists.newArrayListWithExpectedSize(children.size());
+        List<Expression> coercedKeyExpressions = Lists.newArrayListWithExpectedSize(children.size());
+        keys.add(firstChild);
+        coercedKeyExpressions.add(firstChild);
+        for (int i = 1; i < children.size(); i++) {
+            Expression rhs = children.get(i);
+            if (rhs.evaluate(null, ptr)) {
+                if (ptr.getLength() == 0) {
+                    if (!addedNull) {
+                        addedNull = true;
+                        keys.add(LiteralExpression.newConstant(null, PDataType.VARBINARY, true));
+                        coercedKeyExpressions.add(LiteralExpression.newConstant(null, firstChildType, true));
+                    }
+                } else {
+                    // Don't specify the firstChild column modifier here, as we specify it in the LiteralExpression creation below
+                    try {
+                        firstChildType.coerceBytes(ptr, rhs.getDataType(), rhs.getColumnModifier(), null);
+                        keys.add(LiteralExpression.newConstant(ByteUtil.copyKeyBytesIfNecessary(ptr), PDataType.VARBINARY, firstChild.getColumnModifier(), true));
+                        if(rhs.getDataType() == firstChildType) {
+                            coercedKeyExpressions.add(rhs);
+                        } else {
+                            coercedKeyExpressions.add(CoerceExpression.create(rhs, firstChildType));    
+                        }
+                    } catch (ConstraintViolationException e) { // Ignore and continue
+                    }
+                }
+            }
+            
+        }
+        if (keys.size() == 1) {
+            return LiteralExpression.newConstant(false, PDataType.BOOLEAN, true);
+        }
+        if (keys.size() == 2 && addedNull) {
+            return LiteralExpression.newConstant(null, PDataType.BOOLEAN, true);
+        }
+        Expression expression;
+        // TODO: if inChildren.isEmpty() then Oracle throws a type mismatch exception. This means
+        // that none of the list elements match in type and there's no null element. We'd return
+        // false in this case. Should we throw?
+        if (keys.size() == 2) {
+            expression = new ComparisonExpression(CompareOp.EQUAL, keys);
+        } else {
+            expression = new InListExpression(keys, coercedKeyExpressions);
+        }
+        return expression;
+    }    
+    public InListExpression() {
+    }
+
+    private InListExpression(List<Expression> keys, List<Expression> keyExpressions) throws SQLException {
+        super(keyExpressions.get(0));
+        this.keyExpressions = keyExpressions.subList(1, keyExpressions.size());
+        Set<ImmutableBytesPtr> values = Sets.newHashSetWithExpectedSize(keys.size()-1);
+        int fixedWidth = -1;
+        boolean isFixedLength = true;
+        for (int i = 1; i < keys.size(); i++) {
+            ImmutableBytesPtr ptr = new ImmutableBytesPtr();
+            Expression child = keys.get(i);
+            assert(child.getDataType() == PDataType.VARBINARY);
+            child.evaluate(null, ptr);
+            if (ptr.getLength() == 0) {
+                containsNull = true;
+            } else {
+                if (values.add(ptr)) {
+                    int length = ptr.getLength();
+                    if (fixedWidth == -1) {
+                        fixedWidth = length;
+                    } else {
+                        isFixedLength &= fixedWidth == length;
+                    }
+                    
+                    valuesByteLength += ptr.getLength();
+                }
+            }
+        }
+        this.fixedWidth = isFixedLength ? fixedWidth : -1;
+        // Sort values by byte value so we can get min/max easily
+        ImmutableBytesPtr[] valuesArray = values.toArray(new ImmutableBytesPtr[values.size()]);
+        Arrays.sort(valuesArray, ByteUtil.BYTES_PTR_COMPARATOR);
+        this.minValue = valuesArray[0];
+        this.maxValue = valuesArray[valuesArray.length-1];
+        this.values = new LinkedHashSet<ImmutableBytesPtr>(Arrays.asList(valuesArray));
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        if (!getChild().evaluate(tuple, ptr)) {
+            return false;
+        }
+        value.set(ptr);
+        if (values.contains(value)) {
+            ptr.set(PDataType.TRUE_BYTES);
+            return true;
+        }
+        if (containsNull) { // If any null value and value not found
+            ptr.set(ByteUtil.EMPTY_BYTE_ARRAY);
+            return true;
+        }
+        ptr.set(PDataType.FALSE_BYTES);
+        return true;
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + (containsNull ? 1231 : 1237);
+        result = prime * result + values.hashCode();
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) return true;
+        if (obj == null) return false;
+        if (getClass() != obj.getClass()) return false;
+        InListExpression other = (InListExpression)obj;
+        if (containsNull != other.containsNull) return false;
+        if (!values.equals(other.values)) return false;
+        return true;
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return PDataType.BOOLEAN;
+    }
+
+    @Override
+    public boolean isNullable() {
+        return super.isNullable() || containsNull;
+    }
+
+    private int readValue(DataInput input, byte[] valuesBytes, int offset, ImmutableBytesPtr ptr) throws IOException {
+        int valueLen = fixedWidth == -1 ? WritableUtils.readVInt(input) : fixedWidth;
+        values.add(new ImmutableBytesPtr(valuesBytes,offset,valueLen));
+        return offset + valueLen;
+    }
+    
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        super.readFields(input);
+        containsNull = input.readBoolean();
+        fixedWidth = WritableUtils.readVInt(input);
+        byte[] valuesBytes = Bytes.readByteArray(input);
+        valuesByteLength = valuesBytes.length;
+        int len = fixedWidth == -1 ? WritableUtils.readVInt(input) : valuesByteLength / fixedWidth;
+        values = Sets.newLinkedHashSetWithExpectedSize(len);
+        int offset = 0;
+        int i  = 0;
+        if (i < len) {
+            offset = readValue(input, valuesBytes, offset, minValue = new ImmutableBytesPtr());
+            while (++i < len-1) {
+                offset = readValue(input, valuesBytes, offset, new ImmutableBytesPtr());
+            }
+            if (i < len) {
+                offset = readValue(input, valuesBytes, offset, maxValue = new ImmutableBytesPtr());
+            } else {
+                maxValue = minValue;
+            }
+        } else {
+            minValue = maxValue = new ImmutableBytesPtr(ByteUtil.EMPTY_BYTE_ARRAY);
+        }
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+        super.write(output);
+        output.writeBoolean(containsNull);
+        WritableUtils.writeVInt(output, fixedWidth);
+        WritableUtils.writeVInt(output, valuesByteLength);
+        for (ImmutableBytesPtr ptr : values) {
+            output.write(ptr.get(), ptr.getOffset(), ptr.getLength());
+        }
+        if (fixedWidth == -1) {
+            WritableUtils.writeVInt(output, values.size());
+            for (ImmutableBytesPtr ptr : values) {
+                WritableUtils.writeVInt(output, ptr.getLength());
+            }
+        }
+    }
+
+    @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 List<Expression> getKeyExpressions() {
+        return keyExpressions;
+    }
+
+    public ImmutableBytesWritable getMinKey() {
+        return minValue;
+    }
+
+    public ImmutableBytesWritable getMaxKey() {
+        return maxValue;
+    }
+
+    @Override
+    public String toString() {
+        int maxToStringLen = 200;
+        Expression firstChild = children.get(0);
+        PDataType type = firstChild.getDataType();
+        StringBuilder buf = new StringBuilder(firstChild + " IN (");
+        if (containsNull) {
+            buf.append("null,");
+        }
+        for (ImmutableBytesPtr value : values) {
+            if (firstChild.getColumnModifier() != null) {
+                type.coerceBytes(value, type, firstChild.getColumnModifier(), null);
+            }
+            buf.append(type.toStringLiteral(value, null));
+            buf.append(',');
+            if (buf.length() >= maxToStringLen) {
+                buf.append("... ");
+                break;
+            }
+        }
+        buf.setCharAt(buf.length()-1,')');
+        return buf.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/IndexKeyValueColumnExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/IndexKeyValueColumnExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/IndexKeyValueColumnExpression.java
new file mode 100644
index 0000000..fb066c1
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/IndexKeyValueColumnExpression.java
@@ -0,0 +1,26 @@
+package org.apache.phoenix.expression;
+
+import org.apache.hadoop.hbase.util.Bytes;
+
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.util.IndexUtil;
+import org.apache.phoenix.util.SchemaUtil;
+
+public class IndexKeyValueColumnExpression extends KeyValueColumnExpression {
+    public IndexKeyValueColumnExpression() {
+    }
+
+    public IndexKeyValueColumnExpression(PColumn column) {
+        super(column);
+    }
+    
+    @Override
+    public String toString() {
+        // Translate to the data table column name
+        String indexColumnName = Bytes.toString(this.getColumnName());
+        String dataFamilyName = IndexUtil.getDataColumnFamilyName(indexColumnName);
+        String dataColumnName = IndexUtil.getDataColumnName(indexColumnName);
+        return SchemaUtil.getColumnDisplayName(dataFamilyName, dataColumnName);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/IsNullExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/IsNullExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/IsNullExpression.java
new file mode 100644
index 0000000..e6ef621
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/IsNullExpression.java
@@ -0,0 +1,106 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.*;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.expression.visitor.ExpressionVisitor;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+
+/**
+ * 
+ * Implementation of IS NULL and IS NOT NULL expression
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class IsNullExpression extends BaseSingleExpression {
+    private boolean isNegate;
+
+    public IsNullExpression() {
+    }
+    
+    public IsNullExpression(Expression expression, boolean negate) {
+        super(expression);
+        this.isNegate = negate;
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        boolean evaluated = getChild().evaluate(tuple, ptr);
+        if (evaluated) {
+            ptr.set(isNegate ^ ptr.getLength() == 0 ? PDataType.TRUE_BYTES : PDataType.FALSE_BYTES);
+            return true;
+        }
+        if (tuple.isImmutable()) {
+            ptr.set(isNegate ? PDataType.FALSE_BYTES : PDataType.TRUE_BYTES);
+            return true;
+        }
+        
+        return false;
+    }
+
+    public boolean isNegate() {
+        return isNegate;
+    }
+    
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        super.readFields(input);
+        isNegate = input.readBoolean();
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+        super.write(output);
+        output.writeBoolean(isNegate);
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return PDataType.BOOLEAN;
+    }
+    
+    @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;
+    }
+    
+    @Override
+    public String toString() {
+        StringBuilder buf = new StringBuilder(children.get(0).toString());
+        if (isNegate) {
+            buf.append(" IS NOT NULL");
+        } else {
+            buf.append(" IS NULL");
+        }
+        return buf.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..b082aba
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/KeyValueColumnExpression.java
@@ -0,0 +1,113 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.Arrays;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import org.apache.phoenix.expression.visitor.ExpressionVisitor;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.SchemaUtil;
+
+
+/**
+ * 
+ * Class to access a column value stored in a KeyValue
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class KeyValueColumnExpression extends ColumnExpression {
+    private byte[] cf;
+    private byte[] cq;
+
+    public KeyValueColumnExpression() {
+    }
+
+    public KeyValueColumnExpression(PColumn column) {
+        super(column);
+        this.cf = column.getFamilyName().getBytes();
+        this.cq = column.getName().getBytes();
+    }
+
+    public byte[] getColumnFamily() {
+        return cf;
+    }
+
+    public byte[] getColumnName() {
+        return cq;
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + Arrays.hashCode(cf);
+        result = prime * result + Arrays.hashCode(cq);
+        return result;
+    }
+
+    // TODO: assumes single table
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) return true;
+        if (obj == null) return false;
+        if (getClass() != obj.getClass()) return false;
+        KeyValueColumnExpression other = (KeyValueColumnExpression)obj;
+        if (!Arrays.equals(cf, other.cf)) return false;
+        if (!Arrays.equals(cq, other.cq)) return false;
+        return true;
+    }
+
+    @Override
+    public String toString() {
+        return SchemaUtil.getColumnDisplayName(cf, cq);
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        return tuple.getValue(cf, cq, ptr);
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        super.readFields(input);
+        cf = Bytes.readByteArray(input);
+        cq = Bytes.readByteArray(input);
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+        super.write(output);
+        Bytes.writeByteArray(output, cf);
+        Bytes.writeByteArray(output, cq);
+    }
+
+    @Override
+    public final <T> T accept(ExpressionVisitor<T> visitor) {
+        return visitor.visit(this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/LikeExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/LikeExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/LikeExpression.java
new file mode 100644
index 0000000..038f705
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/LikeExpression.java
@@ -0,0 +1,302 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.IOException;
+import java.util.List;
+import java.util.regex.Pattern;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.phoenix.expression.visitor.ExpressionVisitor;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.StringUtil;
+
+
+/**
+ * 
+ * Implementation for LIKE operation where the first child expression is the string
+ * and the second is the pattern. The pattern supports '_' character for single 
+ * character wildcard match and '%' for zero or more character match. where these
+ * characters may be escaped by preceding them with a '\'.
+ * 
+ * Example: foo LIKE 'ab%' will match a row in which foo starts with 'ab'
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class LikeExpression extends BaseCompoundExpression {
+    private static final Logger logger = LoggerFactory.getLogger(LikeExpression.class);
+    
+    private static final String ZERO_OR_MORE = "\\E.*\\Q";
+    private static final String ANY_ONE = "\\E.\\Q";
+    
+    public static String unescapeLike(String s) {
+        return StringUtil.replace(s, StringUtil.LIKE_ESCAPE_SEQS, StringUtil.LIKE_UNESCAPED_SEQS);
+    }
+
+    /**
+     * @return the substring of s for which we have a literal string
+     *  that we can potentially use to set the start/end key, or null
+     *  if there is none.
+     */
+    public static String getStartsWithPrefix(String s) {
+        int i = indexOfWildcard(s);
+        return i == -1 ? s : s.substring(0,i);
+    }
+
+    public static boolean hasWildcards(String s) {
+        return indexOfWildcard(s) != -1;
+    }
+    
+    /**
+     * Replace unescaped '*' and '?' in s with '%' and '_' respectively
+     * such that the returned string may be used in a LIKE expression.
+     * Provides an alternate way of expressing a LIKE pattern which is
+     * more friendly for wildcard matching when the source string is
+     * likely to contain an '%' or '_' character.
+     * @param s wildcard pattern that may use '*' for multi character
+     * match and '?' for single character match, escaped by the backslash
+     * character
+     * @return replaced 
+     */
+    public static String wildCardToLike(String s) {
+        s = StringUtil.escapeLike(s);
+        StringBuilder buf = new StringBuilder();
+        // Look for another unprotected * or ? in the middle
+        int i = 0;
+        int j = 0;
+        while (true) {
+            int pctPos = s.indexOf(StringUtil.MULTI_CHAR_WILDCARD, i);
+            int underPos = s.indexOf(StringUtil.SINGLE_CHAR_WILDCARD, i);
+            if (pctPos == -1 && underPos == -1) {
+                return i == 0 ? s : buf.append(s.substring(i)).toString();
+            }
+            i = pctPos;
+            if (underPos != -1 && (i == -1 || underPos < i)) {
+                i = underPos;
+            }
+            
+            if (i > 0 && s.charAt(i - 1) == '\\') {
+                // If we found protection then keep looking
+                buf.append(s.substring(j,i-1));
+                buf.append(s.charAt(i));
+            } else {
+                // We found an unprotected % or _ in the middle
+                buf.append(s.substring(j,i));
+                buf.append(s.charAt(i) == StringUtil.MULTI_CHAR_WILDCARD ? StringUtil.MULTI_CHAR_LIKE : StringUtil.SINGLE_CHAR_LIKE);
+            }
+            j = ++i;
+        }
+    }
+    
+    public static int indexOfWildcard(String s) {
+        // Look for another unprotected % or _ in the middle
+        if (s == null) {
+            return -1;
+        }
+        int i = 0;
+        while (true) {
+            int pctPos = s.indexOf(StringUtil.MULTI_CHAR_LIKE, i);
+            int underPos = s.indexOf(StringUtil.SINGLE_CHAR_LIKE, i);
+            if (pctPos == -1 && underPos == -1) {
+                return -1;
+            }
+            i = pctPos;
+            if (underPos != -1 && (i == -1 || underPos < i)) {
+                i = underPos;
+            }
+            
+            if (i > 0 && s.charAt(i - 1) == '\\') {
+                // If we found protection then keep looking
+                i++;
+            } else {
+                // We found an unprotected % or _ in the middle
+                return i;
+            }
+        }
+    }
+
+    private static String toPattern(String s) {
+        StringBuilder sb = new StringBuilder(s.length());
+
+        // From the JDK doc: \Q and \E protect everything between them
+        sb.append("\\Q");
+        boolean wasSlash = false;
+        for (int i = 0; i < s.length(); i++) {
+            char c = s.charAt(i);
+            if (wasSlash) {
+                sb.append(c);
+                wasSlash = false;
+            } else if (c == StringUtil.SINGLE_CHAR_LIKE) {
+                sb.append(ANY_ONE);
+            } else if (c == StringUtil.MULTI_CHAR_LIKE) {
+                sb.append(ZERO_OR_MORE);
+            } else if (c == '\\') {
+                wasSlash = true;
+            } else {
+                sb.append(c);
+            }
+        }
+        sb.append("\\E");
+        // Found nothing interesting
+        return sb.toString();
+    }
+
+//    private static String fromPattern(String s) {
+//        StringBuilder sb = new StringBuilder(s.length());
+//
+//        for (int i = 0; i < s.length(); i++) {
+//            if (s.substring(i).startsWith("\\Q")) {
+//                while (s.substring(i + "\\Q".length()).startsWith("\\E")) {
+//                    sb.append(s.charAt(i++ + "\\Q".length()));
+//                }
+//                i+= "\\E".length();
+//            }
+//            if (s.charAt(i) == '.') {
+//                if (s.charAt(i+1) == '*') {
+//                    sb.append('%');
+//                    i+=2;                    
+//                } else {
+//                    sb.append('_');
+//                    i++;
+//                }
+//            }
+//        }
+//        return sb.toString();
+//    }
+
+    private Pattern pattern;
+    
+    public LikeExpression() {
+    }
+
+    public LikeExpression(List<Expression> children) {
+        super(children);
+        init();
+    }
+    
+    public boolean startsWithWildcard() {
+        return pattern != null && pattern.pattern().startsWith("\\Q\\E");
+    }
+    
+    private void init() {
+        Expression e = getPatternExpression();
+        if (e instanceof LiteralExpression) {
+            LiteralExpression patternExpression = (LiteralExpression)e;
+            String value = (String)patternExpression.getValue();
+            pattern = Pattern.compile(toPattern(value));
+        }
+    }
+
+    private Expression getStrExpression() {
+        return children.get(0);
+    }
+
+    private Expression getPatternExpression() {
+        return children.get(1);
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        Pattern pattern = this.pattern;
+        if (pattern == null) { // TODO: don't allow? this is going to be slooowwww
+            if (!getPatternExpression().evaluate(tuple, ptr)) {
+                if (logger.isDebugEnabled()) {
+                    logger.debug("LIKE is FALSE: pattern is null");
+                }
+                return false;
+            }
+            String value = (String)PDataType.VARCHAR.toObject(ptr, getPatternExpression().getColumnModifier());
+            pattern = Pattern.compile(toPattern(value));
+            if (logger.isDebugEnabled()) {
+                logger.debug("LIKE pattern is expression: " + pattern.pattern());
+            }
+        }
+        
+        if (!getStrExpression().evaluate(tuple, ptr)) {
+            if (logger.isDebugEnabled()) {
+                logger.debug("LIKE is FALSE: child expression is null");
+            }
+            return false;
+        }
+        if (ptr.getLength() == 0) {
+            return true;
+        }
+        
+        String value = (String)PDataType.VARCHAR.toObject(ptr, getStrExpression().getColumnModifier());
+        boolean matched = pattern.matcher(value).matches();
+        ptr.set(matched ? PDataType.TRUE_BYTES : PDataType.FALSE_BYTES);
+        if (logger.isDebugEnabled()) {
+            logger.debug("LIKE(value='" + value + "'pattern='" + pattern.pattern() + "' is " + matched);
+        }
+        return true;
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        super.readFields(input);
+        init();
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return PDataType.BOOLEAN;
+    }
+    
+    @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 String getLiteralPrefix() {
+        if (pattern == null) {
+            return "";
+        }
+        String pattern = this.pattern.pattern();
+        int fromIndex = "\\Q".length();
+        return pattern.substring(fromIndex, pattern.indexOf("\\E", fromIndex));
+    }
+
+    public boolean endsWithOnlyWildcard() {
+        if (pattern == null) {
+            return false;
+        }
+        String pattern = this.pattern.pattern();
+        String endsWith = ZERO_OR_MORE + "\\E";
+        return pattern.endsWith(endsWith) && 
+        pattern.lastIndexOf(ANY_ONE, pattern.length() - endsWith.length() - 1) == -1 &&
+        pattern.lastIndexOf(ZERO_OR_MORE, pattern.length() - endsWith.length() - 1) == -1;
+    }
+    
+    @Override
+    public String toString() {
+        return (children.get(0) + " LIKE " + children.get(1));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..202bcb0
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/LiteralExpression.java
@@ -0,0 +1,316 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.sql.SQLException;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.io.WritableUtils;
+
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+import org.apache.phoenix.expression.visitor.ExpressionVisitor;
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.IllegalDataException;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.TypeMismatchException;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.StringUtil;
+
+
+
+/**
+ * 
+ * Accessor for a literal value.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class LiteralExpression extends BaseTerminalExpression {
+    public static final LiteralExpression NULL_EXPRESSION = new LiteralExpression(null, false);
+    private static final LiteralExpression ND_NULL_EXPRESSION = new LiteralExpression(null, true);
+    private static final LiteralExpression[] TYPED_NULL_EXPRESSIONS = new LiteralExpression[PDataType.values().length * 2];
+    static {
+        for (int i = 0; i < PDataType.values().length; i++) {
+            TYPED_NULL_EXPRESSIONS[i] = new LiteralExpression(PDataType.values()[i], true);
+        }
+        for (int i = 0; i < PDataType.values().length; i++) {
+            TYPED_NULL_EXPRESSIONS[i+PDataType.values().length] = new LiteralExpression(PDataType.values()[i], false);
+        }
+    }
+    private static final LiteralExpression FALSE_EXPRESSION = new LiteralExpression(Boolean.FALSE, PDataType.BOOLEAN, PDataType.BOOLEAN.toBytes(Boolean.FALSE), true);
+    private static final LiteralExpression TRUE_EXPRESSION = new LiteralExpression(Boolean.TRUE, PDataType.BOOLEAN, PDataType.BOOLEAN.toBytes(Boolean.TRUE), true);
+    private static final LiteralExpression ND_FALSE_EXPRESSION = new LiteralExpression(Boolean.FALSE, PDataType.BOOLEAN, PDataType.BOOLEAN.toBytes(Boolean.FALSE), false);
+    private static final LiteralExpression ND_TRUE_EXPRESSION = new LiteralExpression(Boolean.TRUE, PDataType.BOOLEAN, PDataType.BOOLEAN.toBytes(Boolean.TRUE), false);
+
+    private Object value;
+    private PDataType type;
+    private boolean isDeterministic;
+    private byte[] byteValue;
+    private Integer byteSize;
+    private Integer maxLength;
+    private Integer scale;
+    private ColumnModifier columnModifier;
+
+
+    public static boolean isFalse(Expression child) {
+        return child == FALSE_EXPRESSION || child == ND_FALSE_EXPRESSION;
+    }
+    
+    public static boolean isTrue(Expression child) {
+        return child == TRUE_EXPRESSION || child == ND_TRUE_EXPRESSION;
+    }
+    
+    public static LiteralExpression newConstant(Object value) {
+        return newConstant(value, true);
+    }
+    
+    // TODO: cache?
+    public static LiteralExpression newConstant(Object value, boolean isDeterministic) {
+        if (Boolean.FALSE.equals(value)) {
+            return isDeterministic ? FALSE_EXPRESSION : ND_FALSE_EXPRESSION;
+        }
+        if (Boolean.TRUE.equals(value)) {
+            return isDeterministic ? TRUE_EXPRESSION : ND_TRUE_EXPRESSION;
+        }
+        if (value == null) {
+            return isDeterministic ? NULL_EXPRESSION : ND_NULL_EXPRESSION;
+        }
+        PDataType type = PDataType.fromLiteral(value);
+        byte[] b = type.toBytes(value);
+        if (b.length == 0) {
+            return TYPED_NULL_EXPRESSIONS[type.ordinal() + ( isDeterministic ? 0 : TYPED_NULL_EXPRESSIONS.length/2)];
+        }
+        if (type == PDataType.VARCHAR) {
+            String s = (String) value;
+            if (s.length() == b.length) { // single byte characters only
+                type = PDataType.CHAR;
+            }
+        }
+        return new LiteralExpression(value, type, b, isDeterministic);
+    }
+
+    public static LiteralExpression newConstant(Object value, PDataType type) throws SQLException {
+        return newConstant(value, type, true);
+    }
+    
+    public static LiteralExpression newConstant(Object value, PDataType type, boolean isDeterministic) throws SQLException {
+        return newConstant(value, type, null, isDeterministic);
+    }
+    
+    public static LiteralExpression newConstant(Object value, PDataType type, ColumnModifier columnModifier) throws SQLException {
+        return newConstant(value, type, null, null, columnModifier, true);
+    }
+    
+    public static LiteralExpression newConstant(Object value, PDataType type, ColumnModifier columnModifier, boolean isDeterministic) throws SQLException {
+        return newConstant(value, type, null, null, columnModifier, isDeterministic);
+    }
+    
+    public static LiteralExpression newConstant(Object value, PDataType type, Integer maxLength, Integer scale) throws SQLException {
+        return newConstant(value, type, maxLength, scale, null, true);
+    }
+    
+    public static LiteralExpression newConstant(Object value, PDataType type, Integer maxLength, Integer scale, boolean isDeterministic) throws SQLException { // remove?
+        return newConstant(value, type, maxLength, scale, null, isDeterministic);
+    }
+
+    // TODO: cache?
+    public static LiteralExpression newConstant(Object value, PDataType type, Integer maxLength, Integer scale, ColumnModifier columnModifier, boolean isDeterministic)
+            throws SQLException {
+        if (value == null) {
+            if (type == null) {
+                return NULL_EXPRESSION;
+            }
+            return TYPED_NULL_EXPRESSIONS[type.ordinal()];
+        }
+        if (Boolean.FALSE.equals(value)) {
+            return isDeterministic ? FALSE_EXPRESSION : ND_FALSE_EXPRESSION;
+        }
+        if (Boolean.TRUE.equals(value)) {
+            return isDeterministic ? TRUE_EXPRESSION : ND_TRUE_EXPRESSION;
+        }
+        PDataType actualType = PDataType.fromLiteral(value);
+        // For array we should check individual element in it?
+        // It would be costly though!!!!!
+        if (!actualType.isCoercibleTo(type, value)) {
+            throw TypeMismatchException.newException(type, actualType, value.toString());
+        }
+        value = type.toObject(value, actualType);
+        try {
+            byte[] b = type.toBytes(value, columnModifier);
+            if (type == PDataType.VARCHAR || type == PDataType.CHAR) {
+                if (type == PDataType.CHAR && maxLength != null  && b.length < maxLength) {
+                    b = StringUtil.padChar(b, maxLength);
+                } else if (value != null) {
+                    maxLength = ((String)value).length();
+                }
+            }
+            if (b.length == 0) {
+                return TYPED_NULL_EXPRESSIONS[type.ordinal()];
+            }
+            return new LiteralExpression(value, type, b, maxLength, scale, columnModifier, isDeterministic);
+        } catch (IllegalDataException e) {
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode.ILLEGAL_DATA).setRootCause(e).build().buildException();
+        }
+    }
+
+    public LiteralExpression() {
+    }
+
+    private LiteralExpression(PDataType type, boolean isDeterministic) {
+        this(null, type, ByteUtil.EMPTY_BYTE_ARRAY, isDeterministic);
+    }
+
+    private LiteralExpression(Object value, PDataType type, byte[] byteValue, boolean isDeterministic) {
+        this(value, type, byteValue, type == null? null : type.getMaxLength(value), type == null? null : type.getScale(value), null, isDeterministic);
+    }
+
+    private LiteralExpression(Object value, PDataType type, byte[] byteValue,
+            Integer maxLength, Integer scale, ColumnModifier columnModifier, boolean isDeterministic) {
+        this.value = value;
+        this.type = type;
+        this.byteValue = byteValue;
+        this.byteSize = byteValue.length;
+        this.maxLength = maxLength;
+        this.scale = scale;
+        this.columnModifier = columnModifier;
+        this.isDeterministic = isDeterministic;
+    }
+
+    @Override
+    public boolean isDeterministic() {
+        return isDeterministic;
+    }
+    
+    @Override
+    public String toString() {
+        return type != null && type.isCoercibleTo(PDataType.VARCHAR) ? "'" + value + "'" : "" + value;
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + ((value == null) ? 0 : value.hashCode());
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) return true;
+        if (obj == null) return false;
+        if (getClass() != obj.getClass()) return false;
+        LiteralExpression other = (LiteralExpression)obj;
+        if (value == null) {
+            if (other.value != null) return false;
+        } else if (!value.equals(other.value)) return false;
+        return true;
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        int encodedByteLengthAndBool = WritableUtils.readVInt(input);
+        this.isDeterministic = encodedByteLengthAndBool > 0;
+        int byteLength = Math.abs(encodedByteLengthAndBool)-1;
+        this.byteValue = new byte[byteLength];
+        input.readFully(byteValue, 0, byteLength);
+        columnModifier = ColumnModifier.fromSystemValue(WritableUtils.readVInt(input));
+        int typeOrdinal = WritableUtils.readVInt(input);
+        if (typeOrdinal < 0) {
+            this.type = null;
+        } else {
+            this.type = PDataType.values()[typeOrdinal];
+        }
+        if (this.byteValue.length == 0) {
+            this.value = null;
+        } else {
+            this.value = this.type.toObject(byteValue, 0, byteValue.length, this.type, columnModifier);
+        }
+        // Only to prevent continual reallocations of Integer
+        this.byteSize = this.byteValue.length;
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+        WritableUtils.writeVInt(output, (byteValue.length + 1) * (this.isDeterministic ? 1 : -1));
+        output.write(byteValue);
+        WritableUtils.writeVInt(output, ColumnModifier.toSystemValue(columnModifier));
+        WritableUtils.writeVInt(output, type == null ? -1 : this.type.ordinal());
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        // Literal always evaluates, even when it returns null
+        ptr.set(byteValue);
+        return true;
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return type;
+    }
+
+    @Override
+    public Integer getByteSize() {
+        return byteSize;
+    }
+
+    @Override
+    public Integer getMaxLength() {
+        return maxLength;
+    }
+
+    @Override
+    public Integer getScale() {
+        return scale;
+    }
+    
+    @Override
+    public ColumnModifier getColumnModifier() {
+        return columnModifier;
+    }
+
+    @Override
+    public boolean isNullable() {
+        return value == null;
+    }
+
+    public Object getValue() {
+        return value;
+    }
+
+    public byte[] getBytes() {
+        return byteValue;
+    }
+    
+    @Override
+    public final <T> T accept(ExpressionVisitor<T> visitor) {
+        return visitor.visit(this);
+    }
+    
+    @Override
+    public boolean isStateless() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/LongAddExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/LongAddExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/LongAddExpression.java
new file mode 100644
index 0000000..a14286d
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/LongAddExpression.java
@@ -0,0 +1,62 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+
+public class LongAddExpression extends AddExpression {
+
+    public LongAddExpression() {
+    }
+
+    public LongAddExpression(List<Expression> children) {
+        super(children);
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        long finalResult=0;
+        
+        for(int i=0;i<children.size();i++) {
+            Expression child = children.get(i);
+            if (!child.evaluate(tuple, ptr) || ptr.getLength() == 0) {
+                return false;
+            }
+            long childvalue = child.getDataType().getCodec().decodeLong(ptr, child.getColumnModifier());
+            finalResult += childvalue;
+        }
+        byte[] resultPtr=new byte[PDataType.LONG.getByteSize()];
+        ptr.set(resultPtr);
+        getDataType().getCodec().encodeLong(finalResult, ptr);
+        return true;
+    }
+
+    @Override
+    public final PDataType getDataType() {
+        return PDataType.LONG;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/LongDivideExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/LongDivideExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/LongDivideExpression.java
new file mode 100644
index 0000000..68560a6
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/LongDivideExpression.java
@@ -0,0 +1,66 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+
+public class LongDivideExpression extends DivideExpression {
+
+    public LongDivideExpression() {
+    }
+
+    public LongDivideExpression(List<Expression> children) {
+        super(children);
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        long finalResult=0;
+        
+        for(int i=0;i<children.size();i++) {
+            Expression child = children.get(i);
+            if (!child.evaluate(tuple, ptr) || ptr.getLength() == 0) {
+                return false;
+            }
+            long childvalue = child.getDataType().getCodec().decodeLong(ptr, child.getColumnModifier()); 
+            if (i == 0) {
+                finalResult = childvalue;
+            } else {
+                finalResult /= childvalue;
+            }
+        }
+        byte[] resultPtr=new byte[PDataType.LONG.getByteSize()];
+        ptr.set(resultPtr);
+        getDataType().getCodec().encodeLong(finalResult, ptr);
+        return true;
+    }
+
+    @Override
+    public final PDataType getDataType() {
+        return PDataType.LONG;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/LongMultiplyExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/LongMultiplyExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/LongMultiplyExpression.java
new file mode 100644
index 0000000..508d33c
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/LongMultiplyExpression.java
@@ -0,0 +1,65 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+
+public class LongMultiplyExpression extends MultiplyExpression {
+
+    public LongMultiplyExpression() {
+    }
+
+    public LongMultiplyExpression(List<Expression> children) {
+        super(children);
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        long finalResult=1;
+        
+        for(int i=0;i<children.size();i++) {
+            Expression child = children.get(i);
+            if (!child.evaluate(tuple, ptr)) {
+                return false;
+            }
+            if (ptr.getLength() == 0) {
+                return false;
+            }
+            long childvalue = child.getDataType().getCodec().decodeLong(ptr, child.getColumnModifier());
+            finalResult *= childvalue;
+        }
+        byte[] resultPtr=new byte[getDataType().getByteSize()];
+        ptr.set(resultPtr);
+        getDataType().getCodec().encodeLong(finalResult, ptr);
+        return true;
+    }
+
+    @Override
+    public final PDataType getDataType() {
+        return PDataType.LONG;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/LongSubtractExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/LongSubtractExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/LongSubtractExpression.java
new file mode 100644
index 0000000..56317d6
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/LongSubtractExpression.java
@@ -0,0 +1,83 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+
+
+/**
+ * 
+ * Subtract expression implementation
+ *
+ * @author kmahadik
+ * @since 0.1
+ */
+public class LongSubtractExpression extends SubtractExpression {
+    public LongSubtractExpression() {
+    }
+
+    public LongSubtractExpression(List<Expression> children) {
+        super(children);
+    }
+
+    @Override
+	public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+		long finalResult=0;
+		
+		for(int i=0;i<children.size();i++) {
+		    Expression child = children.get(i);
+            if (!child.evaluate(tuple, ptr) || ptr.getLength() == 0) {
+                return false;
+            }
+            PDataType childType = child.getDataType();
+            boolean isDate = childType.isCoercibleTo(PDataType.DATE);
+            long childvalue = childType.getCodec().decodeLong(ptr, child.getColumnModifier());
+            if (i == 0) {
+                finalResult = childvalue;
+            } else {
+                finalResult -= childvalue;
+                /*
+                 * Special case for date subtraction - note that only first two expression may be dates.
+                 * We need to convert the date to a unit of "days" because that's what sql expects.
+                 */
+                if (isDate) {
+                    finalResult /= QueryConstants.MILLIS_IN_DAY;
+                }
+            }
+		}
+		byte[] resultPtr=new byte[getDataType().getByteSize()];
+		ptr.set(resultPtr);
+		getDataType().getCodec().encodeLong(finalResult, ptr);
+		return true;
+	}
+
+	@Override
+	public final PDataType getDataType() {
+		return PDataType.LONG;
+	}
+	
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/MultiplyExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/MultiplyExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/MultiplyExpression.java
new file mode 100644
index 0000000..fa815a2
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/MultiplyExpression.java
@@ -0,0 +1,98 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.util.List;
+
+import org.apache.phoenix.expression.visitor.ExpressionVisitor;
+import org.apache.phoenix.schema.PDataType;
+
+
+/**
+ * 
+ * Subtract expression implementation
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public abstract class MultiplyExpression extends ArithmeticExpression {
+    private Integer maxLength;
+    private Integer scale;
+
+    public MultiplyExpression() {
+    }
+
+    public MultiplyExpression(List<Expression> children) {
+        super(children);
+        for (int i=0; i<children.size(); i++) {
+            Expression childExpr = children.get(i);
+            if (i == 0) {
+                maxLength = childExpr.getMaxLength();
+                scale = childExpr.getScale();
+            } else {
+                maxLength = getPrecision(maxLength, childExpr.getMaxLength(), scale, childExpr.getScale());
+                scale = getScale(maxLength, childExpr.getMaxLength(), scale, childExpr.getScale());
+            }
+        }
+    }
+
+    @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;
+    }
+
+    @Override
+    public String getOperatorString() {
+        return " * ";
+    }
+    
+    private static Integer getPrecision(Integer lp, Integer rp, Integer ls, Integer rs) {
+    	if (ls == null || rs == null) {
+    		return PDataType.MAX_PRECISION;
+    	}
+        int val = lp + rp;
+        return Math.min(PDataType.MAX_PRECISION, val);
+    }
+
+    private static Integer getScale(Integer lp, Integer rp, Integer ls, Integer rs) {
+    	// If we are adding a decimal with scale and precision to a decimal
+    	// with no precision nor scale, the scale system does not apply.
+    	if (ls == null || rs == null) {
+    		return null;
+    	}
+        int val = ls + rs;
+        return Math.min(PDataType.MAX_PRECISION, val);
+    }
+    
+    @Override
+    public Integer getScale() {
+        return scale;
+    }
+
+    @Override
+    public Integer getMaxLength() {
+        return maxLength;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/NotExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/NotExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/NotExpression.java
new file mode 100644
index 0000000..c3a07c9
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/NotExpression.java
@@ -0,0 +1,86 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.expression.visitor.ExpressionVisitor;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+
+/**
+ * 
+ * Implementation of the NOT operator that negates it's
+ * single boolean child expression.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class NotExpression extends BaseSingleExpression {
+
+    public NotExpression() {
+    }
+
+    public NotExpression(Expression expression) {
+        super(expression);
+        if (expression == null || expression.getDataType() != PDataType.BOOLEAN) {
+            throw new IllegalArgumentException("NOT expectes a single BOOLEAN expression, but got " + expression);
+        }
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        if (!getChild().evaluate(tuple, ptr)) {
+            return false;
+        }
+        if (ptr.getLength() == 0) {
+            return true;
+        }
+        
+        ptr.set(Boolean.TRUE.equals(PDataType.BOOLEAN.toObject(ptr)) ? PDataType.FALSE_BYTES : PDataType.TRUE_BYTES);
+        return true;
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return PDataType.BOOLEAN;
+    }
+    
+    @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;
+    }
+    
+    @Override
+    public String toString() {
+        StringBuilder buf = new StringBuilder("NOT (");
+        buf.append(children.get(0).toString());
+        buf.append(")");
+        return buf.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/OrExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/OrExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/OrExpression.java
new file mode 100644
index 0000000..7339852
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/OrExpression.java
@@ -0,0 +1,67 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.util.List;
+
+import org.apache.phoenix.expression.visitor.ExpressionVisitor;
+
+
+/**
+ * 
+ * OR expression implementation
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class OrExpression extends AndOrExpression {
+    public OrExpression() {
+    }
+
+    public OrExpression(List<Expression> children) {
+        super(children);
+    }
+
+    @Override
+    protected boolean getStopValue() {
+        return Boolean.TRUE;
+    }
+
+    @Override
+    public String toString() {
+        StringBuilder buf = new StringBuilder("(");
+        for (int i = 0; i < children.size() - 1; i++) {
+            buf.append(children.get(i) + " OR ");
+        }
+        buf.append(children.get(children.size()-1));
+        buf.append(')');
+        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;
+    }
+}


[22/51] [partial] Initial commit of master branch from github

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixEmbeddedDriver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixEmbeddedDriver.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixEmbeddedDriver.java
new file mode 100644
index 0000000..3550584
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixEmbeddedDriver.java
@@ -0,0 +1,325 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.jdbc;
+
+import java.sql.*;
+import java.util.*;
+import java.util.logging.Logger;
+
+import com.google.common.collect.Maps;
+import org.apache.phoenix.coprocessor.MetaDataProtocol;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+import org.apache.phoenix.query.ConnectionQueryServices;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.util.*;
+
+
+
+/**
+ * 
+ * Abstract base class for JDBC Driver implementation of Phoenix
+ * 
+ * @author jtaylor
+ * @since 0.1
+ */
+public abstract class PhoenixEmbeddedDriver implements Driver, org.apache.phoenix.jdbc.Jdbc7Shim.Driver, SQLCloseable {
+    /**
+     * The protocol for Phoenix Network Client 
+     */ 
+    private final static String DNC_JDBC_PROTOCOL_SUFFIX = "//";
+    private static final String TERMINATOR = "" + PhoenixRuntime.JDBC_PROTOCOL_TERMINATOR;
+    private static final String DELIMITERS = TERMINATOR + PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR;
+
+    private final static DriverPropertyInfo[] EMPTY_INFO = new DriverPropertyInfo[0];
+    public final static String MAJOR_VERSION_PROP = "DriverMajorVersion";
+    public final static String MINOR_VERSION_PROP = "DriverMinorVersion";
+    public final static String DRIVER_NAME_PROP = "DriverName";
+    
+    private final QueryServices services;
+
+    
+    PhoenixEmbeddedDriver(QueryServices queryServices) {
+        services = queryServices;
+    }
+    
+    private String getDriverName() {
+        return this.getClass().getName();
+    }
+    
+    public QueryServices getQueryServices() {
+        return services;
+    }
+     
+    @Override
+    public boolean acceptsURL(String url) throws SQLException {
+        if (url.startsWith(PhoenixRuntime.JDBC_PROTOCOL)) {
+            // A connection string of "jdbc:phoenix" is supported, since
+            // all the connection information can potentially be gotten
+            // out of the HBase config file
+            if (url.length() == PhoenixRuntime.JDBC_PROTOCOL.length()) {
+                return true;
+            }
+            // Same as above, except for "jdbc:phoenix;prop=<value>..."
+            if (PhoenixRuntime.JDBC_PROTOCOL_TERMINATOR == url.charAt(PhoenixRuntime.JDBC_PROTOCOL.length())) {
+                return true;
+            }
+            if (PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR == url.charAt(PhoenixRuntime.JDBC_PROTOCOL.length())) {
+                int protoLength = PhoenixRuntime.JDBC_PROTOCOL.length() + 1;
+                // A connection string of "jdbc:phoenix:" matches this driver,
+                // but will end up as a MALFORMED_CONNECTION_URL exception later.
+                if (url.length() == protoLength) {
+                    return true;
+                }
+                // A connection string of the form "jdbc:phoenix://" means that
+                // the driver is remote which isn't supported, so return false.
+                if (!url.startsWith(DNC_JDBC_PROTOCOL_SUFFIX, protoLength)) {
+                    return true;
+                }
+            }
+        }
+        return false;
+    }
+
+    @Override
+    public Connection connect(String url, Properties info) throws SQLException {
+        if (!acceptsURL(url)) {
+            return null;
+        }
+
+        ConnectionQueryServices connectionServices = getConnectionQueryServices(url, info);
+        info.setProperty(MAJOR_VERSION_PROP, Integer.toString(getMajorVersion()));
+        info.setProperty(MINOR_VERSION_PROP, Integer.toString(getMinorVersion()));
+        info.setProperty(DRIVER_NAME_PROP, getDriverName());
+        PhoenixConnection connection = connectionServices.connect(url, info);
+        return connection;
+    }
+
+    /**
+     * Get or create if necessary a QueryServices that is associated with the HBase zookeeper quorum
+     * name (part of the connection URL). This will cause the underlying Configuration held by the
+     * QueryServices to be shared for all connections to the same HBase cluster.
+     * @param url connection URL
+     * @param info connection properties
+     * @return new or cached QuerySerices used to establish a new Connection.
+     * @throws SQLException
+     */
+    protected abstract ConnectionQueryServices getConnectionQueryServices(String url, Properties info) throws SQLException;
+    
+    @Override
+    public int getMajorVersion() {
+        return MetaDataProtocol.PHOENIX_MAJOR_VERSION;
+    }
+
+    @Override
+    public int getMinorVersion() {
+        return MetaDataProtocol.PHOENIX_MINOR_VERSION;
+    }
+
+    @Override
+    public DriverPropertyInfo[] getPropertyInfo(String url, Properties info) throws SQLException {
+        return EMPTY_INFO;
+    }
+
+    @Override
+    public boolean jdbcCompliant() {
+        return false;
+    }
+
+    @Override
+    public Logger getParentLogger() throws SQLFeatureNotSupportedException {
+        return null;
+    }
+
+    @Override
+    public void close() throws SQLException {
+    }
+    
+    /**
+     * 
+     * Class to encapsulate connection info for HBase
+     *
+     * @author jtaylor
+     * @since 0.1.1
+     */
+    public static class ConnectionInfo {
+        protected static ConnectionInfo create(String url) throws SQLException {
+            StringTokenizer tokenizer = new StringTokenizer(url == null ? "" : url.substring(PhoenixRuntime.JDBC_PROTOCOL.length()),DELIMITERS, true);
+            int i = 0;
+            boolean isMalformedUrl = false;
+            String[] tokens = new String[3];
+            String token = null;
+            while (tokenizer.hasMoreTokens() && !(token=tokenizer.nextToken()).equals(TERMINATOR) && tokenizer.hasMoreTokens() && i < tokens.length) {
+                token = tokenizer.nextToken();
+                // This would mean we have an empty string for a token which is illegal
+                if (DELIMITERS.contains(token)) {
+                    isMalformedUrl = true;
+                    break;
+                }
+                tokens[i++] = token;
+            }
+            Integer port = null;
+            if (!isMalformedUrl) {
+                if (tokenizer.hasMoreTokens() && !TERMINATOR.equals(token)) {
+                    isMalformedUrl = true;
+                } else if (i > 1) {
+                    try {
+                        port = Integer.parseInt(tokens[1]);
+                        isMalformedUrl = port < 0;
+                    } catch (NumberFormatException e) {
+                        // If we have 3 tokens, then the second one must be a port.
+                        // If we only have 2 tokens, the second one might be the root node:
+                        // Assume that is the case if we get a NumberFormatException
+                        if (! (isMalformedUrl = i == 3) ) {
+                            tokens[2] = tokens[1];
+                        }
+                        
+                    }
+                }
+            }
+            if (isMalformedUrl) {
+                throw new SQLExceptionInfo.Builder(SQLExceptionCode.MALFORMED_CONNECTION_URL)
+                .setMessage(url).build().buildException();
+            }
+            return new ConnectionInfo(tokens[0],port,tokens[2]);
+        }
+        
+        public ConnectionInfo normalize(ReadOnlyProps props) throws SQLException {
+            String zookeeperQuorum = this.getZookeeperQuorum();
+            Integer port = this.getPort();
+            String rootNode = this.getRootNode();
+            // Normalize connInfo so that a url explicitly specifying versus implicitly inheriting
+            // the default values will both share the same ConnectionQueryServices.
+            if (zookeeperQuorum == null) {
+                zookeeperQuorum = props.get(QueryServices.ZOOKEEPER_QUARUM_ATTRIB);
+                if (zookeeperQuorum == null) {
+                    throw new SQLExceptionInfo.Builder(SQLExceptionCode.MALFORMED_CONNECTION_URL)
+                    .setMessage(this.toString()).build().buildException();
+                }
+            }
+
+            if (port == null) {
+                if (!isConnectionless) {
+                    String portStr = props.get(QueryServices.ZOOKEEPER_PORT_ATTRIB);
+                    if (portStr != null) {
+                        try {
+                            port = Integer.parseInt(portStr);
+                        } catch (NumberFormatException e) {
+                            throw new SQLExceptionInfo.Builder(SQLExceptionCode.MALFORMED_CONNECTION_URL)
+                            .setMessage(this.toString()).build().buildException();
+                        }
+                    }
+                }
+            } else if (isConnectionless) {
+                throw new SQLExceptionInfo.Builder(SQLExceptionCode.MALFORMED_CONNECTION_URL)
+                .setMessage("Port may not be specified when using the connectionless url \"" + this.toString() + "\"").build().buildException();
+            }
+            if (rootNode == null) {
+                if (!isConnectionless) {
+                    rootNode = props.get(QueryServices.ZOOKEEPER_ROOT_NODE_ATTRIB);
+                }
+            } else if (isConnectionless) {
+                throw new SQLExceptionInfo.Builder(SQLExceptionCode.MALFORMED_CONNECTION_URL)
+                .setMessage("Root node may not be specified when using the connectionless url \"" + this.toString() + "\"").build().buildException();
+            }
+            return new ConnectionInfo(zookeeperQuorum, port, rootNode);
+        }
+        
+        private final Integer port;
+        private final String rootNode;
+        private final String zookeeperQuorum;
+        private final boolean isConnectionless;
+        
+        // used for testing
+        ConnectionInfo(String zookeeperQuorum, Integer port, String rootNode) {
+            this.zookeeperQuorum = zookeeperQuorum;
+            this.port = port;
+            this.rootNode = rootNode;
+            this.isConnectionless = PhoenixRuntime.CONNECTIONLESS.equals(zookeeperQuorum);
+        }
+
+        public ReadOnlyProps asProps() {
+            Map<String,String> connectionProps = Maps.newHashMapWithExpectedSize(3);
+            if (getZookeeperQuorum() != null) {
+                connectionProps.put(QueryServices.ZOOKEEPER_QUARUM_ATTRIB, getZookeeperQuorum());
+            }
+            if (getPort() != null) {
+                connectionProps.put(QueryServices.ZOOKEEPER_PORT_ATTRIB, getPort().toString());
+            }
+            if (getRootNode() != null) {
+                connectionProps.put(QueryServices.ZOOKEEPER_ROOT_NODE_ATTRIB, getRootNode());
+            }
+            return connectionProps.isEmpty() ? ReadOnlyProps.EMPTY_PROPS : new ReadOnlyProps(connectionProps.entrySet().iterator());
+        }
+        
+        public boolean isConnectionless() {
+            return isConnectionless;
+        }
+        
+        public String getZookeeperQuorum() {
+            return zookeeperQuorum;
+        }
+
+        public Integer getPort() {
+            return port;
+        }
+
+        public String getRootNode() {
+            return rootNode;
+        }
+
+        @Override
+        public int hashCode() {
+            final int prime = 31;
+            int result = 1;
+            result = prime * result + ((zookeeperQuorum == null) ? 0 : zookeeperQuorum.hashCode());
+            result = prime * result + ((port == null) ? 0 : port.hashCode());
+            result = prime * result + ((rootNode == null) ? 0 : rootNode.hashCode());
+            return result;
+        }
+
+        @Override
+        public boolean equals(Object obj) {
+            if (this == obj) return true;
+            if (obj == null) return false;
+            if (getClass() != obj.getClass()) return false;
+            ConnectionInfo other = (ConnectionInfo)obj;
+            if (zookeeperQuorum == null) {
+                if (other.zookeeperQuorum != null) return false;
+            } else if (!zookeeperQuorum.equals(other.zookeeperQuorum)) return false;
+            if (port == null) {
+                if (other.port != null) return false;
+            } else if (!port.equals(other.port)) return false;
+            if (rootNode == null) {
+                if (other.rootNode != null) return false;
+            } else if (!rootNode.equals(other.rootNode)) return false;
+            return true;
+        }
+        
+        @Override
+        public String toString() {
+            return zookeeperQuorum + (port == null ? "" : ":" + port) + (rootNode == null ? "" : ":" + rootNode);
+        }
+    }
+
+    public static boolean isTestUrl(String url) {
+        return url.endsWith(";test=true") || url.contains(";test=true;");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixParameterMetaData.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixParameterMetaData.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixParameterMetaData.java
new file mode 100644
index 0000000..830e054
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixParameterMetaData.java
@@ -0,0 +1,172 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.jdbc;
+
+import java.sql.ParameterMetaData;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+import org.apache.phoenix.parse.BindParseNode;
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.PDatum;
+import org.apache.phoenix.schema.TypeMismatchException;
+
+
+
+/**
+ * 
+ * Implementation of ParameterMetaData for Phoenix
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class PhoenixParameterMetaData implements ParameterMetaData {
+    private final PDatum[] params;
+    private static final PDatum EMPTY_DATUM = new PDatum() {
+        @Override
+        public boolean isNullable() {
+            return false;
+        }
+        
+        @Override
+        public Integer getScale() {
+            return null;
+        }
+        
+        @Override
+        public Integer getMaxLength() {
+            return null;
+        }
+        
+        @Override
+        public PDataType getDataType() {
+            return null;
+        }
+        
+        @Override
+        public ColumnModifier getColumnModifier() {
+            return null;
+        }
+        
+        @Override
+        public Integer getByteSize() {
+            return null;
+        }
+    };
+    public static final PhoenixParameterMetaData EMPTY_PARAMETER_META_DATA = new PhoenixParameterMetaData(0);
+    public PhoenixParameterMetaData(int paramCount) {
+        params = new PDatum[paramCount];
+        //initialize the params array with the empty_datum marker value.
+        for(int i = 0; i < paramCount; i++) {
+            params[i] = EMPTY_DATUM;
+        }
+    }
+ 
+    private PDatum getParam(int index) throws SQLException {
+        if (index <= 0 || index > params.length) {
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode.PARAM_INDEX_OUT_OF_BOUND)
+                .setMessage("The index is " + index + ". Must be between 1 and " + params.length)
+                .build().buildException();
+        }
+        PDatum param = params[index-1];
+        
+        if (param == EMPTY_DATUM) {
+            //value at params[index-1] was never set.
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode.PARAM_VALUE_UNBOUND)
+                .setMessage("Parameter at index " + index + " is unbound").build().buildException();
+        }
+        return param;
+    }
+    @Override
+    public String getParameterClassName(int index) throws SQLException {
+        PDatum datum = getParam(index);
+        PDataType type = datum == null ? null : datum.getDataType();
+        return type == null ? null : type.getJavaClassName();
+    }
+
+    @Override
+    public int getParameterCount() throws SQLException {
+        return params.length;
+    }
+
+    @Override
+    public int getParameterMode(int index) throws SQLException {
+        return ParameterMetaData.parameterModeIn;
+    }
+
+    @Override
+    public int getParameterType(int index) throws SQLException {
+        return getParam(index).getDataType().getSqlType();
+    }
+
+    @Override
+    public String getParameterTypeName(int index) throws SQLException {
+        return getParam(index).getDataType().getSqlTypeName();
+    }
+
+    @Override
+    public int getPrecision(int index) throws SQLException {
+        return 0;
+    }
+
+    @Override
+    public int getScale(int index) throws SQLException {
+        return 0;
+    }
+
+    @Override
+    public int isNullable(int index) throws SQLException {
+        return getParam(index).isNullable() ? ResultSetMetaData.columnNullable : ResultSetMetaData.columnNoNulls;
+    }
+
+    @Override
+    public boolean isSigned(int index) throws SQLException {
+        @SuppressWarnings("rawtypes")
+		Class clazz = getParam(index).getDataType().getJavaClass();
+        return Number.class.isInstance(clazz);
+    }
+
+    @Override
+    public boolean isWrapperFor(Class<?> iface) throws SQLException {
+        return iface.isInstance(this);
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public <T> T unwrap(Class<T> iface) throws SQLException {
+        if (!iface.isInstance(this)) {
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode.CLASS_NOT_UNWRAPPABLE)
+                .setMessage(this.getClass().getName() + " not unwrappable from " + iface.getName())
+                .build().buildException();
+        }
+        return (T)this;
+    }
+
+    public void addParam(BindParseNode bind, PDatum datum) throws SQLException {
+        PDatum bindDatum = params[bind.getIndex()];
+        if (bindDatum != null && bindDatum.getDataType() != null && !datum.getDataType().isCoercibleTo(bindDatum.getDataType())) {
+            throw TypeMismatchException.newException(datum.getDataType(), bindDatum.getDataType());
+        }
+        params[bind.getIndex()] = datum;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixPreparedStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixPreparedStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixPreparedStatement.java
new file mode 100644
index 0000000..8582e74
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixPreparedStatement.java
@@ -0,0 +1,469 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.jdbc;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.Reader;
+import java.math.BigDecimal;
+import java.net.URL;
+import java.sql.Array;
+import java.sql.Blob;
+import java.sql.Clob;
+import java.sql.Date;
+import java.sql.NClob;
+import java.sql.ParameterMetaData;
+import java.sql.PreparedStatement;
+import java.sql.Ref;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.RowId;
+import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
+import java.sql.SQLXML;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.BitSet;
+import java.util.Calendar;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.phoenix.compile.BindManager;
+import org.apache.phoenix.compile.QueryPlan;
+import org.apache.phoenix.compile.StatementPlan;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.util.DateUtil;
+import org.apache.phoenix.util.SQLCloseable;
+
+/**
+ * JDBC PreparedStatement implementation of Phoenix. Currently only the following methods (in addition to the ones
+ * supported on {@link PhoenixStatement} are supported: - {@link #executeQuery()} - {@link #setInt(int, int)} -
+ * {@link #setShort(int, short)} - {@link #setLong(int, long)} - {@link #setFloat(int, float)} -
+ * {@link #setDouble(int, double)} - {@link #setBigDecimal(int, BigDecimal)} - {@link #setString(int, String)} -
+ * {@link #setDate(int, Date)} - {@link #setDate(int, Date, Calendar)} - {@link #setTime(int, Time)} -
+ * {@link #setTime(int, Time, Calendar)} - {@link #setTimestamp(int, Timestamp)} -
+ * {@link #setTimestamp(int, Timestamp, Calendar)} - {@link #setNull(int, int)} - {@link #setNull(int, int, String)} -
+ * {@link #setBytes(int, byte[])} - {@link #clearParameters()} - {@link #getMetaData()}
+ * 
+ * @author jtaylor
+ * @since 0.1
+ */
+public class PhoenixPreparedStatement extends PhoenixStatement implements PreparedStatement, SQLCloseable {
+    private final List<Object> parameters;
+    private final ExecutableStatement statement;
+
+    private final String query;
+
+    public PhoenixPreparedStatement(PhoenixConnection connection, PhoenixStatementParser parser) throws SQLException,
+            IOException {
+        super(connection);
+        this.statement = parser.nextStatement(new ExecutableNodeFactory());
+        if (this.statement == null) { throw new EOFException(); }
+        this.query = null; // TODO: add toString on SQLStatement
+        this.parameters = Arrays.asList(new Object[statement.getBindCount()]);
+        Collections.fill(parameters, BindManager.UNBOUND_PARAMETER);
+    }
+
+    public PhoenixPreparedStatement(PhoenixConnection connection, String query) throws SQLException {
+        super(connection);
+        this.query = query;
+        this.statement = parseStatement(query);
+        this.parameters = Arrays.asList(new Object[statement.getBindCount()]);
+        Collections.fill(parameters, BindManager.UNBOUND_PARAMETER);
+    }
+
+    @Override
+    public void addBatch() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void clearParameters() throws SQLException {
+        Collections.fill(parameters, BindManager.UNBOUND_PARAMETER);
+    }
+
+    @Override
+    public List<Object> getParameters() {
+        return parameters;
+    }
+
+    private void throwIfUnboundParameters() throws SQLException {
+        int i = 0;
+        for (Object param : getParameters()) {
+            if (param == BindManager.UNBOUND_PARAMETER) {
+                throw new SQLExceptionInfo.Builder(SQLExceptionCode.PARAM_VALUE_UNBOUND)
+                    .setMessage("Parameter " + (i + 1) + " is unbound").build().buildException();
+            }
+            i++;
+        }
+    }
+    
+    @Override
+    public boolean execute() throws SQLException {
+        throwIfUnboundParameters();
+        try {
+            return statement.execute();
+        } catch (RuntimeException e) {
+            // FIXME: Expression.evaluate does not throw SQLException
+            // so this will unwrap throws from that.
+            if (e.getCause() instanceof SQLException) {
+                throw (SQLException) e.getCause();
+            }
+            throw e;
+        }
+    }
+
+    @Override
+    public ResultSet executeQuery() throws SQLException {
+        throwIfUnboundParameters();
+        try {
+            return statement.executeQuery();
+        } catch (RuntimeException e) {
+            // FIXME: Expression.evaluate does not throw SQLException
+            // so this will unwrap throws from that.
+            if (e.getCause() instanceof SQLException) {
+                throw (SQLException) e.getCause();
+            }
+            throw e;
+        }
+    }
+
+    @Override
+    public int executeUpdate() throws SQLException {
+        throwIfUnboundParameters();
+        try {
+            return statement.executeUpdate();
+        } catch (RuntimeException e) {
+            // FIXME: Expression.evaluate does not throw SQLException
+            // so this will unwrap throws from that.
+            if (e.getCause() instanceof SQLException) {
+                throw (SQLException) e.getCause();
+            }
+            throw e;
+        }
+    }
+
+    public QueryPlan optimizeQuery() throws SQLException {
+        throwIfUnboundParameters();
+        return (QueryPlan)statement.optimizePlan();
+    }
+
+    @Override
+    public ResultSetMetaData getMetaData() throws SQLException {
+        int paramCount = statement.getBindCount();
+        List<Object> params = this.getParameters();
+        BitSet unsetParams = new BitSet(statement.getBindCount());
+        for (int i = 0; i < paramCount; i++) {
+            if ( params.get(i) == BindManager.UNBOUND_PARAMETER) {
+                unsetParams.set(i);
+                params.set(i, null);
+            }
+        }
+        try {
+            return statement.getResultSetMetaData();
+        } finally {
+            int lastSetBit = 0;
+            while ((lastSetBit = unsetParams.nextSetBit(lastSetBit)) != -1) {
+                params.set(lastSetBit, BindManager.UNBOUND_PARAMETER);
+                lastSetBit++;
+            }
+        }
+    }
+
+    @Override
+    public ParameterMetaData getParameterMetaData() throws SQLException {
+        int paramCount = statement.getBindCount();
+        List<Object> params = this.getParameters();
+        BitSet unsetParams = new BitSet(statement.getBindCount());
+        for (int i = 0; i < paramCount; i++) {
+            if ( params.get(i) == BindManager.UNBOUND_PARAMETER) {
+                unsetParams.set(i);
+                params.set(i, null);
+            }
+        }
+        try {
+            StatementPlan plan = statement.compilePlan();
+            return plan.getParameterMetaData();
+        } finally {
+            int lastSetBit = 0;
+            while ((lastSetBit = unsetParams.nextSetBit(lastSetBit)) != -1) {
+                params.set(lastSetBit, BindManager.UNBOUND_PARAMETER);
+                lastSetBit++;
+            }
+        }
+    }
+
+    @Override
+    public String toString() {
+        return query;
+    }
+
+    @Override
+    public void setArray(int parameterIndex, Array x) throws SQLException {
+    	parameters.set(parameterIndex - 1, x);
+    }
+
+    @Override
+    public void setAsciiStream(int parameterIndex, InputStream x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void setAsciiStream(int parameterIndex, InputStream x, int length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void setAsciiStream(int parameterIndex, InputStream x, long length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void setBigDecimal(int parameterIndex, BigDecimal x) throws SQLException {
+        parameters.set(parameterIndex - 1, x);
+    }
+
+    @Override
+    public void setBytes(int parameterIndex, byte[] x) throws SQLException {
+        parameters.set(parameterIndex - 1, x);
+    }
+
+    @Override
+    public void setBinaryStream(int parameterIndex, InputStream x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void setBinaryStream(int parameterIndex, InputStream x, int length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void setBinaryStream(int parameterIndex, InputStream x, long length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void setBlob(int parameterIndex, Blob x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void setBlob(int parameterIndex, InputStream inputStream) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void setBlob(int parameterIndex, InputStream inputStream, long length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void setBoolean(int parameterIndex, boolean x) throws SQLException {
+        parameters.set(parameterIndex - 1, x);
+    }
+
+    @Override
+    public void setByte(int parameterIndex, byte x) throws SQLException {
+        parameters.set(parameterIndex - 1, x);
+    }
+
+    @Override
+    public void setCharacterStream(int parameterIndex, Reader reader) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void setCharacterStream(int parameterIndex, Reader reader, int length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void setCharacterStream(int parameterIndex, Reader reader, long length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void setClob(int parameterIndex, Clob x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void setClob(int parameterIndex, Reader reader) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void setClob(int parameterIndex, Reader reader, long length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void setDate(int parameterIndex, Date x) throws SQLException {
+        parameters.set(parameterIndex - 1, x);
+    }
+
+    @Override
+    public void setDate(int parameterIndex, Date x, Calendar cal) throws SQLException {
+        cal.setTime(x);
+        parameters.set(parameterIndex - 1, new Date(cal.getTimeInMillis()));
+    }
+
+    @Override
+    public void setDouble(int parameterIndex, double x) throws SQLException {
+//        parameters.set(parameterIndex - 1, BigDecimal.valueOf(x));
+        parameters.set(parameterIndex - 1, x);
+    }
+
+    @Override
+    public void setFloat(int parameterIndex, float x) throws SQLException {
+//        parameters.set(parameterIndex - 1, BigDecimal.valueOf(x));
+        parameters.set(parameterIndex - 1, x);
+    }
+
+    @Override
+    public void setInt(int parameterIndex, int x) throws SQLException {
+        parameters.set(parameterIndex - 1, x);
+    }
+
+    @Override
+    public void setLong(int parameterIndex, long x) throws SQLException {
+        parameters.set(parameterIndex - 1, x);
+    }
+
+    @Override
+    public void setNCharacterStream(int parameterIndex, Reader value) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void setNCharacterStream(int parameterIndex, Reader value, long length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void setNClob(int parameterIndex, NClob value) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void setNClob(int parameterIndex, Reader reader) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void setNClob(int parameterIndex, Reader reader, long length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void setNString(int parameterIndex, String value) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void setNull(int parameterIndex, int sqlType) throws SQLException {
+        parameters.set(parameterIndex - 1, null);
+    }
+
+    @Override
+    public void setNull(int parameterIndex, int sqlType, String typeName) throws SQLException {
+        parameters.set(parameterIndex - 1, null);
+    }
+
+    @Override
+    public void setObject(int parameterIndex, Object o) throws SQLException {
+        parameters.set(parameterIndex - 1, o);
+    }
+
+    @Override
+    public void setObject(int parameterIndex, Object o, int targetSqlType) throws SQLException {
+        PDataType targetType = PDataType.fromTypeId(targetSqlType);
+        PDataType sourceType = PDataType.fromLiteral(o);
+        o = targetType.toObject(o, sourceType);
+        parameters.set(parameterIndex - 1, o);
+    }
+
+    @Override
+    public void setObject(int parameterIndex, Object x, int targetSqlType, int scaleOrLength) throws SQLException {
+        setObject(parameterIndex, x, targetSqlType);
+    }
+
+    @Override
+    public void setRef(int parameterIndex, Ref x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void setRowId(int parameterIndex, RowId x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void setSQLXML(int parameterIndex, SQLXML xmlObject) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void setShort(int parameterIndex, short x) throws SQLException {
+        parameters.set(parameterIndex - 1, x);
+    }
+
+    @Override
+    public void setString(int parameterIndex, String x) throws SQLException {
+        parameters.set(parameterIndex - 1, x);
+    }
+
+    @Override
+    public void setTime(int parameterIndex, Time x) throws SQLException {
+        parameters.set(parameterIndex - 1, x);
+    }
+
+    @Override
+    public void setTime(int parameterIndex, Time x, Calendar cal) throws SQLException {
+        cal.setTime(x);
+        parameters.set(parameterIndex - 1, new Time(cal.getTimeInMillis()));
+    }
+
+    @Override
+    public void setTimestamp(int parameterIndex, Timestamp x) throws SQLException {
+        parameters.set(parameterIndex - 1, x);
+    }
+
+    @Override
+    public void setTimestamp(int parameterIndex, Timestamp x, Calendar cal) throws SQLException {
+        cal.setTime(x);
+        parameters.set(parameterIndex - 1,  DateUtil.getTimestamp(cal.getTimeInMillis(), x.getNanos()));
+    }
+
+    @Override
+    public void setURL(int parameterIndex, URL x) throws SQLException {
+        parameters.set(parameterIndex - 1, x.toExternalForm()); // Just treat as String
+    }
+
+    @Override
+    public void setUnicodeStream(int parameterIndex, InputStream x, int length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..a5807af
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixResultSet.java
@@ -0,0 +1,1240 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.jdbc;
+
+import java.io.InputStream;
+import java.io.Reader;
+import java.math.BigDecimal;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.sql.Array;
+import java.sql.Blob;
+import java.sql.Clob;
+import java.sql.Date;
+import java.sql.NClob;
+import java.sql.Ref;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.RowId;
+import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
+import java.sql.SQLWarning;
+import java.sql.SQLXML;
+import java.sql.Statement;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.text.Format;
+import java.util.Calendar;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.compile.ColumnProjector;
+import org.apache.phoenix.compile.RowProjector;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+import org.apache.phoenix.iterate.ResultIterator;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.ResultTuple;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.DateUtil;
+import org.apache.phoenix.util.SQLCloseable;
+
+
+
+/**
+ * 
+ * JDBC ResultSet implementation of Phoenix.
+ * Currently only the following data types are supported:
+ * - String
+ * - Date
+ * - Time
+ * - Timestamp
+ * - BigDecimal
+ * - Double
+ * - Float
+ * - Int
+ * - Short
+ * - Long
+ * - Binary
+ * - Array - 1D
+ * None of the update or delete methods are supported.
+ * The ResultSet only supports the following options:
+ * - ResultSet.FETCH_FORWARD
+ * - ResultSet.CONCUR_READ_ONLY
+ * - ResultSet.TYPE_FORWARD_ONLY
+ * - ResultSet.CLOSE_CURSORS_AT_COMMIT
+ * 
+ * @author jtaylor
+ * @since 0.1
+ */
+public class PhoenixResultSet implements ResultSet, SQLCloseable, org.apache.phoenix.jdbc.Jdbc7Shim.ResultSet {
+    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 ResultIterator scanner;
+    private final RowProjector rowProjector;
+    private final PhoenixStatement statement;
+    private final ImmutableBytesWritable ptr = new ImmutableBytesWritable();
+    
+    private Tuple currentRow = BEFORE_FIRST;
+    private boolean isClosed = false;
+    private boolean wasNull = false;
+    
+    public PhoenixResultSet(ResultIterator resultIterator, RowProjector rowProjector, PhoenixStatement statement) throws SQLException {
+        this.rowProjector = rowProjector;
+        this.scanner = resultIterator;
+        this.statement = statement;
+    }
+    
+    @Override
+    public boolean absolute(int row) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void afterLast() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void beforeFirst() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void cancelRowUpdates() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void clearWarnings() throws SQLException {
+    }
+
+    @Override
+    public void close() throws SQLException {
+        if (isClosed) {
+            return;
+        }
+        try {
+            scanner.close();
+        } finally {
+            isClosed = true;
+            statement.getResultSets().remove(this);
+        }
+    }
+
+    @Override
+    public void deleteRow() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public int findColumn(String columnLabel) throws SQLException {
+        Integer index = rowProjector.getColumnIndex(columnLabel);
+        return index + 1;
+    }
+
+    @Override
+    public boolean first() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public Array getArray(int columnIndex) throws SQLException {
+    	checkCursorState();
+        // Get the value using the expected type instead of trying to coerce to VARCHAR.
+        // We can't coerce using our formatter because we don't have enough context in PDataType.
+    	ColumnProjector projector = rowProjector.getColumnProjector(columnIndex-1);
+        Array value = (Array)projector.getValue(currentRow, projector.getExpression().getDataType(), ptr);
+        wasNull = (value == null);
+        return value;
+    }
+
+    @Override
+    public Array getArray(String columnLabel) throws SQLException {
+        return getArray(findColumn(columnLabel));
+    }
+
+    @Override
+    public InputStream getAsciiStream(int columnIndex) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public InputStream getAsciiStream(String columnLabel) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    private void checkOpen() throws SQLException {
+        if (isClosed) {
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode.RESULTSET_CLOSED).build().buildException();
+        }
+    }
+
+    private void checkCursorState() throws SQLException {
+        checkOpen();
+        if (currentRow == BEFORE_FIRST) {
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode.CURSOR_BEFORE_FIRST_ROW).build().buildException();
+        }else if (currentRow == null) {
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode.CURSOR_PAST_LAST_ROW).build().buildException();
+        }
+    }
+
+    @Override
+    public BigDecimal getBigDecimal(int columnIndex) throws SQLException {
+        checkCursorState();
+        BigDecimal value = (BigDecimal)rowProjector.getColumnProjector(columnIndex-1).getValue(currentRow, PDataType.DECIMAL, ptr);
+        wasNull = (value == null);
+        return value;
+    }
+
+    @Override
+    public BigDecimal getBigDecimal(String columnLabel) throws SQLException {
+        return getBigDecimal(findColumn(columnLabel));
+    }
+
+    @Override
+    public BigDecimal getBigDecimal(int columnIndex, int scale) throws SQLException {
+        BigDecimal value = getBigDecimal(columnIndex);
+        return value.setScale(scale);
+    }
+
+    @Override
+    public BigDecimal getBigDecimal(String columnLabel, int scale) throws SQLException {
+        return getBigDecimal(findColumn(columnLabel), scale);
+    }
+
+    @Override
+    public InputStream getBinaryStream(int columnIndex) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public InputStream getBinaryStream(String columnLabel) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public Blob getBlob(int columnIndex) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public Blob getBlob(String columnLabel) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public boolean getBoolean(int columnIndex) throws SQLException {
+        checkCursorState();
+        ColumnProjector colProjector = rowProjector.getColumnProjector(columnIndex-1);
+        PDataType type = colProjector.getExpression().getDataType();
+        Object value = colProjector.getValue(currentRow, type, ptr);
+        wasNull = (value == null);
+        if (value == null) {
+            return false;
+        }
+        switch(type) {
+        case BOOLEAN:
+            return Boolean.TRUE.equals(value);
+        case VARCHAR:
+            return !STRING_FALSE.equals(value);
+        case INTEGER:
+            return !INTEGER_FALSE.equals(value);
+        case DECIMAL:
+            return !BIG_DECIMAL_FALSE.equals(value);
+        default:
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_CALL_METHOD_ON_TYPE)
+                .setMessage("Method: getBoolean; Type:" + type).build().buildException();
+        }
+    }
+
+    @Override
+    public boolean getBoolean(String columnLabel) throws SQLException {
+        return getBoolean(findColumn(columnLabel));
+    }
+
+    @Override
+    public byte[] getBytes(int columnIndex) throws SQLException {
+        checkCursorState();
+        byte[] value = (byte[])rowProjector.getColumnProjector(columnIndex-1).getValue(currentRow, PDataType.VARBINARY, ptr);
+        wasNull = (value == null);
+        return value;
+    }
+
+    @Override
+    public byte[] getBytes(String columnLabel) throws SQLException {
+        return getBytes(findColumn(columnLabel));
+    }
+
+    @Override
+    public byte getByte(int columnIndex) throws SQLException {
+//        throw new SQLFeatureNotSupportedException();
+        checkCursorState();
+        Byte value = (Byte)rowProjector.getColumnProjector(columnIndex-1).getValue(currentRow, PDataType.TINYINT, ptr);
+        wasNull = (value == null);
+        if (value == null) {
+            return 0;
+        }
+        return value;
+    }
+
+    @Override
+    public byte getByte(String columnLabel) throws SQLException {
+        return getByte(findColumn(columnLabel));
+    }
+
+    @Override
+    public Reader getCharacterStream(int columnIndex) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public Reader getCharacterStream(String columnLabel) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public Clob getClob(int columnIndex) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public Clob getClob(String columnLabel) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public int getConcurrency() throws SQLException {
+        return ResultSet.CONCUR_READ_ONLY;
+    }
+
+    @Override
+    public String getCursorName() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public Date getDate(int columnIndex) throws SQLException {
+        checkCursorState();
+        Date value = (Date)rowProjector.getColumnProjector(columnIndex-1).getValue(currentRow, PDataType.DATE, ptr);
+        wasNull = (value == null);
+        if (value == null) {
+            return null;
+        }
+        return value;
+    }
+
+    @Override
+    public Date getDate(String columnLabel) throws SQLException {
+        return getDate(findColumn(columnLabel));
+    }
+
+    @Override
+    public Date getDate(int columnIndex, Calendar cal) throws SQLException {
+        checkCursorState();
+        Date value = (Date)rowProjector.getColumnProjector(columnIndex-1).getValue(currentRow, PDataType.DATE, ptr);
+        cal.setTime(value);
+        return new Date(cal.getTimeInMillis());
+    }
+
+    @Override
+    public Date getDate(String columnLabel, Calendar cal) throws SQLException {
+        return getDate(findColumn(columnLabel), cal);
+    }
+
+    @Override
+    public double getDouble(int columnIndex) throws SQLException {
+        checkCursorState();
+        Double value = (Double)rowProjector.getColumnProjector(columnIndex-1).getValue(currentRow, PDataType.DOUBLE, ptr);
+        wasNull = (value == null);
+        if (value == null) {
+            return 0;
+        }
+        return value;
+    }
+
+    @Override
+    public double getDouble(String columnLabel) throws SQLException {
+        return getDouble(findColumn(columnLabel));
+    }
+
+    @Override
+    public int getFetchDirection() throws SQLException {
+        return ResultSet.FETCH_FORWARD;
+    }
+
+    @Override
+    public int getFetchSize() throws SQLException {
+        return statement.getFetchSize();
+    }
+
+    @Override
+    public float getFloat(int columnIndex) throws SQLException {
+        checkCursorState();
+        Float value = (Float)rowProjector.getColumnProjector(columnIndex-1).getValue(currentRow, PDataType.FLOAT, ptr);
+        wasNull = (value == null);
+        if (value == null) {
+            return 0;
+        }
+        return value;
+    }
+
+    @Override
+    public float getFloat(String columnLabel) throws SQLException {
+        return getFloat(findColumn(columnLabel));
+    }
+
+    @Override
+    public int getHoldability() throws SQLException {
+        return ResultSet.CLOSE_CURSORS_AT_COMMIT;
+    }
+
+    @Override
+    public int getInt(int columnIndex) throws SQLException {
+        checkCursorState();
+        Integer value = (Integer)rowProjector.getColumnProjector(columnIndex-1).getValue(currentRow, PDataType.INTEGER, ptr);
+        wasNull = (value == null);
+        if (value == null) {
+            return 0;
+        }
+        return value;
+    }
+
+    @Override
+    public int getInt(String columnLabel) throws SQLException {
+        return getInt(findColumn(columnLabel));
+    }
+
+    @Override
+    public long getLong(int columnIndex) throws SQLException {
+        checkCursorState();
+        Long value = (Long)rowProjector.getColumnProjector(columnIndex-1).getValue(currentRow, PDataType.LONG, ptr);
+        wasNull = (value == null);
+        if (value == null) {
+            return 0;
+        }
+        return value;
+    }
+
+    @Override
+    public long getLong(String columnLabel) throws SQLException {
+        return getLong(findColumn(columnLabel));
+    }
+
+    @Override
+    public ResultSetMetaData getMetaData() throws SQLException {
+        return new PhoenixResultSetMetaData(statement.getConnection(), rowProjector);
+    }
+
+    @Override
+    public Reader getNCharacterStream(int columnIndex) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public Reader getNCharacterStream(String columnLabel) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public NClob getNClob(int columnIndex) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public NClob getNClob(String columnLabel) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public String getNString(int columnIndex) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public String getNString(String columnLabel) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public Object getObject(int columnIndex) throws SQLException {
+        checkCursorState();
+        ColumnProjector projector = rowProjector.getColumnProjector(columnIndex-1);
+        Object value = projector.getValue(currentRow, projector.getExpression().getDataType(), ptr);
+        wasNull = (value == null);
+        return value;
+    }
+
+    @Override
+    public Object getObject(String columnLabel) throws SQLException {
+        return getObject(findColumn(columnLabel));
+    }
+
+    @Override
+    public Object getObject(int columnIndex, Map<String, Class<?>> map) throws SQLException {
+        return getObject(columnIndex); // Just ignore map since we only support built-in types
+    }
+
+    @Override
+    public Object getObject(String columnLabel, Map<String, Class<?>> map) throws SQLException {
+        return getObject(findColumn(columnLabel), map);
+    }
+
+    @Override
+    public Ref getRef(int columnIndex) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public Ref getRef(String columnLabel) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public int getRow() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public RowId getRowId(int columnIndex) throws SQLException {
+        // TODO: support?
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public RowId getRowId(String columnLabel) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public SQLXML getSQLXML(int columnIndex) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public SQLXML getSQLXML(String columnLabel) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public short getShort(int columnIndex) throws SQLException {
+        checkCursorState();
+        Short value = (Short)rowProjector.getColumnProjector(columnIndex-1).getValue(currentRow, PDataType.SMALLINT, ptr);
+        wasNull = (value == null);
+        if (value == null) {
+            return 0;
+        }
+        return value;
+    }
+
+    @Override
+    public short getShort(String columnLabel) throws SQLException {
+        return getShort(findColumn(columnLabel));
+    }
+
+    @Override
+    public Statement getStatement() throws SQLException {
+        return statement;
+    }
+
+    @Override
+    public String getString(int columnIndex) throws SQLException {
+        checkCursorState();
+        // Get the value using the expected type instead of trying to coerce to VARCHAR.
+        // We can't coerce using our formatter because we don't have enough context in PDataType.
+        ColumnProjector projector = rowProjector.getColumnProjector(columnIndex-1);
+        PDataType type = projector.getExpression().getDataType();
+        Object value = projector.getValue(currentRow,type, ptr);
+        if (wasNull = (value == null)) {
+            return null;
+        }
+        // Run Object through formatter to get String.
+        // This provides a simple way of getting a reasonable string representation
+        // for types like DATE and TIME
+        Format formatter = statement.getFormatter(type);
+        return formatter == null ? value.toString() : formatter.format(value);
+    }
+
+    @Override
+    public String getString(String columnLabel) throws SQLException {
+        return getString(findColumn(columnLabel));
+    }
+
+    @Override
+    public Time getTime(int columnIndex) throws SQLException {
+        checkCursorState();
+        Time value = (Time)rowProjector.getColumnProjector(columnIndex-1).getValue(currentRow, PDataType.TIME, ptr);
+        wasNull = (value == null);
+        return value;
+    }
+
+    @Override
+    public Time getTime(String columnLabel) throws SQLException {
+        return getTime(findColumn(columnLabel));
+    }
+
+    @Override
+    public Time getTime(int columnIndex, Calendar cal) throws SQLException {
+        checkCursorState();
+        Time value = (Time)rowProjector.getColumnProjector(columnIndex-1).getValue(currentRow, PDataType.TIME, ptr);
+        wasNull = (value == null);
+        if (value == null) {
+            return null;
+        }
+        cal.setTime(value);
+        value.setTime(cal.getTimeInMillis());
+        return value;
+    }
+
+    @Override
+    public Time getTime(String columnLabel, Calendar cal) throws SQLException {
+        return getTime(findColumn(columnLabel),cal);
+    }
+
+    @Override
+    public Timestamp getTimestamp(int columnIndex) throws SQLException {
+        checkCursorState();
+        Timestamp value = (Timestamp)rowProjector.getColumnProjector(columnIndex-1).getValue(currentRow, PDataType.TIMESTAMP, ptr);
+        wasNull = (value == null);
+        return value;
+    }
+
+    @Override
+    public Timestamp getTimestamp(String columnLabel) throws SQLException {
+        return getTimestamp(findColumn(columnLabel));
+    }
+
+    @Override
+    public Timestamp getTimestamp(int columnIndex, Calendar cal) throws SQLException {
+        checkCursorState();
+        Timestamp value = (Timestamp)rowProjector.getColumnProjector(columnIndex-1).getValue(currentRow, PDataType.TIMESTAMP, ptr);
+        wasNull = (value == null);
+        if (value == null) {
+            return null;
+        }
+        cal.setTime(value); //this resets the millisecond part of timestamp according to the time zone of the calendar.
+        return DateUtil.getTimestamp(cal.getTimeInMillis(), value.getNanos());
+    }
+
+    @Override
+    public Timestamp getTimestamp(String columnLabel, Calendar cal) throws SQLException {
+        return getTimestamp(findColumn(columnLabel),cal);
+    }
+
+    @Override
+    public int getType() throws SQLException {
+        return ResultSet.TYPE_FORWARD_ONLY;
+    }
+
+    @Override
+    public URL getURL(int columnIndex) throws SQLException {
+        checkCursorState();
+        String value = (String)rowProjector.getColumnProjector(columnIndex-1).getValue(currentRow, PDataType.VARCHAR, ptr);
+        wasNull = (value == null);
+        if (value == null) {
+            return null;
+        }
+        try {
+            return new URL(value);
+        } catch (MalformedURLException e) {
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode.MALFORMED_URL).setRootCause(e).build().buildException();
+        }
+    }
+
+    @Override
+    public URL getURL(String columnLabel) throws SQLException {
+        return getURL(findColumn(columnLabel));
+    }
+
+    @Override
+    public InputStream getUnicodeStream(int columnIndex) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public InputStream getUnicodeStream(String columnLabel) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public SQLWarning getWarnings() throws SQLException {
+        return null;
+    }
+
+    @Override
+    public void insertRow() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public boolean isAfterLast() throws SQLException {
+        return currentRow == null;
+    }
+
+    @Override
+    public boolean isBeforeFirst() throws SQLException {
+        return currentRow == BEFORE_FIRST;
+    }
+
+    @Override
+    public boolean isClosed() throws SQLException {
+        return isClosed;
+    }
+
+    @Override
+    public boolean isFirst() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public boolean isLast() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public boolean last() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void moveToCurrentRow() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void moveToInsertRow() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public boolean next() throws SQLException {
+        checkOpen();
+        try {
+            currentRow = scanner.next();
+        } catch (RuntimeException e) {
+            // FIXME: Expression.evaluate does not throw SQLException
+            // so this will unwrap throws from that.
+            if (e.getCause() instanceof SQLException) {
+                throw (SQLException) e.getCause();
+            }
+            throw e;
+        }
+        return currentRow != null;
+    }
+
+    @Override
+    public boolean previous() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void refreshRow() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public boolean relative(int rows) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public boolean rowDeleted() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public boolean rowInserted() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public boolean rowUpdated() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void setFetchDirection(int direction) throws SQLException {
+        if (direction != ResultSet.FETCH_FORWARD) {
+            throw new SQLFeatureNotSupportedException();
+        }
+    }
+
+    @Override
+    public void setFetchSize(int rows) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateArray(int columnIndex, Array x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateArray(String columnLabel, Array x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateAsciiStream(int columnIndex, InputStream x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateAsciiStream(String columnLabel, InputStream x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateAsciiStream(int columnIndex, InputStream x, int length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateAsciiStream(String columnLabel, InputStream x, int length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateAsciiStream(int columnIndex, InputStream x, long length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateAsciiStream(String columnLabel, InputStream x, long length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateBigDecimal(int columnIndex, BigDecimal x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateBigDecimal(String columnLabel, BigDecimal x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateBinaryStream(int columnIndex, InputStream x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateBinaryStream(String columnLabel, InputStream x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateBinaryStream(int columnIndex, InputStream x, int length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateBinaryStream(String columnLabel, InputStream x, int length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateBinaryStream(int columnIndex, InputStream x, long length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateBinaryStream(String columnLabel, InputStream x, long length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateBlob(int columnIndex, Blob x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateBlob(String columnLabel, Blob x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateBlob(int columnIndex, InputStream inputStream) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateBlob(String columnLabel, InputStream inputStream) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateBlob(int columnIndex, InputStream inputStream, long length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateBlob(String columnLabel, InputStream inputStream, long length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateBoolean(int columnIndex, boolean x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateBoolean(String columnLabel, boolean x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateByte(int columnIndex, byte x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateByte(String columnLabel, byte x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateBytes(int columnIndex, byte[] x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateBytes(String columnLabel, byte[] x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateCharacterStream(int columnIndex, Reader x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateCharacterStream(String columnLabel, Reader reader) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateCharacterStream(int columnIndex, Reader x, int length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateCharacterStream(String columnLabel, Reader reader, int length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateCharacterStream(int columnIndex, Reader x, long length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateCharacterStream(String columnLabel, Reader reader, long length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateClob(int columnIndex, Clob x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateClob(String columnLabel, Clob x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateClob(int columnIndex, Reader reader) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateClob(String columnLabel, Reader reader) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateClob(int columnIndex, Reader reader, long length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateClob(String columnLabel, Reader reader, long length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateDate(int columnIndex, Date x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateDate(String columnLabel, Date x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateDouble(int columnIndex, double x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateDouble(String columnLabel, double x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateFloat(int columnIndex, float x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateFloat(String columnLabel, float x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateInt(int columnIndex, int x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateInt(String columnLabel, int x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateLong(int columnIndex, long x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateLong(String columnLabel, long x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateNCharacterStream(int columnIndex, Reader x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateNCharacterStream(String columnLabel, Reader reader) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateNCharacterStream(int columnIndex, Reader x, long length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateNCharacterStream(String columnLabel, Reader reader, long length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateNClob(int columnIndex, NClob nClob) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateNClob(String columnLabel, NClob nClob) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateNClob(int columnIndex, Reader reader) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateNClob(String columnLabel, Reader reader) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateNClob(int columnIndex, Reader reader, long length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateNClob(String columnLabel, Reader reader, long length) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateNString(int columnIndex, String nString) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateNString(String columnLabel, String nString) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateNull(int columnIndex) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateNull(String columnLabel) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateObject(int columnIndex, Object x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateObject(String columnLabel, Object x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateObject(int columnIndex, Object x, int scaleOrLength) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateObject(String columnLabel, Object x, int scaleOrLength) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateRef(int columnIndex, Ref x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateRef(String columnLabel, Ref x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateRow() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateRowId(int columnIndex, RowId x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateRowId(String columnLabel, RowId x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateSQLXML(int columnIndex, SQLXML xmlObject) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateSQLXML(String columnLabel, SQLXML xmlObject) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateShort(int columnIndex, short x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateShort(String columnLabel, short x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateString(int columnIndex, String x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateString(String columnLabel, String x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateTime(int columnIndex, Time x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateTime(String columnLabel, Time x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateTimestamp(int columnIndex, Timestamp x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void updateTimestamp(String columnLabel, Timestamp x) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public boolean wasNull() throws SQLException {
+        return wasNull;
+    }
+
+    @Override
+    public boolean isWrapperFor(Class<?> iface) throws SQLException {
+        return iface.isInstance(this);
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public <T> T unwrap(Class<T> iface) throws SQLException {
+        if (!iface.isInstance(this)) {
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode.CLASS_NOT_UNWRAPPABLE)
+                .setMessage(this.getClass().getName() + " not unwrappable from " + iface.getName())
+                .build().buildException();
+        }
+        return (T)this;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public <T> T getObject(int columnIndex, Class<T> type) throws SQLException {
+        return (T) getObject(columnIndex); // Just ignore type since we only support built-in types
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public <T> T getObject(String columnLabel, Class<T> type) throws SQLException {
+        return (T) getObject(columnLabel); // Just ignore type since we only support built-in types
+    }
+
+    @Override
+    public String toString(){
+      return "ResultSet:\n"+ "\tclosed: "+this.isClosed+"\n\tcurrent row: "+currentRow;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixResultSetMetaData.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixResultSetMetaData.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixResultSetMetaData.java
new file mode 100644
index 0000000..55d558a
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixResultSetMetaData.java
@@ -0,0 +1,213 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.jdbc;
+
+import java.sql.*;
+
+import org.apache.phoenix.compile.ColumnProjector;
+import org.apache.phoenix.compile.RowProjector;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.PDataType;
+
+
+/**
+ * 
+ * JDBC ResultSetMetaData implementation of Phoenix.
+ * Currently only the following methods are supported:
+ * - {@link #getColumnCount()}
+ * - {@link #getColumnDisplaySize(int)}
+ * - {@link #getColumnLabel(int)} displays alias name if present and column name otherwise
+ * - {@link #getColumnName(int)} same as {@link #getColumnLabel(int)}
+ * - {@link #isCaseSensitive(int)}
+ * - {@link #getColumnType(int)}
+ * - {@link #getColumnTypeName(int)}
+ * - {@link #getTableName(int)}
+ * - {@link #getSchemaName(int)} always returns empty string
+ * - {@link #getCatalogName(int)} always returns empty string
+ * - {@link #isNullable(int)}
+ * - {@link #isSigned(int)}
+ * - {@link #isAutoIncrement(int)} always false
+ * - {@link #isCurrency(int)} always false
+ * - {@link #isDefinitelyWritable(int)} always false
+ * - {@link #isReadOnly(int)} always true
+ * - {@link #isSearchable(int)} always true
+ * 
+ * @author jtaylor
+ * @since 0.1
+ */
+public class PhoenixResultSetMetaData implements ResultSetMetaData {
+    private static final int MIN_DISPLAY_WIDTH = 3;
+    private static final int MAX_DISPLAY_WIDTH = 40;
+    private static final int DEFAULT_DISPLAY_WIDTH = 10;
+    private final RowProjector rowProjector;
+    private final PhoenixConnection connection;
+    
+    public PhoenixResultSetMetaData(PhoenixConnection connection, RowProjector projector) {
+        this.connection = connection;
+        this.rowProjector = projector;
+    }
+    
+    @Override
+    public String getCatalogName(int column) throws SQLException {
+        return "";
+    }
+
+    @Override
+    public String getColumnClassName(int column) throws SQLException {
+        PDataType type = rowProjector.getColumnProjector(column-1).getExpression().getDataType();
+        return type == null ? null : type.getJavaClassName();
+    }
+
+    @Override
+    public int getColumnCount() throws SQLException {
+        return rowProjector.getColumnCount();
+    }
+
+    @Override
+    public int getColumnDisplaySize(int column) throws SQLException {
+        ColumnProjector projector = rowProjector.getColumnProjector(column-1);
+        int displaySize = Math.max(projector.getName().length(),MIN_DISPLAY_WIDTH);
+        PDataType type = projector.getExpression().getDataType();
+        if (type == null) {
+            return Math.min(Math.max(displaySize, QueryConstants.NULL_DISPLAY_TEXT.length()), MAX_DISPLAY_WIDTH);
+        }
+        if (type.isCoercibleTo(PDataType.DATE)) {
+            return Math.min(Math.max(displaySize, connection.getDatePattern().length()), MAX_DISPLAY_WIDTH);
+        }
+        if (projector.getExpression().getByteSize() != null) {
+            return Math.min(Math.max(displaySize, projector.getExpression().getByteSize()), MAX_DISPLAY_WIDTH);
+        }
+        
+        return Math.min(Math.max(displaySize, DEFAULT_DISPLAY_WIDTH), MAX_DISPLAY_WIDTH);
+    }
+
+    @Override
+    public String getColumnLabel(int column) throws SQLException {
+        return rowProjector.getColumnProjector(column-1).getName();
+    }
+
+    @Override
+    public String getColumnName(int column) throws SQLException {
+        // TODO: will return alias if there is one
+        return rowProjector.getColumnProjector(column-1).getName();
+    }
+
+    @Override
+    public int getColumnType(int column) throws SQLException {
+        PDataType type = rowProjector.getColumnProjector(column-1).getExpression().getDataType();
+        return type == null ? Types.NULL : type.getResultSetSqlType();
+    }
+
+    @Override
+    public String getColumnTypeName(int column) throws SQLException {
+        PDataType type = rowProjector.getColumnProjector(column-1).getExpression().getDataType();
+        return type == null ? null : type.getSqlTypeName();
+    }
+
+    @Override
+    public int getPrecision(int column) throws SQLException {
+        Integer precision = rowProjector.getColumnProjector(column-1).getExpression().getMaxLength();
+        return precision == null ? 0 : precision;
+    }
+
+    @Override
+    public int getScale(int column) throws SQLException {
+        Integer scale = rowProjector.getColumnProjector(column-1).getExpression().getScale();
+        return scale == null ? 0 : scale;
+    }
+
+    @Override
+    public String getSchemaName(int column) throws SQLException {
+        return ""; // TODO
+    }
+
+    @Override
+    public String getTableName(int column) throws SQLException {
+        return rowProjector.getColumnProjector(column-1).getTableName();
+    }
+
+    @Override
+    public boolean isAutoIncrement(int column) throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean isCaseSensitive(int column) throws SQLException {
+        return rowProjector.getColumnProjector(column-1).isCaseSensitive();
+    }
+
+    @Override
+    public boolean isCurrency(int column) throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean isDefinitelyWritable(int column) throws SQLException {
+        return false;
+    }
+
+    @Override
+    public int isNullable(int column) throws SQLException {
+        return rowProjector.getColumnProjector(column-1).getExpression().isNullable() ? ResultSetMetaData.columnNullable : ResultSetMetaData.columnNoNulls;
+    }
+
+    @Override
+    public boolean isReadOnly(int column) throws SQLException {
+        return true;
+    }
+
+    @Override
+    public boolean isSearchable(int column) throws SQLException {
+        return true;
+    }
+
+    @Override
+    public boolean isSigned(int column) throws SQLException {
+        PDataType type = rowProjector.getColumnProjector(column-1).getExpression().getDataType();
+        if (type == null) {
+            return false;
+        }
+        return type.isCoercibleTo(PDataType.DECIMAL);
+    }
+
+    @Override
+    public boolean isWritable(int column) throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean isWrapperFor(Class<?> iface) throws SQLException {
+        return iface.isInstance(this);
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public <T> T unwrap(Class<T> iface) throws SQLException {
+        if (!iface.isInstance(this)) {
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode.CLASS_NOT_UNWRAPPABLE)
+                .setMessage(this.getClass().getName() + " not unwrappable from " + iface.getName())
+                .build().buildException();
+        }
+        return (T)this;
+    }
+    
+}


[19/51] [partial] Initial commit of master branch from github

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/ArrayConstructorNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ArrayConstructorNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ArrayConstructorNode.java
new file mode 100644
index 0000000..7d62b32
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ArrayConstructorNode.java
@@ -0,0 +1,44 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Holds the list of array elements that will be used by the upsert stmt with ARRAY column 
+ *
+ */
+public class ArrayConstructorNode extends CompoundParseNode {
+
+	public ArrayConstructorNode(List<ParseNode> children) {
+		super(children);
+	}
+
+	@Override
+	public <T> T accept(ParseNodeVisitor<T> visitor) throws SQLException {
+		List<T> l = Collections.emptyList();
+        if (visitor.visitEnter(this)) {
+            l = acceptChildren(visitor);
+        }
+        return visitor.visitLeave(this, l);
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/AvgAggregateParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/AvgAggregateParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/AvgAggregateParseNode.java
new file mode 100644
index 0000000..0b944c0
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/AvgAggregateParseNode.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.function.*;
+
+
+public class AvgAggregateParseNode extends AggregateFunctionParseNode {
+
+    public AvgAggregateParseNode(String name, List<ParseNode> children, BuiltInFunctionInfo info) {
+        super(name, children, info);
+    }
+
+    @Override
+    public FunctionExpression create(List<Expression> children, StatementContext context) throws SQLException {
+        SumAggregateFunction sumFunc;
+        CountAggregateFunction countFunc = (CountAggregateFunction)context.getExpressionManager().addIfAbsent(new CountAggregateFunction(children));
+        if (!countFunc.isConstantExpression()) {
+            sumFunc = (SumAggregateFunction)context.getExpressionManager().addIfAbsent(new SumAggregateFunction(countFunc.getChildren(),null));
+        } else {
+            sumFunc = null;
+        }
+
+        return new AvgAggregateFunction(children, countFunc, sumFunc);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/BaseParseNodeVisitor.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/BaseParseNodeVisitor.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/BaseParseNodeVisitor.java
new file mode 100644
index 0000000..233d5ec
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/BaseParseNodeVisitor.java
@@ -0,0 +1,70 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
+import java.util.List;
+
+
+
+/**
+ * 
+ * Base class for parse node visitors.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public abstract class BaseParseNodeVisitor<E> implements ParseNodeVisitor<E> {
+
+    /**
+     * Fall through visitEnter method. Anything coming through
+     * here means that a more specific method wasn't found
+     * and thus this CompoundNode is not yet supported.
+     */
+    @Override
+    public boolean visitEnter(CompoundParseNode expressionNode) throws SQLException {
+        throw new SQLFeatureNotSupportedException(expressionNode.toString());
+    }
+
+    @Override
+    public E visitLeave(CompoundParseNode expressionNode, List<E> l) throws SQLException {
+        throw new SQLFeatureNotSupportedException(expressionNode.toString());
+    }
+
+    /**
+     * Fall through visit method. Anything coming through
+     * here means that a more specific method wasn't found
+     * and thus this Node is not yet supported.
+     */
+    @Override
+    public E visit(ParseNode expressionNode) throws SQLException {
+        throw new SQLFeatureNotSupportedException(expressionNode.toString());
+    }
+    
+    @Override
+    public List<E> newElementList(int size) {
+        return null;
+    }
+    
+    @Override
+    public void addElement(List<E> l, E element) {
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/BetweenParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/BetweenParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/BetweenParseNode.java
new file mode 100644
index 0000000..6306d67
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/BetweenParseNode.java
@@ -0,0 +1,54 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+import java.util.*;
+
+
+
+/**
+ * 
+ * Node representing BETWEEN in SQL
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class BetweenParseNode extends CompoundParseNode {
+    private final boolean negate;
+
+    BetweenParseNode(ParseNode l, ParseNode r1, ParseNode r2, boolean negate) {
+        super(Arrays.asList(l, r1, r2));
+        this.negate = negate;
+    }
+    
+    public boolean isNegate() {
+        return negate;
+    }
+    
+    @Override
+    public <T> T accept(ParseNodeVisitor<T> visitor) throws SQLException {
+        List<T> l = Collections.emptyList();
+        if (visitor.visitEnter(this)) {
+            l = acceptChildren(visitor);
+        }
+        return visitor.visitLeave(this, l);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/BinaryParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/BinaryParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/BinaryParseNode.java
new file mode 100644
index 0000000..82695f5
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/BinaryParseNode.java
@@ -0,0 +1,44 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.util.Arrays;
+
+/**
+ * 
+ * Abstract class for operators that operate on exactly two nodes
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public abstract class BinaryParseNode extends CompoundParseNode {
+
+    BinaryParseNode(ParseNode lhs, ParseNode rhs) {
+        super(Arrays.asList(lhs, rhs));
+    }
+
+    public ParseNode getLHS() {
+        return getChildren().get(0);
+    }
+    
+    public ParseNode getRHS() {
+        return getChildren().get(1);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/BindParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/BindParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/BindParseNode.java
new file mode 100644
index 0000000..5d4f173
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/BindParseNode.java
@@ -0,0 +1,61 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+
+
+
+/**
+ * 
+ * Node representing a bind variable in a SQL expression
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class BindParseNode extends NamedParseNode {
+    private final int index;
+    
+    BindParseNode(String name) {
+        super(name);
+        index = Integer.parseInt(name);
+    }
+    
+    public int getIndex() {
+        return index-1;
+    }
+
+    @Override
+    public <T> T accept(ParseNodeVisitor<T> visitor) throws SQLException {
+        return visitor.visit(this);
+    }
+
+    
+    @Override
+    public boolean isStateless() {
+        return true;
+    }
+    
+    @Override
+    public String toString() {
+        return ":" + index;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/BindTableNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/BindTableNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/BindTableNode.java
new file mode 100644
index 0000000..7aca261
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/BindTableNode.java
@@ -0,0 +1,44 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+
+
+
+/**
+ * 
+ * Node representing a TABLE bound using an ARRAY variable
+ * TODO: modify grammar to support this
+ * @author jtaylor
+ * @since 0.1
+ */
+public class BindTableNode extends ConcreteTableNode {
+
+    BindTableNode(String alias, TableName name) {
+        super(alias, name);
+    }
+
+    @Override
+    public void accept(TableNodeVisitor visitor) throws SQLException {
+        visitor.visit(this);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/BindableStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/BindableStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/BindableStatement.java
new file mode 100644
index 0000000..7ce9162
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/BindableStatement.java
@@ -0,0 +1,25 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+
+public interface BindableStatement {
+    public int getBindCount();
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/CaseParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/CaseParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/CaseParseNode.java
new file mode 100644
index 0000000..bd04490
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/CaseParseNode.java
@@ -0,0 +1,49 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.List;
+
+
+
+/**
+ * 
+ * Node representing a CASE in SQL
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class CaseParseNode extends CompoundParseNode {
+
+    CaseParseNode(List<ParseNode> children) {
+        super(children);
+    }
+
+    @Override
+    public <T> T accept(ParseNodeVisitor<T> visitor) throws SQLException {
+        List<T> l = Collections.emptyList();
+        if (visitor.visitEnter(this)) {
+            l = acceptChildren(visitor);
+        }
+        return visitor.visitLeave(this, l);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/CastParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/CastParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/CastParseNode.java
new file mode 100644
index 0000000..24cae70
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/CastParseNode.java
@@ -0,0 +1,80 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.function.RoundDecimalExpression;
+import org.apache.phoenix.expression.function.RoundTimestampExpression;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.TypeMismatchException;
+
+/**
+ * 
+ * Node representing the CAST operator in SQL.
+ * 
+ * @author samarth.jain
+ * @since 0.1
+ *
+ */
+public class CastParseNode extends UnaryParseNode {
+	
+	private final PDataType dt;
+	
+	CastParseNode(ParseNode expr, String dataType) {
+		super(expr);
+		dt = PDataType.fromSqlTypeName(dataType);
+	}
+	
+	CastParseNode(ParseNode expr, PDataType dataType) {
+		super(expr);
+		dt = dataType;
+	}
+
+	@Override
+    public <T> T accept(ParseNodeVisitor<T> visitor) throws SQLException {
+        List<T> l = Collections.emptyList();
+        if (visitor.visitEnter(this)) {
+            l = acceptChildren(visitor);
+        }
+        return visitor.visitLeave(this, l);
+    }
+
+	public PDataType getDataType() {
+		return dt;
+	}
+	
+	public static Expression convertToRoundExpressionIfNeeded(PDataType fromDataType, PDataType targetDataType, List<Expression> expressions) throws SQLException {
+	    Expression firstChildExpr = expressions.get(0);
+	    if(fromDataType == targetDataType) {
+	        return firstChildExpr;
+	    } else if(fromDataType == PDataType.DECIMAL && targetDataType.isCoercibleTo(PDataType.LONG)) {
+	        return new RoundDecimalExpression(expressions);
+	    } else if((fromDataType == PDataType.TIMESTAMP || fromDataType == PDataType.UNSIGNED_TIMESTAMP) && targetDataType.isCoercibleTo(PDataType.DATE)) {
+	        return RoundTimestampExpression.create(expressions);
+	    } else if(!fromDataType.isCoercibleTo(targetDataType)) {
+	        throw TypeMismatchException.newException(fromDataType, targetDataType, firstChildExpr.toString());
+	    }
+	    return firstChildExpr;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/CeilParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/CeilParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/CeilParseNode.java
new file mode 100644
index 0000000..b041c84
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/CeilParseNode.java
@@ -0,0 +1,82 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.function.CeilDateExpression;
+import org.apache.phoenix.expression.function.CeilDecimalExpression;
+import org.apache.phoenix.expression.function.CeilFunction;
+import org.apache.phoenix.expression.function.CeilTimestampExpression;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.TypeMismatchException;
+
+/**
+ * Parse node corresponding to {@link CeilFunction}. 
+ * It also acts as a factory for creating the right kind of
+ * ceil expression according to the data type of the 
+ * first child.
+ *
+ * @author samarth.jain
+ * @since 3.0.0
+ */
+public class CeilParseNode extends FunctionParseNode {
+    
+    CeilParseNode(String name, List<ParseNode> children, BuiltInFunctionInfo info) {
+        super(name, children, info);
+    }
+    
+    @Override
+    public Expression create(List<Expression> children, StatementContext context) throws SQLException {
+        return getCeilExpression(children);
+    }
+    
+    public static Expression getCeilExpression(List<Expression> children) throws SQLException {
+        final Expression firstChild = children.get(0);
+        final PDataType firstChildDataType = firstChild.getDataType();
+        if(firstChildDataType.isCoercibleTo(PDataType.DATE)) {
+            return CeilDateExpression.create(children);
+        } else if (firstChildDataType == PDataType.TIMESTAMP || firstChildDataType == PDataType.UNSIGNED_TIMESTAMP) {
+            return CeilTimestampExpression.create(children);
+        } else if(firstChildDataType.isCoercibleTo(PDataType.DECIMAL)) {
+            return new CeilDecimalExpression(children);
+        } else {
+            throw TypeMismatchException.newException(firstChildDataType, "1");
+        }
+    }
+    
+    /**
+     * When ceiling off decimals, user need not specify the scale. In such cases, 
+     * we need to prevent the function from getting evaluated as null. This is really
+     * a hack. A better way would have been if {@link org.apache.phoenix.parse.FunctionParseNode.BuiltInFunctionInfo} provided a 
+     * way of associating default values for each permissible data type.
+     * Something like: @ Argument(allowedTypes={PDataType.VARCHAR, PDataType.INTEGER}, defaultValues = {"null", "1"} isConstant=true)
+     * Till then, this will have to do.
+     */
+    @Override
+    public boolean evalToNullIfParamIsNull(StatementContext context, int index) throws SQLException {
+        return index == 0;
+    }
+    
+    
+}   

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/ColumnDef.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ColumnDef.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ColumnDef.java
new file mode 100644
index 0000000..183da7c
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ColumnDef.java
@@ -0,0 +1,178 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.util.SchemaUtil;
+
+
+/**
+ * 
+ * Represents a column definition during DDL
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class ColumnDef {
+    private final ColumnName columnDefName;
+    private PDataType dataType;
+    private final boolean isNull;
+    private final Integer maxLength;
+    private final Integer scale;
+    private final boolean isPK;
+    private final ColumnModifier columnModifier;
+    private final boolean isArray;
+    private final Integer arrSize;
+ 
+    ColumnDef(ColumnName columnDefName, String sqlTypeName, boolean isArray, Integer arrSize, boolean isNull, Integer maxLength,
+    		            Integer scale, boolean isPK, ColumnModifier columnModifier) {
+   	 try {
+   	     PDataType localType = null;
+         this.columnDefName = columnDefName;
+         this.isArray = isArray;
+         // TODO : Add correctness check for arrSize.  Should this be ignored as in postgres
+         // Also add what is the limit that we would support.  Are we going to support a
+         //  fixed size or like postgres allow infinite.  May be the data types max limit can 
+         // be used for the array size (May be too big)
+         if(this.isArray) {
+        	 localType = sqlTypeName == null ? null : PDataType.fromTypeId(PDataType.sqlArrayType(SchemaUtil.normalizeIdentifier(sqlTypeName)));
+        	 this.dataType = sqlTypeName == null ? null : PDataType.fromSqlTypeName(SchemaUtil.normalizeIdentifier(sqlTypeName));
+             this.arrSize = arrSize; // Can only be non negative based on parsing
+         } else {
+             this.dataType = sqlTypeName == null ? null : PDataType.fromSqlTypeName(SchemaUtil.normalizeIdentifier(sqlTypeName));
+             this.arrSize = null;
+         }
+         
+         this.isNull = isNull;
+         if (this.dataType == PDataType.CHAR) {
+             if (maxLength == null) {
+                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.MISSING_CHAR_LENGTH)
+                     .setColumnName(columnDefName.getColumnName()).build().buildException();
+             }
+             if (maxLength < 1) {
+                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.NONPOSITIVE_CHAR_LENGTH)
+                     .setColumnName(columnDefName.getColumnName()).build().buildException();
+             }
+             scale = null;
+         } else if (this.dataType == PDataType.VARCHAR) {
+             if (maxLength != null && maxLength < 1) {
+                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.NONPOSITIVE_CHAR_LENGTH)
+                     .setColumnName(columnDefName.getColumnName()).build().buildException(); 
+             }
+             scale = null;
+         } else if (this.dataType == PDataType.DECIMAL) {
+         	Integer origMaxLength = maxLength;
+             maxLength = maxLength == null ? PDataType.MAX_PRECISION : maxLength;
+             // for deciaml, 1 <= maxLength <= PDataType.MAX_PRECISION;
+             if (maxLength < 1 || maxLength > PDataType.MAX_PRECISION) {
+                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.DECIMAL_PRECISION_OUT_OF_RANGE)
+                     .setColumnName(columnDefName.getColumnName()).build().buildException();
+             }
+             // When a precision is specified and a scale is not specified, it is set to 0. 
+             // 
+             // This is the standard as specified in
+             // http://docs.oracle.com/cd/B28359_01/server.111/b28318/datatype.htm#CNCPT1832
+             // and 
+             // http://docs.oracle.com/javadb/10.6.2.1/ref/rrefsqlj15260.html.
+             // Otherwise, if scale is bigger than maxLength, just set it to the maxLength;
+             //
+             // When neither a precision nor a scale is specified, the precision and scale is
+             // ignored. All decimal are stored with as much decimal points as possible.
+             scale = scale == null ? 
+             		origMaxLength == null ? null : PDataType.DEFAULT_SCALE : 
+             		scale > maxLength ? maxLength : scale; 
+         } else if (this.dataType == PDataType.BINARY) {
+             if (maxLength == null) {
+                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.MISSING_BINARY_LENGTH)
+                     .setColumnName(columnDefName.getColumnName()).build().buildException();
+             }
+             if (maxLength < 1) {
+                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.NONPOSITIVE_BINARY_LENGTH)
+                     .setColumnName(columnDefName.getColumnName()).build().buildException();
+             }
+             scale = null;
+         } else if (this.dataType == PDataType.INTEGER) {
+             maxLength = PDataType.INT_PRECISION;
+             scale = PDataType.ZERO;
+         } else if (this.dataType == PDataType.LONG) {
+             maxLength = PDataType.LONG_PRECISION;
+             scale = PDataType.ZERO;
+         } else {
+             // ignore maxLength and scale for other types.
+             maxLength = null;
+             scale = null;
+         }
+         this.maxLength = maxLength;
+         this.scale = scale;
+         this.isPK = isPK;
+         this.columnModifier = columnModifier;
+         if(this.isArray) {
+             this.dataType = localType;
+         }
+     } catch (SQLException e) {
+         throw new ParseException(e);
+     }
+    }
+    ColumnDef(ColumnName columnDefName, String sqlTypeName, boolean isNull, Integer maxLength,
+            Integer scale, boolean isPK, ColumnModifier columnModifier) {
+    	this(columnDefName, sqlTypeName, false, 0, isNull, maxLength, scale, isPK, columnModifier);
+    }
+
+    public ColumnName getColumnDefName() {
+        return columnDefName;
+    }
+
+    public PDataType getDataType() {
+        return dataType;
+    }
+
+    public boolean isNull() {
+        return isNull;
+    }
+
+    public Integer getMaxLength() {
+        return maxLength;
+    }
+
+    public Integer getScale() {
+        return scale;
+    }
+
+    public boolean isPK() {
+        return isPK;
+    }
+    
+    public ColumnModifier getColumnModifier() {
+    	return columnModifier;
+    }
+        
+	public boolean isArray() {
+		return isArray;
+	}
+
+	public Integer getArraySize() {
+		return arrSize;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/ColumnFamilyDef.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ColumnFamilyDef.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ColumnFamilyDef.java
new file mode 100644
index 0000000..5323066
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ColumnFamilyDef.java
@@ -0,0 +1,57 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.util.*;
+
+
+import com.google.common.collect.ImmutableList;
+import org.apache.phoenix.util.SchemaUtil;
+
+/**
+ * 
+ * Definition of a Column Family at DDL time
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class ColumnFamilyDef {
+    private final String name;
+    private final List<ColumnDef> columnDefs;
+    private final Map<String,Object> props;
+    
+    ColumnFamilyDef(String name, List<ColumnDef> columnDefs, Map<String,Object> props) {
+        this.name = SchemaUtil.normalizeIdentifier(name);
+        this.columnDefs = ImmutableList.copyOf(columnDefs);
+        this.props = props == null ? Collections.<String,Object>emptyMap() : props;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public List<ColumnDef> getColumnDefs() {
+        return columnDefs;
+    }
+
+    public Map<String,Object> getProps() {
+        return props;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/ColumnName.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ColumnName.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ColumnName.java
new file mode 100644
index 0000000..34511c7
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ColumnName.java
@@ -0,0 +1,102 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import org.apache.phoenix.util.SchemaUtil;
+
+
+public class ColumnName {
+    private final NamedNode familyNode;
+    private final NamedNode columnNode;
+    
+    public static ColumnName caseSensitiveColumnName(String familyName, String columnName) {
+        return new ColumnName(NamedNode.caseSensitiveNamedNode(familyName), NamedNode.caseSensitiveNamedNode(columnName));
+    }
+    
+    public static ColumnName caseSensitiveColumnName(String columnName) {
+        return new ColumnName(null, NamedNode.caseSensitiveNamedNode(columnName));
+    }
+    
+    public static ColumnName newColumnName(NamedNode columnName) {
+        return new ColumnName(null, columnName);
+    }
+    
+    public static ColumnName newColumnName(NamedNode familyName, NamedNode columnName) {
+        return new ColumnName(familyName, columnName);
+    }
+    
+    private ColumnName(NamedNode familyNode, NamedNode columnNode) {
+        this.familyNode = familyNode;
+        this.columnNode = columnNode;
+    }
+    
+
+    ColumnName(String familyName, String columnName) {
+        this.familyNode = familyName == null ? null : new NamedNode(familyName);
+        this.columnNode = new NamedNode(columnName);
+    }
+
+    ColumnName(String columnName) {
+        this(null, columnName);
+    }
+
+    public String getFamilyName() {
+        return familyNode == null ? null : familyNode.getName();
+    }
+
+    public String getColumnName() {
+        return columnNode.getName();
+    }
+
+    public NamedNode getFamilyNode() {
+        return familyNode;
+    }
+
+    public NamedNode getColumnNode() {
+        return columnNode;
+    }
+
+    @Override
+    public String toString() {
+        return SchemaUtil.getColumnName(getFamilyName(),getColumnName());
+    }
+    
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + columnNode.hashCode();
+        result = prime * result + ((familyNode == null) ? 0 : familyNode.hashCode());
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) return true;
+        if (obj == null) return false;
+        if (getClass() != obj.getClass()) return false;
+        ColumnName other = (ColumnName)obj;
+        if (!columnNode.equals(other.columnNode)) return false;
+        if (familyNode == null) {
+            if (other.familyNode != null) return false;
+        } else if (!familyNode.equals(other.familyNode)) return false;
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/ColumnParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ColumnParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ColumnParseNode.java
new file mode 100644
index 0000000..33058e9
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ColumnParseNode.java
@@ -0,0 +1,90 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+
+import org.apache.phoenix.query.QueryConstants;
+
+/**
+ * Node representing a reference to a column in a SQL expression
+ * 
+ * @author jtaylor
+ * @since 0.1
+ */
+public class ColumnParseNode extends NamedParseNode {
+    private final TableName tableName;
+    private final String fullName;
+    private final String alias;
+
+    public ColumnParseNode(TableName tableName, String name, String alias) {
+        // Upper case here so our Maps can depend on this (and we don't have to upper case and create a string on every
+        // lookup
+        super(name);
+        this.alias = alias;
+        this.tableName = tableName;
+        fullName = tableName == null ? getName() : tableName.toString() + QueryConstants.NAME_SEPARATOR + getName();
+    }
+
+    public ColumnParseNode(TableName tableName, String name) {
+        this(tableName, name, null);
+    }
+    
+    @Override
+    public <T> T accept(ParseNodeVisitor<T> visitor) throws SQLException {
+        return visitor.visit(this);
+    }
+
+    public String getTableName() {
+        return tableName == null ? null : tableName.getTableName();
+    }
+
+    public String getSchemaName() {
+        return tableName == null ? null : tableName.getSchemaName();
+    }
+
+    public String getFullName() {
+        return fullName;
+    }
+
+    @Override
+    public String getAlias() {
+        return alias;
+    }
+
+    @Override
+    public String toString() {
+        return fullName;
+    }
+
+    @Override
+    public int hashCode() {
+        return fullName.hashCode();
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) return true;
+        if (obj == null) return false;
+        if (getClass() != obj.getClass()) return false;
+        ColumnParseNode other = (ColumnParseNode)obj;
+        return fullName.equals(other.fullName);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/ComparisonParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ComparisonParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ComparisonParseNode.java
new file mode 100644
index 0000000..1a50fb7
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ComparisonParseNode.java
@@ -0,0 +1,59 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.hbase.filter.CompareFilter;
+
+/**
+ * 
+ * Common base class for =, >, >=, <, <=, !=
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public abstract class ComparisonParseNode extends BinaryParseNode {
+
+    ComparisonParseNode(ParseNode lhs, ParseNode rhs) {
+        super(lhs, rhs);
+    }
+
+    @Override
+    public final <T> T accept(ParseNodeVisitor<T> visitor) throws SQLException {
+        List<T> l = Collections.emptyList();
+        if (visitor.visitEnter(this)) {
+            l = acceptChildren(visitor);
+        }
+        return visitor.visitLeave(this, l);
+    }
+
+    /**
+     * Return the comparison operator associated with the given comparison expression node
+     */
+    public abstract CompareFilter.CompareOp getFilterOp();
+    
+    /**
+     * Return the inverted operator for the CompareOp
+     */
+    public abstract CompareFilter.CompareOp getInvertFilterOp();
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/CompoundParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/CompoundParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/CompoundParseNode.java
new file mode 100644
index 0000000..45c196e
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/CompoundParseNode.java
@@ -0,0 +1,75 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.List;
+
+
+
+/**
+ * 
+ * Abstract node representing an expression node that has children
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public abstract class CompoundParseNode extends ParseNode {
+    private final List<ParseNode> children;
+    private final boolean isStateless;
+    
+    CompoundParseNode(List<ParseNode> children) {
+        this.children = Collections.unmodifiableList(children);
+        boolean isStateless = true;
+        for (ParseNode child : children) {
+            isStateless &= child.isStateless();
+            if (!isStateless) {
+                break;
+            }
+        }
+        this.isStateless = isStateless;
+    }
+    
+    @Override
+    public boolean isStateless() {
+        return isStateless;
+    }
+    
+    @Override
+    public final List<ParseNode> getChildren() {
+        return children;
+    }
+
+
+    final <T> List<T> acceptChildren(ParseNodeVisitor<T> visitor) throws SQLException {
+        List<T> l = visitor.newElementList(children.size());        
+        for (int i = 0; i < children.size(); i++) {
+            T e = children.get(i).accept(visitor);
+            visitor.addElement(l, e);
+        }
+        return l;
+    }
+
+    @Override
+    public String toString() {
+        return this.getClass().getName() + children.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/ConcreteTableNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ConcreteTableNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ConcreteTableNode.java
new file mode 100644
index 0000000..cd00316
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ConcreteTableNode.java
@@ -0,0 +1,43 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import org.apache.phoenix.util.SchemaUtil;
+
+/**
+ * 
+ * Abstract node representing a table reference in the FROM clause in SQL
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public abstract class ConcreteTableNode extends TableNode {
+    private final TableName name;
+
+    ConcreteTableNode(String alias, TableName name) {
+        super(SchemaUtil.normalizeIdentifier(alias));
+        this.name = name;
+    }
+
+    public TableName getName() {
+        return name;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/CreateIndexStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/CreateIndexStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/CreateIndexStatement.java
new file mode 100644
index 0000000..95d3233
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/CreateIndexStatement.java
@@ -0,0 +1,74 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.hbase.util.Pair;
+
+import com.google.common.collect.ListMultimap;
+
+
+public class CreateIndexStatement extends SingleTableSQLStatement {
+    private final TableName indexTableName;
+    private final PrimaryKeyConstraint indexConstraint;
+    private final List<ColumnName> includeColumns;
+    private final List<ParseNode> splitNodes;
+    private final ListMultimap<String,Pair<String,Object>> props;
+    private final boolean ifNotExists;
+
+    public CreateIndexStatement(NamedNode indexTableName, NamedTableNode dataTable, 
+            PrimaryKeyConstraint indexConstraint, List<ColumnName> includeColumns, List<ParseNode> splits,
+            ListMultimap<String,Pair<String,Object>> props, boolean ifNotExists, int bindCount) {
+        super(dataTable, bindCount);
+        this.indexTableName =TableName.createNormalized(dataTable.getName().getSchemaName(),indexTableName.getName());
+        this.indexConstraint = indexConstraint == null ? PrimaryKeyConstraint.EMPTY : indexConstraint;
+        this.includeColumns = includeColumns == null ? Collections.<ColumnName>emptyList() : includeColumns;
+        this.splitNodes = splits == null ? Collections.<ParseNode>emptyList() : splits;
+        this.props = props;
+        this.ifNotExists = ifNotExists;
+    }
+
+    public PrimaryKeyConstraint getIndexConstraint() {
+        return indexConstraint;
+    }
+
+    public List<ColumnName> getIncludeColumns() {
+        return includeColumns;
+    }
+
+    public TableName getIndexTableName() {
+        return indexTableName;
+    }
+
+    public List<ParseNode> getSplitNodes() {
+        return splitNodes;
+    }
+
+    public ListMultimap<String,Pair<String,Object>> getProps() {
+        return props;
+    }
+
+    public boolean ifNotExists() {
+        return ifNotExists;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/CreateSequenceStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/CreateSequenceStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/CreateSequenceStatement.java
new file mode 100644
index 0000000..b24c076
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/CreateSequenceStatement.java
@@ -0,0 +1,64 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+public class CreateSequenceStatement implements BindableStatement {
+
+	private final TableName sequenceName;
+	private final ParseNode startWith;
+	private final ParseNode incrementBy;
+    private final ParseNode cacheSize;
+    private final boolean ifNotExists;
+	private final int bindCount;
+
+	protected CreateSequenceStatement(TableName sequenceName, ParseNode startsWith, ParseNode incrementBy, ParseNode cacheSize, boolean ifNotExists, int bindCount) {
+		this.sequenceName = sequenceName;
+		this.startWith = startsWith == null ? LiteralParseNode.ONE : startsWith;
+		this.incrementBy = incrementBy == null ? LiteralParseNode.ONE : incrementBy;
+        this.cacheSize = cacheSize == null ? null : cacheSize;
+		this.ifNotExists = ifNotExists;
+		this.bindCount = bindCount;
+	}
+
+	@Override
+	public int getBindCount() {
+		return this.bindCount;
+	}
+	
+	public ParseNode getIncrementBy() {
+		return incrementBy;
+	}
+
+	public TableName getSequenceName() {
+		return sequenceName;
+	}
+
+    public ParseNode getCacheSize() {
+        return cacheSize;
+    }
+
+	public ParseNode getStartWith() {
+		return startWith;
+	}
+
+    public boolean ifNotExists() {
+        return ifNotExists;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/CreateTableStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/CreateTableStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/CreateTableStatement.java
new file mode 100644
index 0000000..9d6d551
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/CreateTableStatement.java
@@ -0,0 +1,100 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.hbase.util.Pair;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableListMultimap;
+import com.google.common.collect.ListMultimap;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.schema.PTableType;
+
+public class CreateTableStatement implements BindableStatement {
+    private final TableName tableName;
+    private final PTableType tableType;
+    private final List<ColumnDef> columns;
+    private final PrimaryKeyConstraint pkConstraint;
+    private final List<ParseNode> splitNodes;
+    private final int bindCount;
+    private final ListMultimap<String,Pair<String,Object>> props;
+    private final boolean ifNotExists;
+    private final TableName baseTableName;
+    private final ParseNode whereClause;
+    
+    protected CreateTableStatement(TableName tableName, ListMultimap<String,Pair<String,Object>> props, List<ColumnDef> columns, PrimaryKeyConstraint pkConstraint,
+            List<ParseNode> splitNodes, PTableType tableType, boolean ifNotExists, 
+            TableName baseTableName, ParseNode whereClause, int bindCount) {
+        this.tableName = tableName;
+        this.props = props == null ? ImmutableListMultimap.<String,Pair<String,Object>>of() : props;
+        this.tableType = PhoenixDatabaseMetaData.TYPE_SCHEMA.equals(tableName.getSchemaName()) ? PTableType.SYSTEM : tableType;
+        this.columns = columns == null ? ImmutableList.<ColumnDef>of() : ImmutableList.<ColumnDef>copyOf(columns);
+        this.pkConstraint = pkConstraint == null ? PrimaryKeyConstraint.EMPTY : pkConstraint;
+        this.splitNodes = splitNodes == null ? Collections.<ParseNode>emptyList() : ImmutableList.copyOf(splitNodes);
+        this.bindCount = bindCount;
+        this.ifNotExists = ifNotExists;
+        this.baseTableName = baseTableName;
+        this.whereClause = whereClause;
+    }
+    
+    public ParseNode getWhereClause() {
+        return whereClause;
+    }
+    
+    @Override
+    public int getBindCount() {
+        return bindCount;
+    }
+
+    public TableName getTableName() {
+        return tableName;
+    }
+
+    public TableName getBaseTableName() {
+        return baseTableName;
+    }
+
+    public List<ColumnDef> getColumnDefs() {
+        return columns;
+    }
+
+    public List<ParseNode> getSplitNodes() {
+        return splitNodes;
+    }
+
+    public PTableType getTableType() {
+        return tableType;
+    }
+
+    public ListMultimap<String,Pair<String,Object>> getProps() {
+        return props;
+    }
+
+    public boolean ifNotExists() {
+        return ifNotExists;
+    }
+
+    public PrimaryKeyConstraint getPrimaryKeyConstraint() {
+        return pkConstraint;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/CurrentDateParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/CurrentDateParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/CurrentDateParseNode.java
new file mode 100644
index 0000000..4ad29bc
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/CurrentDateParseNode.java
@@ -0,0 +1,41 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.function.CurrentDateFunction;
+import org.apache.phoenix.expression.function.FunctionExpression;
+
+
+public class CurrentDateParseNode extends FunctionParseNode {
+
+    public CurrentDateParseNode(String name, List<ParseNode> children, BuiltInFunctionInfo info) {
+        super(name, children, info);
+    }
+
+    @Override
+    public FunctionExpression create(List<Expression> children, StatementContext context) throws SQLException {
+        return new CurrentDateFunction(context.getCurrentTime());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/CurrentTimeParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/CurrentTimeParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/CurrentTimeParseNode.java
new file mode 100644
index 0000000..ede88a2
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/CurrentTimeParseNode.java
@@ -0,0 +1,41 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.function.CurrentTimeFunction;
+import org.apache.phoenix.expression.function.FunctionExpression;
+
+
+public class CurrentTimeParseNode extends FunctionParseNode {
+
+    public CurrentTimeParseNode(String name, List<ParseNode> children, BuiltInFunctionInfo info) {
+        super(name, children, info);
+    }
+
+    @Override
+    public FunctionExpression create(List<Expression> children, StatementContext context) throws SQLException {
+        return new CurrentTimeFunction(context.getCurrentTime());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/DelegateConstantToCountParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/DelegateConstantToCountParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/DelegateConstantToCountParseNode.java
new file mode 100644
index 0000000..8e436ce
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/DelegateConstantToCountParseNode.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.function.CountAggregateFunction;
+import org.apache.phoenix.expression.function.FunctionExpression;
+
+
+public abstract class DelegateConstantToCountParseNode extends AggregateFunctionParseNode {
+
+    public DelegateConstantToCountParseNode(String name, List<ParseNode> children, BuiltInFunctionInfo info) {
+        super(name, children, info);
+    }
+    
+    protected CountAggregateFunction getDelegateFunction(List<Expression> children, StatementContext context) {
+        CountAggregateFunction countFunc = null;
+        if (getChildren().get(0).isStateless()) {
+            countFunc = (CountAggregateFunction)context.getExpressionManager().addIfAbsent(new CountAggregateFunction(children));
+        }
+        return countFunc;
+    }
+    
+    @Override
+    public abstract FunctionExpression create(List<Expression> children, StatementContext context) throws SQLException;
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/DeleteStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/DeleteStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/DeleteStatement.java
new file mode 100644
index 0000000..8b23c9a
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/DeleteStatement.java
@@ -0,0 +1,69 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.util.Collections;
+import java.util.List;
+
+public class DeleteStatement extends SingleTableSQLStatement implements FilterableStatement {
+    private final ParseNode whereNode;
+    private final List<OrderByNode> orderBy;
+    private final LimitNode limit;
+    private final HintNode hint;
+    
+    public DeleteStatement(NamedTableNode table, HintNode hint, ParseNode whereNode, List<OrderByNode> orderBy, LimitNode limit, int bindCount) {
+        super(table, bindCount);
+        this.whereNode = whereNode;
+        this.orderBy = orderBy == null ? Collections.<OrderByNode>emptyList() : orderBy;
+        this.limit = limit;
+        this.hint = hint == null ? HintNode.EMPTY_HINT_NODE : hint;
+    }
+
+    @Override
+    public ParseNode getWhere() {
+        return whereNode;
+    }
+
+    @Override
+    public List<OrderByNode> getOrderBy() {
+        return orderBy;
+    }
+
+    @Override
+    public LimitNode getLimit() {
+        return limit;
+    }
+
+    @Override
+    public HintNode getHint() {
+        return hint;
+    }
+
+    @Override
+    public boolean isDistinct() {
+        return false;
+    }
+
+    @Override
+    public boolean isAggregate() {
+        return false;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/DerivedTableNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/DerivedTableNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/DerivedTableNode.java
new file mode 100644
index 0000000..3d76698
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/DerivedTableNode.java
@@ -0,0 +1,51 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+
+
+
+/**
+ * 
+ * Node representing a subselect in the FROM clause of SQL
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class DerivedTableNode extends TableNode {
+
+    private final SelectStatement select;
+
+    DerivedTableNode(String alias, SelectStatement select) {
+        super(alias);
+        this.select = select;
+    }
+
+    public SelectStatement getSelect() {
+        return select;
+    }
+
+    @Override
+    public void accept(TableNodeVisitor visitor) throws SQLException {
+        visitor.visit(this);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/DistinctCountParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/DistinctCountParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/DistinctCountParseNode.java
new file mode 100644
index 0000000..d270f2b
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/DistinctCountParseNode.java
@@ -0,0 +1,46 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.function.DistinctCountAggregateFunction;
+import org.apache.phoenix.expression.function.FunctionExpression;
+
+/**
+ * 
+ * @author anoopsjohn
+ * @since 1.2.1
+ */
+public class DistinctCountParseNode extends DelegateConstantToCountParseNode {
+    
+    public DistinctCountParseNode(String name, List<ParseNode> children, BuiltInFunctionInfo info) {
+        super(name, children, info);
+    }
+
+    @Override
+    public FunctionExpression create(List<Expression> children, StatementContext context)
+            throws SQLException {
+        return new DistinctCountAggregateFunction(children, getDelegateFunction(children, context));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/DivideParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/DivideParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/DivideParseNode.java
new file mode 100644
index 0000000..bef0d8c
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/DivideParseNode.java
@@ -0,0 +1,49 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.List;
+
+
+
+/**
+ * 
+ * Node representing division in a SQL expression
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class DivideParseNode extends ArithmeticParseNode {
+
+    DivideParseNode(List<ParseNode> children) {
+        super(children);
+    }
+
+    @Override
+    public <T> T accept(ParseNodeVisitor<T> visitor) throws SQLException {
+        List<T> l = Collections.emptyList();
+        if (visitor.visitEnter(this)) {
+            l = acceptChildren(visitor);
+        }
+        return visitor.visitLeave(this, l);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/DropColumnStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/DropColumnStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/DropColumnStatement.java
new file mode 100644
index 0000000..71cf099
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/DropColumnStatement.java
@@ -0,0 +1,43 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.util.List;
+
+import org.apache.phoenix.schema.PTableType;
+
+public class DropColumnStatement extends AlterTableStatement {
+    private final List<ColumnName> columnRefs;
+    private final boolean ifExists;
+    
+    protected DropColumnStatement(NamedTableNode table, PTableType tableType, List<ColumnName> columnRefs, boolean ifExists) {
+        super(table, tableType);
+        this.columnRefs = columnRefs;
+        this.ifExists = ifExists;
+    }
+
+    public List<ColumnName> getColumnRefs() {
+        return columnRefs;
+    }
+
+    public boolean ifExists() {
+        return ifExists;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/DropIndexStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/DropIndexStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/DropIndexStatement.java
new file mode 100644
index 0000000..8a01622
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/DropIndexStatement.java
@@ -0,0 +1,50 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+public class DropIndexStatement implements BindableStatement {
+    private final TableName tableName;
+    private final NamedNode indexName;
+    private final boolean ifExists;
+
+    public DropIndexStatement(NamedNode indexName, TableName tableName, boolean ifExists) {
+        this.indexName = indexName;
+        this.tableName = tableName;
+        this.ifExists = ifExists;
+    }
+
+    public TableName getTableName() {
+        return tableName;
+    }
+
+    public NamedNode getIndexName() {
+        return indexName;
+    }
+
+    @Override
+    public int getBindCount() {
+        return 0;
+    }
+
+    public boolean ifExists() {
+        return ifExists;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/DropSequenceStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/DropSequenceStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/DropSequenceStatement.java
new file mode 100644
index 0000000..6ae9652
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/DropSequenceStatement.java
@@ -0,0 +1,46 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+public class DropSequenceStatement implements BindableStatement {
+
+    private final TableName sequenceName;
+    private final boolean ifExists;
+    private final int bindCount;
+
+    protected DropSequenceStatement(TableName sequenceName, boolean ifExists, int bindCount) {
+        this.sequenceName = sequenceName;
+        this.ifExists = ifExists;
+        this.bindCount = bindCount;
+    }
+
+    @Override
+    public int getBindCount() {
+        return bindCount;
+    }
+
+    public TableName getSequenceName() {
+        return sequenceName;
+    }
+
+    public boolean ifExists() {
+        return ifExists;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/DropTableStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/DropTableStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/DropTableStatement.java
new file mode 100644
index 0000000..22c4530
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/DropTableStatement.java
@@ -0,0 +1,51 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import org.apache.phoenix.schema.PTableType;
+
+public class DropTableStatement implements BindableStatement {
+    private final TableName tableName;
+    private final boolean ifExists;
+    private final PTableType tableType;
+
+    protected DropTableStatement(TableName tableName, PTableType tableType, boolean ifExists) {
+        this.tableName = tableName;
+        this.tableType = tableType;
+        this.ifExists = ifExists;
+    }
+    
+    @Override
+    public int getBindCount() {
+        return 0; // No binds for DROP
+    }
+
+    public TableName getTableName() {
+        return tableName;
+    }
+
+    public PTableType getTableType() {
+        return tableType;
+    }
+
+    public boolean ifExists() {
+        return ifExists;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/EqualParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/EqualParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/EqualParseNode.java
new file mode 100644
index 0000000..bdefff9
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/EqualParseNode.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+
+
+/**
+ * 
+ * Node representing the equal operator in SQL
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class EqualParseNode extends ComparisonParseNode {
+
+    EqualParseNode(ParseNode lhs, ParseNode rhs) {
+        super(lhs, rhs);
+    }
+
+    @Override
+    public CompareOp getFilterOp() {
+        return CompareOp.EQUAL;
+    }
+
+    @Override
+    public CompareOp getInvertFilterOp() {
+        return CompareOp.EQUAL;
+    }
+}


[15/51] [partial] Initial commit of master branch from github

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..4faac66
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -0,0 +1,1428 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.jdbc.PhoenixDatabaseMetaData.TYPE_TABLE_NAME_BYTES;
+import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_DROP_METADATA;
+
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Properties;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableExistsException;
+import org.apache.hadoop.hbase.ZooKeeperConnectionException;
+import org.apache.hadoop.hbase.client.Append;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HConnection;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Increment;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.coprocessor.Batch;
+import org.apache.hadoop.hbase.index.Indexer;
+import org.apache.hadoop.hbase.index.covered.CoveredColumnsIndexBuilder;
+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.VersionInfo;
+import org.apache.hadoop.hbase.zookeeper.ZKConfig;
+import org.apache.phoenix.client.KeyValueBuilder;
+import org.apache.phoenix.compile.MutationPlan;
+import org.apache.phoenix.coprocessor.GroupedAggregateRegionObserver;
+import org.apache.phoenix.coprocessor.MetaDataEndpointImpl;
+import org.apache.phoenix.coprocessor.MetaDataProtocol;
+import org.apache.phoenix.coprocessor.MetaDataProtocol.MetaDataMutationResult;
+import org.apache.phoenix.coprocessor.MetaDataProtocol.MutationCode;
+import org.apache.phoenix.coprocessor.MetaDataRegionObserver;
+import org.apache.phoenix.coprocessor.ScanRegionObserver;
+import org.apache.phoenix.coprocessor.SequenceRegionObserver;
+import org.apache.phoenix.coprocessor.ServerCachingEndpointImpl;
+import org.apache.phoenix.coprocessor.UngroupedAggregateRegionObserver;
+import org.apache.phoenix.exception.PhoenixIOException;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+import org.apache.phoenix.execute.MutationState;
+import org.apache.phoenix.index.PhoenixIndexBuilder;
+import org.apache.phoenix.index.PhoenixIndexCodec;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver.ConnectionInfo;
+import org.apache.phoenix.schema.EmptySequenceCacheException;
+import org.apache.phoenix.schema.MetaDataSplitPolicy;
+import org.apache.phoenix.schema.NewerTableAlreadyExistsException;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PMetaData;
+import org.apache.phoenix.schema.PMetaDataImpl;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.schema.ReadOnlyTableException;
+import org.apache.phoenix.schema.Sequence;
+import org.apache.phoenix.schema.SequenceKey;
+import org.apache.phoenix.schema.TableNotFoundException;
+import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.JDBCUtil;
+import org.apache.phoenix.util.MetaDataUtil;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.ServerUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+public class ConnectionQueryServicesImpl extends DelegateQueryServices implements ConnectionQueryServices {
+    private static final Logger logger = LoggerFactory.getLogger(ConnectionQueryServicesImpl.class);
+    private static final int INITIAL_CHILD_SERVICES_CAPACITY = 100;
+    private static final int DEFAULT_OUT_OF_ORDER_MUTATIONS_WAIT_TIME_MS = 1000;
+    protected final Configuration config;
+    // Copy of config.getProps(), but read-only to prevent synchronization that we
+    // don't need.
+    private final ReadOnlyProps props;
+    private final HConnection connection;
+    private final StatsManager statsManager;
+    private final ConcurrentHashMap<ImmutableBytesWritable,ConnectionQueryServices> childServices;
+    // Cache the latest meta data here for future connections
+    private volatile PMetaData latestMetaData = PMetaDataImpl.EMPTY_META_DATA;
+    private final Object latestMetaDataLock = new Object();
+    // Lowest HBase version on the cluster.
+    private int lowestClusterHBaseVersion = Integer.MAX_VALUE;
+    private boolean hasInvalidIndexConfiguration = false;
+    private int connectionCount = 0;
+    
+    private ConcurrentMap<SequenceKey,Sequence> sequenceMap = Maps.newConcurrentMap();
+    private KeyValueBuilder kvBuilder;
+
+    /**
+     * Construct a ConnectionQueryServicesImpl that represents a connection to an HBase
+     * cluster.
+     * @param services base services from where we derive our default configuration
+     * @param connectionInfo to provide connection information
+     * @throws SQLException
+     */
+    public ConnectionQueryServicesImpl(QueryServices services, ConnectionInfo connectionInfo) throws SQLException {
+        super(services);
+        Configuration config = HBaseFactoryProvider.getConfigurationFactory().getConfiguration();
+        for (Entry<String,String> entry : services.getProps()) {
+            config.set(entry.getKey(), entry.getValue());
+        }
+        for (Entry<String,String> entry : connectionInfo.asProps()) {
+            config.set(entry.getKey(), entry.getValue());
+        }
+        // Without making a copy of the configuration we cons up, we lose some of our properties
+        // on the server side during testing.
+        this.config = HBaseConfiguration.create(config);
+        this.props = new ReadOnlyProps(this.config.iterator());
+        try {
+            this.connection = HBaseFactoryProvider.getHConnectionFactory().createConnection(this.config);
+        } catch (ZooKeeperConnectionException e) {
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ESTABLISH_CONNECTION)
+                .setRootCause(e).build().buildException();
+        }
+        if (this.connection.isClosed()) { // TODO: why the heck doesn't this throw above?
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ESTABLISH_CONNECTION).build().buildException();
+        }
+        // TODO: should we track connection wide memory usage or just org-wide usage?
+        // If connection-wide, create a MemoryManager here, otherwise just use the one from the delegate
+        this.childServices = new ConcurrentHashMap<ImmutableBytesWritable,ConnectionQueryServices>(INITIAL_CHILD_SERVICES_CAPACITY);
+        int statsUpdateFrequencyMs = this.getProps().getInt(QueryServices.STATS_UPDATE_FREQ_MS_ATTRIB, QueryServicesOptions.DEFAULT_STATS_UPDATE_FREQ_MS);
+        int maxStatsAgeMs = this.getProps().getInt(QueryServices.MAX_STATS_AGE_MS_ATTRIB, QueryServicesOptions.DEFAULT_MAX_STATS_AGE_MS);
+        this.statsManager = new StatsManagerImpl(this, statsUpdateFrequencyMs, maxStatsAgeMs);
+
+        // find the HBase version and use that to determine the KeyValueBuilder that should be used
+        String hbaseVersion = VersionInfo.getVersion();
+        this.kvBuilder = KeyValueBuilder.get(hbaseVersion);
+    }
+
+    @Override
+    public StatsManager getStatsManager() {
+        return this.statsManager;
+    }
+    
+    @Override
+    public HTableInterface getTable(byte[] tableName) throws SQLException {
+        try {
+            return HBaseFactoryProvider.getHTableFactory().getTable(tableName, connection, getExecutor());
+        } catch (org.apache.hadoop.hbase.TableNotFoundException e) {
+            byte[][] schemaAndTableName = new byte[2][];
+            SchemaUtil.getVarChars(tableName, schemaAndTableName);
+            throw new TableNotFoundException(Bytes.toString(schemaAndTableName[0]), Bytes.toString(schemaAndTableName[1]));
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+    
+    @Override
+    public HTableDescriptor getTableDescriptor(byte[] tableName) throws SQLException {
+        try {
+            return getTable(tableName).getTableDescriptor();
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public ReadOnlyProps getProps() {
+        return props;
+    }
+
+    /**
+     * Closes the underlying connection to zookeeper. The QueryServices
+     * may not be used after that point. When a Connection is closed,
+     * this is not called, since these instances are pooled by the
+     * Driver. Instead, the Driver should call this if the QueryServices
+     * is ever removed from the pool
+     */
+    @Override
+    public void close() throws SQLException {
+        SQLException sqlE = null;
+        try {
+            // Clear Phoenix metadata cache before closing HConnection
+            clearCache();
+        } catch (SQLException e) {
+            sqlE = e;
+        } finally {
+            try {
+                connection.close();
+            } catch (IOException e) {
+                if (sqlE == null) {
+                    sqlE = ServerUtil.parseServerException(e);
+                } else {
+                    sqlE.setNextException(ServerUtil.parseServerException(e));
+                }
+                throw sqlE;
+            } finally {
+                super.close();
+            }
+        }
+    }    
+
+    protected ConnectionQueryServices newChildQueryService() {
+        return new ChildQueryServices(this);
+    }
+
+    /**
+     * Get (and create if necessary) a child QueryService for a given tenantId.
+     * The QueryService will be cached for the lifetime of the parent QueryService
+     * @param tenantId the tenant ID
+     * @return the child QueryService
+     */
+    @Override
+    public ConnectionQueryServices getChildQueryServices(ImmutableBytesWritable tenantId) {
+        ConnectionQueryServices childQueryService = childServices.get(tenantId);
+        if (childQueryService == null) {
+            childQueryService = newChildQueryService();
+            ConnectionQueryServices prevQueryService = childServices.putIfAbsent(tenantId, childQueryService);
+            return prevQueryService == null ? childQueryService : prevQueryService;
+        }
+        return childQueryService;
+    }
+
+    @Override
+    public void clearTableRegionCache(byte[] tableName) throws SQLException {
+        connection.clearRegionCache(tableName);
+    }
+    
+    @Override
+    public List<HRegionLocation> getAllTableRegions(byte[] tableName) throws SQLException {
+        /*
+         * Use HConnection.getRegionLocation as it uses the cache in HConnection, while getting
+         * all region locations from the HTable doesn't. 
+         */
+        int retryCount = 0, maxRetryCount = 1;
+        boolean reload =false;
+        while (true) {
+            try {
+                // We could surface the package projected HConnectionImplementation.getNumberOfCachedRegionLocations
+                // to get the sizing info we need, but this would require a new class in the same package and a cast
+                // to this implementation class, so it's probably not worth it.
+                List<HRegionLocation> locations = Lists.newArrayList();
+                byte[] currentKey = HConstants.EMPTY_START_ROW;
+                do {
+                  HRegionLocation regionLocation = connection.getRegionLocation(tableName, currentKey, reload);
+                  locations.add(regionLocation);
+                  currentKey = regionLocation.getRegionInfo().getEndKey();
+                } while (!Bytes.equals(currentKey, HConstants.EMPTY_END_ROW));
+                return locations;
+            } catch (IOException e) {
+                if (retryCount++ < maxRetryCount) { // One retry, in case split occurs while navigating
+                    reload = true;
+                    continue;
+                }
+                throw new SQLExceptionInfo.Builder(SQLExceptionCode.GET_TABLE_REGIONS_FAIL)
+                    .setRootCause(e).build().buildException();
+            }
+        }
+    }
+
+    @Override
+    public PMetaData addTable(PTable table) throws SQLException {
+        try {
+            // If existing table isn't older than new table, don't replace
+            // If a client opens a connection at an earlier timestamp, this can happen
+            PTable existingTable = latestMetaData.getTable(table.getName().getString());
+            if (existingTable.getTimeStamp() >= table.getTimeStamp()) {
+                return latestMetaData;
+            }
+        } catch (TableNotFoundException e) {
+        }
+        synchronized(latestMetaDataLock) {
+            latestMetaData = latestMetaData.addTable(table);
+            latestMetaDataLock.notifyAll();
+            return latestMetaData;
+        }
+    }
+
+    private static interface Mutator {
+        PMetaData mutate(PMetaData metaData) throws SQLException;
+    }
+
+    /**
+     * Ensures that metaData mutations are handled in the correct order
+     */
+    private PMetaData metaDataMutated(String tableName, long tableSeqNum, Mutator mutator) throws SQLException {
+        synchronized(latestMetaDataLock) {
+            PMetaData metaData = latestMetaData;
+            PTable table;
+            long endTime = System.currentTimeMillis() + DEFAULT_OUT_OF_ORDER_MUTATIONS_WAIT_TIME_MS;
+            while (true) {
+                try {
+                    try {
+                        table = metaData.getTable(tableName);
+                        /* If the table is at the prior sequence number, then we're good to go.
+                         * We know if we've got this far, that the server validated the mutations,
+                         * so we'd just need to wait until the other connection that mutated the same
+                         * table is processed.
+                         */
+                        if (table.getSequenceNumber() + 1 == tableSeqNum) {
+                            // TODO: assert that timeStamp is bigger that table timeStamp?
+                            metaData = mutator.mutate(metaData);
+                            break;
+                        } else if (table.getSequenceNumber() >= tableSeqNum) {
+                            logger.warn("Attempt to cache older version of " + tableName + ": current= " + table.getSequenceNumber() + ", new=" + tableSeqNum);
+                            break;
+                        }
+                    } catch (TableNotFoundException e) {
+                    }
+                    long waitTime = endTime - System.currentTimeMillis();
+                    // We waited long enough - just remove the table from the cache
+                    // and the next time it's used it'll be pulled over from the server.
+                    if (waitTime <= 0) {
+                        logger.warn("Unable to update meta data repo within " + (DEFAULT_OUT_OF_ORDER_MUTATIONS_WAIT_TIME_MS/1000) + " seconds for " + tableName);
+                        metaData = metaData.removeTable(tableName);
+                        break;
+                    }
+                    latestMetaDataLock.wait(waitTime);
+                } catch (InterruptedException e) {
+                    throw new SQLExceptionInfo.Builder(SQLExceptionCode.INTERRUPTED_EXCEPTION)
+                        .setRootCause(e).build().buildException(); // FIXME
+                }
+            }
+            latestMetaData = metaData;
+            latestMetaDataLock.notifyAll();
+            return metaData;
+        }
+     }
+
+    @Override
+    public PMetaData addColumn(final String tableName, final List<PColumn> columns, final long tableTimeStamp, final long tableSeqNum, final boolean isImmutableRows) throws SQLException {
+        return metaDataMutated(tableName, tableSeqNum, new Mutator() {
+            @Override
+            public PMetaData mutate(PMetaData metaData) throws SQLException {
+                try {
+                    return metaData.addColumn(tableName, columns, tableTimeStamp, tableSeqNum, isImmutableRows);
+                } catch (TableNotFoundException e) {
+                    // The DROP TABLE may have been processed first, so just ignore.
+                    return metaData;
+                }
+            }
+        });
+     }
+
+    @Override
+    public PMetaData removeTable(final String tableName) throws SQLException {
+        synchronized(latestMetaDataLock) {
+            latestMetaData = latestMetaData.removeTable(tableName);
+            latestMetaDataLock.notifyAll();
+            return latestMetaData;
+        }
+    }
+
+    @Override
+    public PMetaData removeColumn(final String tableName, final String familyName, final String columnName, final long tableTimeStamp, final long tableSeqNum) throws SQLException {
+        return metaDataMutated(tableName, tableSeqNum, new Mutator() {
+            @Override
+            public PMetaData mutate(PMetaData metaData) throws SQLException {
+                try {
+                    return metaData.removeColumn(tableName, familyName, columnName, tableTimeStamp, tableSeqNum);
+                } catch (TableNotFoundException e) {
+                    // The DROP TABLE may have been processed first, so just ignore.
+                    return metaData;
+                }
+            }
+        });
+    }
+
+
+    @Override
+    public PhoenixConnection connect(String url, Properties info) throws SQLException {
+        Long scn = JDBCUtil.getCurrentSCN(url, info);
+        PMetaData metaData = scn == null ? latestMetaData : PMetaDataImpl.pruneNewerTables(scn, latestMetaData);
+        return new PhoenixConnection(this, url, info, metaData);
+    }
+
+
+    private HColumnDescriptor generateColumnFamilyDescriptor(Pair<byte[],Map<String,Object>> family, PTableType tableType) throws SQLException {
+        HColumnDescriptor columnDesc = new HColumnDescriptor(family.getFirst());
+        if (tableType != PTableType.VIEW) {
+            columnDesc.setKeepDeletedCells(true);
+            columnDesc.setDataBlockEncoding(SchemaUtil.DEFAULT_DATA_BLOCK_ENCODING);
+            for (Entry<String,Object> entry : family.getSecond().entrySet()) {
+                String key = entry.getKey();
+                Object value = entry.getValue();
+                columnDesc.setValue(key, value == null ? null : value.toString());
+            }
+        }
+        return columnDesc;
+    }
+
+    private void modifyColumnFamilyDescriptor(HColumnDescriptor hcd, Pair<byte[],Map<String,Object>> family) throws SQLException {
+      for (Entry<String, Object> entry : family.getSecond().entrySet()) {
+        String key = entry.getKey();
+        Object value = entry.getValue();
+        hcd.setValue(key, value == null ? null : value.toString());
+      }
+      hcd.setKeepDeletedCells(true);
+    }
+    
+    private HTableDescriptor generateTableDescriptor(byte[] tableName, HTableDescriptor existingDesc, PTableType tableType, Map<String,Object> tableProps, List<Pair<byte[],Map<String,Object>>> families, byte[][] splits) throws SQLException {
+        HTableDescriptor descriptor = (existingDesc != null) ? new HTableDescriptor(existingDesc) : new HTableDescriptor(tableName);
+        for (Entry<String,Object> entry : tableProps.entrySet()) {
+            String key = entry.getKey();
+            Object value = entry.getValue();
+            descriptor.setValue(key, value == null ? null : value.toString());
+        }
+        if (families.isEmpty()) {
+            if (tableType != PTableType.VIEW) {
+                // Add dummy column family so we have key values for tables that 
+                HColumnDescriptor columnDescriptor = generateColumnFamilyDescriptor(new Pair<byte[],Map<String,Object>>(QueryConstants.EMPTY_COLUMN_BYTES,Collections.<String,Object>emptyMap()), tableType);
+                descriptor.addFamily(columnDescriptor);
+            }
+        } else {
+            for (Pair<byte[],Map<String,Object>> family : families) {
+                // If family is only in phoenix description, add it. otherwise, modify its property accordingly.
+                byte[] familyByte = family.getFirst();
+                if (descriptor.getFamily(familyByte) == null) {
+                    if (tableType == PTableType.VIEW) {
+                        String fullTableName = Bytes.toString(tableName);
+                        throw new ReadOnlyTableException(
+                                "The HBase column families for a read-only table must already exist",
+                                SchemaUtil.getSchemaNameFromFullName(fullTableName),
+                                SchemaUtil.getTableNameFromFullName(fullTableName),
+                                Bytes.toString(familyByte));
+                    }
+                    HColumnDescriptor columnDescriptor = generateColumnFamilyDescriptor(family, tableType);
+                    descriptor.addFamily(columnDescriptor);
+                } else {
+                    if (tableType != PTableType.VIEW) {
+                        modifyColumnFamilyDescriptor(descriptor.getFamily(familyByte), family);
+                    }
+                }
+            }
+        }
+        // The phoenix jar must be available on HBase classpath
+        try {
+            if (!descriptor.hasCoprocessor(ScanRegionObserver.class.getName())) {
+                descriptor.addCoprocessor(ScanRegionObserver.class.getName(), null, 1, null);
+            }
+            if (!descriptor.hasCoprocessor(UngroupedAggregateRegionObserver.class.getName())) {
+                descriptor.addCoprocessor(UngroupedAggregateRegionObserver.class.getName(), null, 1, null);
+            }
+            if (!descriptor.hasCoprocessor(GroupedAggregateRegionObserver.class.getName())) {
+                descriptor.addCoprocessor(GroupedAggregateRegionObserver.class.getName(), null, 1, null);
+            }
+            if (!descriptor.hasCoprocessor(ServerCachingEndpointImpl.class.getName())) {
+                descriptor.addCoprocessor(ServerCachingEndpointImpl.class.getName(), null, 1, null);
+            }
+            // TODO: better encapsulation for this
+            // Since indexes can't have indexes, don't install our indexing coprocessor for indexes. Also,
+            // don't install on the metadata table until we fix the TODO there.
+            if (tableType != PTableType.INDEX && !descriptor.hasCoprocessor(Indexer.class.getName())
+                  && !SchemaUtil.isMetaTable(tableName) && !SchemaUtil.isSequenceTable(tableName)) {
+                Map<String, String> opts = Maps.newHashMapWithExpectedSize(1);
+                opts.put(CoveredColumnsIndexBuilder.CODEC_CLASS_NAME_KEY, PhoenixIndexCodec.class.getName());
+                Indexer.enableIndexing(descriptor, PhoenixIndexBuilder.class, opts);
+            }
+            
+            // Setup split policy on Phoenix metadata table to ensure that the key values of a Phoenix table
+            // stay on the same region.
+            if (SchemaUtil.isMetaTable(tableName)) {
+                descriptor.setValue(SchemaUtil.UPGRADE_TO_2_0, Boolean.TRUE.toString());
+                descriptor.setValue(SchemaUtil.UPGRADE_TO_2_1, Boolean.TRUE.toString());
+                if (!descriptor.hasCoprocessor(MetaDataEndpointImpl.class.getName())) {
+                    descriptor.addCoprocessor(MetaDataEndpointImpl.class.getName(), null, 1, null);
+                }
+                if (!descriptor.hasCoprocessor(MetaDataRegionObserver.class.getName())) {
+                    descriptor.addCoprocessor(MetaDataRegionObserver.class.getName(), null, 2, null);
+                }
+            } else if (SchemaUtil.isSequenceTable(tableName)) {
+                if (!descriptor.hasCoprocessor(SequenceRegionObserver.class.getName())) {
+                    descriptor.addCoprocessor(SequenceRegionObserver.class.getName(), null, 1, null);
+                }
+            }
+        } catch (IOException e) {
+            throw ServerUtil.parseServerException(e);
+        }
+        return descriptor;
+    }
+
+    private void ensureFamilyCreated(byte[] tableName, PTableType tableType , Pair<byte[],Map<String,Object>> family) throws SQLException {
+        HBaseAdmin admin = null;
+        SQLException sqlE = null;
+        try {
+            admin = new HBaseAdmin(config);
+            try {
+                HTableDescriptor existingDesc = admin.getTableDescriptor(tableName);
+                HColumnDescriptor oldDescriptor = existingDesc.getFamily(family.getFirst());
+                HColumnDescriptor columnDescriptor = null;
+
+                if (oldDescriptor == null) {
+                    if (tableType == PTableType.VIEW) {
+                        String fullTableName = Bytes.toString(tableName);
+                        throw new ReadOnlyTableException(
+                                "The HBase column families for a read-only table must already exist",
+                                SchemaUtil.getSchemaNameFromFullName(fullTableName),
+                                SchemaUtil.getTableNameFromFullName(fullTableName),
+                                Bytes.toString(family.getFirst()));
+                    }
+                    columnDescriptor = generateColumnFamilyDescriptor(family, tableType );
+                } else {
+                    columnDescriptor = new HColumnDescriptor(oldDescriptor);
+                    if (tableType != PTableType.VIEW) {
+                        modifyColumnFamilyDescriptor(columnDescriptor, family);
+                    }
+                }
+                
+                if (columnDescriptor.equals(oldDescriptor)) {
+                    // Table already has family and it's the same.
+                    return;
+                }
+                admin.disableTable(tableName);
+                if (oldDescriptor == null) {
+                    admin.addColumn(tableName, columnDescriptor);
+                } else {
+                    admin.modifyColumn(tableName, columnDescriptor);
+                }
+                admin.enableTable(tableName);
+            } catch (org.apache.hadoop.hbase.TableNotFoundException e) {
+                sqlE = new SQLExceptionInfo.Builder(SQLExceptionCode.TABLE_UNDEFINED).setRootCause(e).build().buildException();
+            }
+        } catch (IOException e) {
+            sqlE = ServerUtil.parseServerException(e);
+        } finally {
+            try {
+                if (admin != null) {
+                    admin.close();
+                }
+            } catch (IOException e) {
+                if (sqlE == null) {
+                    sqlE = ServerUtil.parseServerException(e);
+                } else {
+                    sqlE.setNextException(ServerUtil.parseServerException(e));
+                }
+            } finally {
+                if (sqlE != null) {
+                    throw sqlE;
+                }
+            }
+        }
+    }
+    
+    /**
+     * 
+     * @param tableName
+     * @param familyNames
+     * @param splits
+     * @return true if table was created and false if it already exists
+     * @throws SQLException
+     */
+    private boolean ensureTableCreated(byte[] tableName, PTableType tableType , Map<String,Object> props, List<Pair<byte[],Map<String,Object>>> families, byte[][] splits) throws SQLException {
+        HBaseAdmin admin = null;
+        SQLException sqlE = null;
+        HTableDescriptor existingDesc = null;
+        boolean isMetaTable = SchemaUtil.isMetaTable(tableName);
+        boolean tableExist = true;
+        try {
+            System.out.println("Found quorum: " + ZKConfig.getZKQuorumServersString(config));
+            admin = new HBaseAdmin(config);
+            try {
+                existingDesc = admin.getTableDescriptor(tableName);
+            } catch (org.apache.hadoop.hbase.TableNotFoundException e) {
+                tableExist = false;
+                if (tableType == PTableType.VIEW) {
+                    String fullTableName = Bytes.toString(tableName);
+                    throw new ReadOnlyTableException(
+                            "An HBase table for a VIEW must already exist",
+                            SchemaUtil.getSchemaNameFromFullName(fullTableName),
+                            SchemaUtil.getTableNameFromFullName(fullTableName));
+                }
+            }
+
+            HTableDescriptor newDesc = generateTableDescriptor(tableName, existingDesc, tableType , props, families, splits);
+            
+            if (!tableExist) {
+                /*
+                 * Remove the splitPolicy attribute due to an HBase bug (see below)
+                 */
+                if (isMetaTable) {
+                    newDesc.remove(HTableDescriptor.SPLIT_POLICY);
+                }
+                try {
+                    if (splits == null) {
+                        admin.createTable(newDesc);
+                    } else {
+                        admin.createTable(newDesc, splits);
+                    }
+                } catch (TableExistsException e) {
+                    // We can ignore this, as it just means that another client beat us
+                    // to creating the HBase metadata.
+                    return false;
+                }
+                if (isMetaTable) {
+                    checkClientServerCompatibility();
+                    /*
+                     * Now we modify the table to add the split policy, since we know that the client and
+                     * server and compatible. This works around a nasty, known HBase bug where if a split
+                     * policy class cannot be found on the server, the HBase table is left in a horrible
+                     * "ghost" state where it can't be used and can't be deleted without bouncing the master. 
+                     */
+                    newDesc.setValue(HTableDescriptor.SPLIT_POLICY, MetaDataSplitPolicy.class.getName());
+                    admin.disableTable(tableName);
+                    admin.modifyTable(tableName, newDesc);
+                    admin.enableTable(tableName);
+                }
+                return true;
+            } else {
+                if (existingDesc.equals(newDesc)) {
+                    // Table is already created. Note that the presplits are ignored in this case
+                    if (isMetaTable) {
+                        checkClientServerCompatibility();
+                    }
+                    return false;
+                }
+
+                if (isMetaTable) {
+                    checkClientServerCompatibility();
+                }
+                
+                // We'll do this alter at the end of the upgrade
+                // Just let the table metadata be updated for 3.0 here, as otherwise
+                // we have a potential race condition
+                // Update metadata of table
+                // TODO: Take advantage of online schema change ability by setting "hbase.online.schema.update.enable" to true
+                admin.disableTable(tableName);
+                // TODO: What if not all existing column families are present?
+                admin.modifyTable(tableName, newDesc);
+                admin.enableTable(tableName);
+                return false;
+            }
+
+        } catch (IOException e) {
+            sqlE = ServerUtil.parseServerException(e);
+        } finally {
+            try {
+                if (admin != null) {
+                    admin.close();
+                }
+            } catch (IOException e) {
+                if (sqlE == null) {
+                    sqlE = ServerUtil.parseServerException(e);
+                } else {
+                    sqlE.setNextException(ServerUtil.parseServerException(e));
+                }
+            } finally {
+                if (sqlE != null) {
+                    throw sqlE;
+                }
+            }
+        }
+        return true; // will never make it here
+    }
+
+    private static boolean isInvalidMutableIndexConfig(Long serverVersion) {
+        if (serverVersion == null) {
+            return false;
+        }
+        return !MetaDataUtil.decodeMutableIndexConfiguredProperly(serverVersion);
+    }
+    
+    private static boolean isCompatible(Long serverVersion) {
+        if (serverVersion == null) {
+            return false;
+        }
+        return MetaDataUtil.areClientAndServerCompatible(serverVersion);
+    }
+
+    private void checkClientServerCompatibility() throws SQLException {
+        StringBuilder buf = new StringBuilder("The following servers require an updated " + QueryConstants.DEFAULT_COPROCESS_PATH + " to be put in the classpath of HBase: ");
+        boolean isIncompatible = false;
+        int minHBaseVersion = Integer.MAX_VALUE;
+        try {
+            List<HRegionLocation> locations = this.getAllTableRegions(TYPE_TABLE_NAME_BYTES);
+            Set<HRegionLocation> serverMap = Sets.newHashSetWithExpectedSize(locations.size());
+            TreeMap<byte[], HRegionLocation> regionMap = Maps.newTreeMap(Bytes.BYTES_COMPARATOR);
+            List<byte[]> regionKeys = Lists.newArrayListWithExpectedSize(locations.size());
+            for (HRegionLocation entry : locations) {
+                if (!serverMap.contains(entry)) {
+                    regionKeys.add(entry.getRegionInfo().getStartKey());
+                    regionMap.put(entry.getRegionInfo().getRegionName(), entry);
+                    serverMap.add(entry);
+                }
+            }
+            final TreeMap<byte[],Long> results = Maps.newTreeMap(Bytes.BYTES_COMPARATOR);
+            connection.processExecs(MetaDataProtocol.class, regionKeys,
+                    PhoenixDatabaseMetaData.TYPE_TABLE_NAME_BYTES, this.getDelegate().getExecutor(), new Batch.Call<MetaDataProtocol,Long>() {
+                        @Override
+                        public Long call(MetaDataProtocol instance) throws IOException {
+                            return instance.getVersion();
+                        }
+                    }, 
+                    new Batch.Callback<Long>(){
+                        @Override
+                        public void update(byte[] region, byte[] row, Long value) {
+                          results.put(region, value);
+                        }
+                    });
+            for (Map.Entry<byte[],Long> result : results.entrySet()) {
+                // This is the "phoenix.jar" is in-place, but server is out-of-sync with client case.
+                if (!isCompatible(result.getValue())) {
+                    isIncompatible = true;
+                    HRegionLocation name = regionMap.get(result.getKey());
+                    buf.append(name);
+                    buf.append(';');
+                }
+                hasInvalidIndexConfiguration |= isInvalidMutableIndexConfig(result.getValue());
+                if (minHBaseVersion > MetaDataUtil.decodeHBaseVersion(result.getValue())) {
+                    minHBaseVersion = MetaDataUtil.decodeHBaseVersion(result.getValue());
+                }
+            }
+            lowestClusterHBaseVersion = minHBaseVersion;
+        } catch (Throwable t) {
+            // This is the case if the "phoenix.jar" is not on the classpath of HBase on the region server
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode.INCOMPATIBLE_CLIENT_SERVER_JAR).setRootCause(t)
+                .setMessage("Ensure that " + QueryConstants.DEFAULT_COPROCESS_PATH + " is put on the classpath of HBase in every region server: " + t.getMessage())
+                .build().buildException();
+        }
+        if (isIncompatible) {
+            buf.setLength(buf.length()-1);
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode.OUTDATED_JARS).setMessage(buf.toString()).build().buildException();
+        }
+    }
+
+    /**
+     * Invoke meta data coprocessor with one retry if the key was found to not be in the regions
+     * (due to a table split)
+     */
+    private MetaDataMutationResult metaDataCoprocessorExec(byte[] tableKey, Batch.Call<MetaDataProtocol, MetaDataMutationResult> callable) throws SQLException {
+        try {
+            boolean retried = false;
+            while (true) {
+                HRegionLocation regionLocation = retried ? connection.relocateRegion(PhoenixDatabaseMetaData.TYPE_TABLE_NAME_BYTES, tableKey) : connection.locateRegion(PhoenixDatabaseMetaData.TYPE_TABLE_NAME_BYTES, tableKey);
+                List<byte[]> regionKeys = Collections.singletonList(regionLocation.getRegionInfo().getStartKey());
+                final Map<byte[],MetaDataMutationResult> results = Maps.newHashMapWithExpectedSize(1);
+                connection.processExecs(MetaDataProtocol.class, regionKeys,
+                        PhoenixDatabaseMetaData.TYPE_TABLE_NAME_BYTES, this.getDelegate().getExecutor(), callable, 
+                        new Batch.Callback<MetaDataMutationResult>(){
+                            @Override
+                            public void update(byte[] region, byte[] row, MetaDataMutationResult value) {
+                              results.put(region, value);
+                            }
+                        });
+                assert(results.size() == 1);
+                MetaDataMutationResult result = results.values().iterator().next();
+                if (result.getMutationCode() == MutationCode.TABLE_NOT_IN_REGION) {
+                    if (retried) return result;
+                    retried = true;
+                    continue;
+                }
+                return result;
+            }
+        } catch (IOException e) {
+            throw ServerUtil.parseServerException(e);
+        } catch (Throwable t) {
+            throw new SQLException(t);
+        }
+    }
+
+
+    @Override
+    public MetaDataMutationResult createTable(final List<Mutation> tableMetaData, byte[] physicalTableName, PTableType tableType,
+            Map<String,Object> tableProps, final List<Pair<byte[],Map<String,Object>>> families, byte[][] splits) throws SQLException {
+        byte[][] rowKeyMetadata = new byte[3][];
+        Mutation m = tableMetaData.get(0);
+        byte[] key = m.getRow();
+        SchemaUtil.getVarChars(key, rowKeyMetadata);
+        byte[] tenantIdBytes = rowKeyMetadata[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
+        byte[] schemaBytes = rowKeyMetadata[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
+        byte[] tableBytes = rowKeyMetadata[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
+        if (tableType != PTableType.VIEW || physicalTableName != null) {
+            if (physicalTableName == null) {
+                physicalTableName = SchemaUtil.getTableNameAsBytes(schemaBytes, tableBytes);
+            }
+            ensureTableCreated(physicalTableName, tableType, tableProps, families, splits);
+        }
+        
+        byte[] tableKey = SchemaUtil.getTableKey(tenantIdBytes, schemaBytes, tableBytes);
+        MetaDataMutationResult result = metaDataCoprocessorExec(tableKey,
+            new Batch.Call<MetaDataProtocol, MetaDataMutationResult>() {
+                @Override
+                public MetaDataMutationResult call(MetaDataProtocol instance) throws IOException {
+                  return instance.createTable(tableMetaData);
+                }
+            });
+        return result;
+    }
+
+    @Override
+    public MetaDataMutationResult getTable(byte[] tenantId, final byte[] schemaBytes, final byte[] tableBytes,
+            final long tableTimestamp, final long clientTimestamp) throws SQLException {
+        final byte[] nonNullTenantId = tenantId == null ? ByteUtil.EMPTY_BYTE_ARRAY : tenantId;
+        byte[] tableKey = SchemaUtil.getTableKey(nonNullTenantId, schemaBytes, tableBytes);
+        return metaDataCoprocessorExec(tableKey,
+                new Batch.Call<MetaDataProtocol, MetaDataMutationResult>() {
+                    @Override
+                    public MetaDataMutationResult call(MetaDataProtocol instance) throws IOException {
+                      return instance.getTable(nonNullTenantId, schemaBytes, tableBytes, tableTimestamp, clientTimestamp);
+                    }
+                });
+    }
+
+    @Override
+    public MetaDataMutationResult dropTable(final List<Mutation> tableMetaData, final PTableType tableType) throws SQLException {
+        byte[][] rowKeyMetadata = new byte[3][];
+        SchemaUtil.getVarChars(tableMetaData.get(0).getRow(), rowKeyMetadata);
+        byte[] tenantIdBytes = rowKeyMetadata[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
+        byte[] schemaBytes = rowKeyMetadata[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
+        byte[] tableBytes = rowKeyMetadata[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
+        byte[] tableKey = SchemaUtil.getTableKey(tenantIdBytes == null ? ByteUtil.EMPTY_BYTE_ARRAY : tenantIdBytes, schemaBytes, tableBytes);
+        final MetaDataMutationResult result =  metaDataCoprocessorExec(tableKey,
+                new Batch.Call<MetaDataProtocol, MetaDataMutationResult>() {
+                    @Override
+                    public MetaDataMutationResult call(MetaDataProtocol instance) throws IOException {
+                      return instance.dropTable(tableMetaData, tableType.getSerializedValue());
+                    }
+                });
+        
+        final MutationCode code = result.getMutationCode();
+        switch(code) {
+        case TABLE_ALREADY_EXISTS:
+            final ReadOnlyProps props = this.getProps();
+            final boolean dropMetadata = props.getBoolean(DROP_METADATA_ATTRIB, DEFAULT_DROP_METADATA);
+            if (dropMetadata) {
+                dropTables(result.getTableNamesToDelete());
+            }
+            break;
+        default:
+            break;
+        }
+          return result;
+    }
+    
+    private void dropTables(final List<byte[]> tableNamesToDelete) throws SQLException {
+        HBaseAdmin admin = null;
+        SQLException sqlE = null;
+        try{
+            admin = new HBaseAdmin(config);
+            if (tableNamesToDelete != null){
+                for ( byte[] tableName : tableNamesToDelete ) {
+                    if ( admin.tableExists(tableName) ) {
+                        admin.disableTable(tableName);
+                        admin.deleteTable(tableName);
+                    }
+                }
+            }
+            
+        } catch (IOException e) {
+            sqlE = ServerUtil.parseServerException(e);
+        } finally {
+            try {
+                if (admin != null) {
+                    admin.close();
+                }
+            } catch (IOException e) {
+                if (sqlE == null) {
+                    sqlE = ServerUtil.parseServerException(e);
+                } else {
+                    sqlE.setNextException(ServerUtil.parseServerException(e));
+                }
+            } finally {
+                if (sqlE != null) {
+                    throw sqlE;
+                }
+            }
+        }
+    }
+
+    @Override
+    public MetaDataMutationResult addColumn(final List<Mutation> tableMetaData, PTableType tableType, List<Pair<byte[],Map<String,Object>>> families) throws SQLException {
+        byte[][] rowKeyMetaData = new byte[3][];
+
+        byte[] rowKey = tableMetaData.get(0).getRow();
+        SchemaUtil.getVarChars(rowKey, rowKeyMetaData);
+        byte[] tenantIdBytes = rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
+        byte[] schemaBytes = rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
+        byte[] tableBytes = rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
+        byte[] tableKey = SchemaUtil.getTableKey(tenantIdBytes, schemaBytes, tableBytes);
+        byte[] tableName = SchemaUtil.getTableNameAsBytes(schemaBytes, tableBytes);
+        for ( Pair<byte[],Map<String,Object>>  family : families) {
+            
+            PTable table = latestMetaData.getTable(Bytes.toString(tableName));
+            ensureFamilyCreated(table.getPhysicalName().getBytes(), tableType, family);
+        }
+        // Special case for call during drop table to ensure that the empty column family exists.
+        // In this, case we only include the table header row, as until we add schemaBytes and tableBytes
+        // as args to this function, we have no way of getting them in this case.
+        // TODO: change to  if (tableMetaData.isEmpty()) once we pass through schemaBytes and tableBytes
+        // Also, could be used to update property values on ALTER TABLE t SET prop=xxx
+        if (tableMetaData.size() == 1 && tableMetaData.get(0).isEmpty()) {
+            return null;
+        }
+        MetaDataMutationResult result =  metaDataCoprocessorExec(tableKey,
+            new Batch.Call<MetaDataProtocol, MetaDataMutationResult>() {
+                @Override
+                public MetaDataMutationResult call(MetaDataProtocol instance) throws IOException {
+                    return instance.addColumn(tableMetaData);
+                }
+            });
+        return result;
+    }
+
+    @Override
+    public MetaDataMutationResult dropColumn(final List<Mutation> tableMetaData, PTableType tableType) throws SQLException {
+        byte[][] rowKeyMetadata = new byte[3][];
+        SchemaUtil.getVarChars(tableMetaData.get(0).getRow(), rowKeyMetadata);
+        byte[] tenantIdBytes = rowKeyMetadata[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
+        byte[] schemaBytes = rowKeyMetadata[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
+        byte[] tableBytes = rowKeyMetadata[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
+        byte[] tableKey = SchemaUtil.getTableKey(tenantIdBytes, schemaBytes, tableBytes);
+        MetaDataMutationResult result = metaDataCoprocessorExec(tableKey,
+            new Batch.Call<MetaDataProtocol, MetaDataMutationResult>() {
+                @Override
+                public MetaDataMutationResult call(MetaDataProtocol instance) throws IOException {
+                    return instance.dropColumn(tableMetaData);
+                }
+            });
+        final MutationCode code = result.getMutationCode();
+        switch(code) {
+        case TABLE_ALREADY_EXISTS:
+            final ReadOnlyProps props = this.getProps();
+            final boolean dropMetadata = props.getBoolean(DROP_METADATA_ATTRIB, DEFAULT_DROP_METADATA);
+            if (dropMetadata) {
+                dropTables(result.getTableNamesToDelete());
+            }
+            break;
+        default:
+            break;
+        }
+        return result;
+       
+    }
+
+    // Keeping this to use for further upgrades
+    protected PhoenixConnection addColumnsIfNotExists(PhoenixConnection oldMetaConnection, long timestamp, String columns) throws SQLException {
+        Properties props = new Properties(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(this, oldMetaConnection.getURL(), props, oldMetaConnection.getPMetaData());
+        SQLException sqlE = null;
+        try {
+            metaConnection.createStatement().executeUpdate("ALTER TABLE " + PhoenixDatabaseMetaData.TYPE_SCHEMA_AND_TABLE + " ADD IF NOT EXISTS " + columns );
+        } catch (SQLException e) {
+            sqlE = e;
+        } finally {
+            try {
+                oldMetaConnection.close();
+            } catch (SQLException e) {
+                if (sqlE != null) {
+                    sqlE.setNextException(e);
+                } else {
+                    sqlE = e;
+                }
+            }
+            if (sqlE != null) {
+                throw sqlE;
+            }
+        }
+        return metaConnection;
+    }
+    
+    @Override
+    public void init(String url, Properties props) throws SQLException {
+        props = new Properties(props);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP));
+        PhoenixConnection metaConnection = new PhoenixConnection(this, url, props, PMetaDataImpl.EMPTY_META_DATA);
+        SQLException sqlE = null;
+        try {
+            try {
+                metaConnection.createStatement().executeUpdate(QueryConstants.CREATE_TABLE_METADATA);
+            } catch (NewerTableAlreadyExistsException ignore) {
+                // Ignore, as this will happen if the SYSTEM.TABLE already exists at this fixed timestamp.
+                // A TableAlreadyExistsException is not thrown, since the table only exists *after* this fixed timestamp.
+            }
+            try {
+                metaConnection.createStatement().executeUpdate(QueryConstants.CREATE_SEQUENCE_METADATA);
+            } catch (NewerTableAlreadyExistsException ignore) {
+                // Ignore, as this will happen if the SYSTEM.SEQUENCE already exists at this fixed timestamp.
+                // A TableAlreadyExistsException is not thrown, since the table only exists *after* this fixed timestamp.
+            }
+        } catch (SQLException e) {
+            sqlE = e;
+        } finally {
+            try {
+                metaConnection.close();
+            } catch (SQLException e) {
+                if (sqlE != null) {
+                    sqlE.setNextException(e);
+                } else {
+                    sqlE = e;
+                }
+            }
+            if (sqlE != null) {
+                throw sqlE;
+            }
+        }
+    }
+
+    @Override
+    public MutationState updateData(MutationPlan plan) throws SQLException {
+        return plan.execute();
+    }
+
+    @Override
+    public int getLowestClusterHBaseVersion() {
+        return lowestClusterHBaseVersion;
+    }
+
+    @Override
+    public boolean hasInvalidIndexConfiguration() {
+        return hasInvalidIndexConfiguration;
+    }
+
+    /**
+     * Clears the Phoenix meta data cache on each region server
+     * @throws SQLException
+     */
+    protected void clearCache() throws SQLException {
+        try {
+            SQLException sqlE = null;
+            HTableInterface htable = this.getTable(PhoenixDatabaseMetaData.TYPE_TABLE_NAME_BYTES);
+            try {
+                htable.coprocessorExec(MetaDataProtocol.class, HConstants.EMPTY_START_ROW,
+                        HConstants.EMPTY_END_ROW, new Batch.Call<MetaDataProtocol, Void>() {
+                    @Override
+                    public Void call(MetaDataProtocol instance) throws IOException {
+                      instance.clearCache();
+                      return null;
+                    }
+                  });
+            } catch (IOException e) {
+                throw ServerUtil.parseServerException(e);
+            } catch (Throwable e) {
+                sqlE = new SQLException(e);
+            } finally {
+                try {
+                    htable.close();
+                } catch (IOException e) {
+                    if (sqlE == null) {
+                        sqlE = ServerUtil.parseServerException(e);
+                    } else {
+                        sqlE.setNextException(ServerUtil.parseServerException(e));
+                    }
+                } finally {
+                    if (sqlE != null) {
+                        throw sqlE;
+                    }
+                }
+            }
+        } catch (Exception e) {
+            throw new SQLException(ServerUtil.parseServerException(e));
+        }
+    }
+
+    @Override
+    public HBaseAdmin getAdmin() throws SQLException {
+        try {
+            return new HBaseAdmin(config);
+        } catch (IOException e) {
+            throw new PhoenixIOException(e);
+        }
+    }
+
+    @Override
+    public MetaDataMutationResult updateIndexState(final List<Mutation> tableMetaData, String parentTableName) throws SQLException {
+        byte[][] rowKeyMetadata = new byte[3][];
+        SchemaUtil.getVarChars(tableMetaData.get(0).getRow(), rowKeyMetadata);
+        byte[] tableKey = SchemaUtil.getTableKey(ByteUtil.EMPTY_BYTE_ARRAY, rowKeyMetadata[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX], rowKeyMetadata[PhoenixDatabaseMetaData.TABLE_NAME_INDEX]);
+        return metaDataCoprocessorExec(tableKey,
+                new Batch.Call<MetaDataProtocol, MetaDataMutationResult>() {
+                    @Override
+                    public MetaDataMutationResult call(MetaDataProtocol instance) throws IOException {
+                      return instance.updateIndexState(tableMetaData);
+                    }
+                });
+    }
+
+    @Override
+    public long createSequence(String tenantId, String schemaName, String sequenceName, long startWith, long incrementBy, int cacheSize, long timestamp) 
+            throws SQLException {
+        SequenceKey sequenceKey = new SequenceKey(tenantId, schemaName, sequenceName);
+        Sequence newSequences = new Sequence(sequenceKey);
+        Sequence sequence = sequenceMap.putIfAbsent(sequenceKey, newSequences);
+        if (sequence == null) {
+            sequence = newSequences;
+        }
+        try {
+            sequence.getLock().lock();
+            // Now that we have the lock we need, create the sequence
+            Append append = sequence.createSequence(startWith, incrementBy, cacheSize, timestamp);
+            HTableInterface htable = this.getTable(PhoenixDatabaseMetaData.SEQUENCE_TABLE_NAME_BYTES);
+            try {
+                Result result = htable.append(append);
+                return sequence.createSequence(result);
+            } catch (IOException e) {
+                throw ServerUtil.parseServerException(e);
+            }
+        } finally {
+            sequence.getLock().unlock();
+        }
+    }
+
+    @Override
+    public long dropSequence(String tenantId, String schemaName, String sequenceName, long timestamp) throws SQLException {
+        SequenceKey sequenceKey = new SequenceKey(tenantId, schemaName, sequenceName);
+        Sequence newSequences = new Sequence(sequenceKey);
+        Sequence sequence = sequenceMap.putIfAbsent(sequenceKey, newSequences);
+        if (sequence == null) {
+            sequence = newSequences;
+        }
+        try {
+            sequence.getLock().lock();
+            // Now that we have the lock we need, create the sequence
+            Append append = sequence.dropSequence(timestamp);
+            HTableInterface htable = this.getTable(PhoenixDatabaseMetaData.SEQUENCE_TABLE_NAME_BYTES);
+            try {
+                Result result = htable.append(append);
+                return sequence.dropSequence(result);
+            } catch (IOException e) {
+                throw ServerUtil.parseServerException(e);
+            }
+        } finally {
+            sequence.getLock().unlock();
+        }
+    }
+
+    /**
+     * Gets the current sequence value
+     * @param tenantId
+     * @param sequence
+     * @return
+     * @throws SQLException if cached sequence cannot be found 
+     */
+    @Override
+    public long getSequenceValue(SequenceKey sequenceKey, long timestamp) throws SQLException {
+        Sequence sequence = sequenceMap.get(sequenceKey);
+        if (sequence == null) {
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_CALL_CURRENT_BEFORE_NEXT_VALUE)
+            .setSchemaName(sequenceKey.getSchemaName()).setTableName(sequenceKey.getSequenceName())
+            .build().buildException();
+        }
+        sequence.getLock().lock();
+        try {
+            return sequence.currentValue(timestamp);
+        } catch (EmptySequenceCacheException e) {
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_CALL_CURRENT_BEFORE_NEXT_VALUE)
+            .setSchemaName(sequenceKey.getSchemaName()).setTableName(sequenceKey.getSequenceName())
+            .build().buildException();
+        } finally {
+            sequence.getLock().unlock();
+        }
+    }
+    
+    /**
+     * Verifies that sequences exist and reserves values for them.
+     * @param tenantId
+     * @param sequences
+     * @param timestamp
+     * @throws SQLException if any sequence does not exist
+     */
+    @Override
+    public void reserveSequenceValues(List<SequenceKey> sequenceKeys, long timestamp, long[] values, SQLException[] exceptions) throws SQLException {
+        incrementSequenceValues(sequenceKeys, timestamp, values, exceptions, 0);
+    }
+    
+    /**
+     * Increment any of the set of sequences that need more values. These are the sequences
+     * that are asking for the next value within a given statement. The returned sequences
+     * are the ones that were not found because they were deleted by another client. 
+     * @param tenantId
+     * @param sequenceKeys sorted list of sequence kyes
+     * @param batchSize
+     * @param timestamp
+     * @return
+     * @throws SQLException if any of the sequences cannot be found
+     * 
+     * PSequences -> Sequence
+     * PSequenceKey -> SequenceKey
+     */
+    @Override
+    public void incrementSequenceValues(List<SequenceKey> sequenceKeys, long timestamp, long[] values, SQLException[] exceptions) throws SQLException {
+        incrementSequenceValues(sequenceKeys, timestamp, values, exceptions, 1);
+    }
+
+    private void incrementSequenceValues(List<SequenceKey> keys, long timestamp, long[] values, SQLException[] exceptions, int factor) throws SQLException {
+        List<Sequence> sequences = Lists.newArrayListWithExpectedSize(keys.size());
+        for (SequenceKey key : keys) {
+            Sequence newSequences = new Sequence(key);
+            Sequence sequence = sequenceMap.putIfAbsent(key, newSequences);
+            if (sequence == null) {
+                sequence = newSequences;
+            }
+            sequences.add(sequence);
+        }
+        try {
+            for (Sequence sequence : sequences) {
+                sequence.getLock().lock();
+            }
+            // Now that we have all the locks we need, increment the sequences
+            List<Increment> incrementBatch = Lists.newArrayListWithExpectedSize(sequences.size());
+            List<Sequence> toIncrementList = Lists.newArrayListWithExpectedSize(sequences.size());
+            int[] indexes = new int[sequences.size()];
+            for (int i = 0; i < sequences.size(); i++) {
+                Sequence sequence = sequences.get(i);
+                try {
+                    values[i] = sequence.incrementValue(timestamp, factor);
+                } catch (EmptySequenceCacheException e) {
+                    indexes[toIncrementList.size()] = i;
+                    toIncrementList.add(sequence);
+                    Increment inc = sequence.newIncrement(timestamp);
+                    incrementBatch.add(inc);
+                }
+            }
+            if (toIncrementList.isEmpty()) {
+                return;
+            }
+            HTableInterface hTable = this.getTable(PhoenixDatabaseMetaData.SEQUENCE_TABLE_NAME_BYTES);
+            Object[] resultObjects = null;
+            SQLException sqlE = null;
+            try {
+                resultObjects= hTable.batch(incrementBatch);
+            } catch (IOException e){
+                sqlE = ServerUtil.parseServerException(e);
+            } catch (InterruptedException e){
+                sqlE = new SQLExceptionInfo.Builder(SQLExceptionCode.INTERRUPTED_EXCEPTION)
+                .setRootCause(e).build().buildException(); // FIXME ?
+            } finally {
+                try {
+                    hTable.close();
+                } catch (IOException e) {
+                    if (sqlE == null) {
+                        sqlE = ServerUtil.parseServerException(e);
+                    } else {
+                        sqlE.setNextException(ServerUtil.parseServerException(e));
+                    }
+                }
+                if (sqlE != null) {
+                    throw sqlE;
+                }
+            }
+            for (int i=0;i<resultObjects.length;i++){
+                Sequence sequence = toIncrementList.get(i);
+                Result result = (Result)resultObjects[i];
+                try {
+                    values[indexes[i]] = sequence.incrementValue(result, factor);
+                } catch (SQLException e) {
+                    exceptions[indexes[i]] = e;
+                }
+            }
+        } finally {
+            for (Sequence sequence : sequences) {
+                sequence.getLock().unlock();
+            }
+        }
+    }
+
+    @Override
+    public void returnSequenceValues(List<SequenceKey> keys, long timestamp, SQLException[] exceptions) throws SQLException {
+        List<Sequence> sequences = Lists.newArrayListWithExpectedSize(keys.size());
+        for (SequenceKey key : keys) {
+            Sequence newSequences = new Sequence(key);
+            Sequence sequence = sequenceMap.putIfAbsent(key, newSequences);
+            if (sequence == null) {
+                sequence = newSequences;
+            }
+            sequences.add(sequence);
+        }
+        try {
+            for (Sequence sequence : sequences) {
+                sequence.getLock().lock();
+            }
+            // Now that we have all the locks we need, attempt to return the unused sequence values
+            List<Append> mutations = Lists.newArrayListWithExpectedSize(sequences.size());
+            List<Sequence> toReturnList = Lists.newArrayListWithExpectedSize(sequences.size());
+            int[] indexes = new int[sequences.size()];
+            for (int i = 0; i < sequences.size(); i++) {
+                Sequence sequence = sequences.get(i);
+                try {
+                    Append append = sequence.newReturn(timestamp);
+                    toReturnList.add(sequence);
+                    mutations.add(append);
+                } catch (EmptySequenceCacheException ignore) { // Nothing to return, so ignore
+                }
+            }
+            if (toReturnList.isEmpty()) {
+                return;
+            }
+            HTableInterface hTable = this.getTable(PhoenixDatabaseMetaData.SEQUENCE_TABLE_NAME_BYTES);
+            Object[] resultObjects = null;
+            SQLException sqlE = null;
+            try {
+                resultObjects= hTable.batch(mutations);
+            } catch (IOException e){
+                sqlE = ServerUtil.parseServerException(e);
+            } catch (InterruptedException e){
+                sqlE = new SQLExceptionInfo.Builder(SQLExceptionCode.INTERRUPTED_EXCEPTION)
+                .setRootCause(e).build().buildException(); // FIXME ?
+            } finally {
+                try {
+                    hTable.close();
+                } catch (IOException e) {
+                    if (sqlE == null) {
+                        sqlE = ServerUtil.parseServerException(e);
+                    } else {
+                        sqlE.setNextException(ServerUtil.parseServerException(e));
+                    }
+                }
+                if (sqlE != null) {
+                    throw sqlE;
+                }
+            }
+            for (int i=0;i<resultObjects.length;i++){
+                Sequence sequence = toReturnList.get(i);
+                Result result = (Result)resultObjects[i];
+                try {
+                    sequence.returnValue(result);
+                } catch (SQLException e) {
+                    exceptions[indexes[i]] = e;
+                }
+            }
+        } finally {
+            for (Sequence sequence : sequences) {
+                sequence.getLock().unlock();
+            }
+        }
+    }
+
+    // Take no locks, as this only gets run when there are no open connections
+    // so there's no danger of contention.
+    private void returnAllSequenceValues(ConcurrentMap<SequenceKey,Sequence> sequenceMap) throws SQLException {
+        List<Append> mutations = Lists.newArrayListWithExpectedSize(sequenceMap.size());
+        for (Sequence sequence : sequenceMap.values()) {
+            mutations.addAll(sequence.newReturns());
+        }
+        if (mutations.isEmpty()) {
+            return;
+        }
+        HTableInterface hTable = this.getTable(PhoenixDatabaseMetaData.SEQUENCE_TABLE_NAME_BYTES);
+        SQLException sqlE = null;
+        try {
+            hTable.batch(mutations);
+        } catch (IOException e){
+            sqlE = ServerUtil.parseServerException(e);
+        } catch (InterruptedException e){
+            sqlE = new SQLExceptionInfo.Builder(SQLExceptionCode.INTERRUPTED_EXCEPTION)
+            .setRootCause(e).build().buildException(); // FIXME ?
+        } finally {
+            try {
+                hTable.close();
+            } catch (IOException e) {
+                if (sqlE == null) {
+                    sqlE = ServerUtil.parseServerException(e);
+                } else {
+                    sqlE.setNextException(ServerUtil.parseServerException(e));
+                }
+            }
+            if (sqlE != null) {
+                throw sqlE;
+            }
+        }
+    }
+    
+    @Override
+    public synchronized void addConnection(PhoenixConnection connection) throws SQLException {
+        connectionCount++;
+    }
+
+    @Override
+    public void removeConnection(PhoenixConnection connection) throws SQLException {
+        ConcurrentMap<SequenceKey,Sequence> formerSequenceMap = null;
+        synchronized(this) {
+            if (--connectionCount == 0) {
+                if (!this.sequenceMap.isEmpty()) {
+                    formerSequenceMap = this.sequenceMap;
+                    this.sequenceMap = Maps.newConcurrentMap();
+                }
+            }
+        }
+        // Since we're using the former sequenceMap, we can do this outside
+        // the lock.
+        if (formerSequenceMap != null) {
+            // When there are no more connections, attempt to return any sequences
+            returnAllSequenceValues(formerSequenceMap);
+        }
+    }
+
+    @Override
+    public KeyValueBuilder getKeyValueBuilder() {
+        return this.kvBuilder;
+    }
+
+    @Override
+    public boolean supportsFeature(Feature feature) {
+        // TODO: Keep map of Feature -> min HBase version
+        // For now, only Feature is REVERSE_SCAN and it's not supported in any version yet
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..dd7cc5f
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
@@ -0,0 +1,360 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.jdbc.PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES;
+
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Mutation;
+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.VersionInfo;
+import org.apache.phoenix.client.KeyValueBuilder;
+import org.apache.phoenix.compile.MutationPlan;
+import org.apache.phoenix.coprocessor.MetaDataProtocol;
+import org.apache.phoenix.coprocessor.MetaDataProtocol.MetaDataMutationResult;
+import org.apache.phoenix.coprocessor.MetaDataProtocol.MutationCode;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+import org.apache.phoenix.execute.MutationState;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.schema.NewerTableAlreadyExistsException;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PIndexState;
+import org.apache.phoenix.schema.PMetaData;
+import org.apache.phoenix.schema.PMetaDataImpl;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTableImpl;
+import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.schema.SequenceAlreadyExistsException;
+import org.apache.phoenix.schema.SequenceKey;
+import org.apache.phoenix.schema.SequenceNotFoundException;
+import org.apache.phoenix.schema.TableAlreadyExistsException;
+import org.apache.phoenix.schema.TableNotFoundException;
+import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.SchemaUtil;
+
+import com.google.common.collect.Maps;
+
+
+/**
+ *
+ * Implementation of ConnectionQueryServices used in testing where no connection to
+ * an hbase cluster is necessary.
+ * 
+ * @author jtaylor
+ * @since 0.1
+ */
+public class ConnectionlessQueryServicesImpl extends DelegateQueryServices implements ConnectionQueryServices  {
+    private PMetaData metaData;
+    private final Map<SequenceKey, Long> sequenceMap = Maps.newHashMap();
+    private KeyValueBuilder kvBuilder;
+    
+    public ConnectionlessQueryServicesImpl(QueryServices queryServices) {
+        super(queryServices);
+        metaData = PMetaDataImpl.EMPTY_META_DATA;
+        // find the HBase version and use that to determine the KeyValueBuilder that should be used
+        String hbaseVersion = VersionInfo.getVersion();
+        this.kvBuilder = KeyValueBuilder.get(hbaseVersion);
+    }
+
+    @Override
+    public ConnectionQueryServices getChildQueryServices(ImmutableBytesWritable childId) {
+        return this; // Just reuse the same query services
+    }
+
+    @Override
+    public HTableInterface getTable(byte[] tableName) throws SQLException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public StatsManager getStatsManager() {
+        return new StatsManager() {
+
+            @Override
+            public byte[] getMinKey(TableRef table) {
+                return HConstants.EMPTY_START_ROW;
+            }
+
+            @Override
+            public byte[] getMaxKey(TableRef table) {
+                return HConstants.EMPTY_END_ROW;
+            }
+
+            @Override
+            public void updateStats(TableRef table) throws SQLException {
+            }
+        };
+    }
+
+    @Override
+    public List<HRegionLocation> getAllTableRegions(byte[] tableName) throws SQLException {
+        return Collections.singletonList(new HRegionLocation(new HRegionInfo(tableName, HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW),"localhost",-1));
+    }
+
+    @Override
+    public PMetaData addTable(PTable table) throws SQLException {
+        return metaData = metaData.addTable(table);
+    }
+
+    @Override
+    public PMetaData addColumn(String tableName, List<PColumn> columns, long tableTimeStamp, long tableSeqNum,
+            boolean isImmutableRows) throws SQLException {
+        return metaData = metaData.addColumn(tableName, columns, tableTimeStamp, tableSeqNum, isImmutableRows);
+    }
+
+    @Override
+    public PMetaData removeTable(String tableName)
+            throws SQLException {
+        return metaData = metaData.removeTable(tableName);
+    }
+
+    @Override
+    public PMetaData removeColumn(String tableName, String familyName, String columnName, long tableTimeStamp,
+            long tableSeqNum) throws SQLException {
+        return metaData = metaData.removeColumn(tableName, familyName, columnName, tableTimeStamp, tableSeqNum);
+    }
+
+    
+    @Override
+    public PhoenixConnection connect(String url, Properties info) throws SQLException {
+        return new PhoenixConnection(this, url, info, metaData);
+    }
+
+    @Override
+    public MetaDataMutationResult getTable(byte[] tenantId, byte[] schemaBytes, byte[] tableBytes, long tableTimestamp, long clientTimestamp) throws SQLException {
+        // Return result that will cause client to use it's own metadata instead of needing
+        // to get anything from the server (since we don't have a connection)
+        try {
+            PTable table = metaData.getTable(SchemaUtil.getTableName(schemaBytes, tableBytes));
+            return new MetaDataMutationResult(MutationCode.TABLE_ALREADY_EXISTS, 0, table);
+        } catch (TableNotFoundException e) {
+            return new MetaDataMutationResult(MutationCode.TABLE_NOT_FOUND, 0, null);
+        }
+        //return new MetaDataMutationResult(MutationCode.TABLE_ALREADY_EXISTS, 0, null);
+    }
+
+    @Override
+    public MetaDataMutationResult createTable(List<Mutation> tableMetaData, byte[] physicalName, PTableType tableType, Map<String,Object> tableProps, List<Pair<byte[],Map<String,Object>>> families, byte[][] splits) throws SQLException {
+        return new MetaDataMutationResult(MutationCode.TABLE_NOT_FOUND, 0, null);
+    }
+
+    @Override
+    public MetaDataMutationResult dropTable(List<Mutation> tableMetadata, PTableType tableType) throws SQLException {
+        return new MetaDataMutationResult(MutationCode.TABLE_ALREADY_EXISTS, 0, null);
+    }
+
+    @Override
+    public MetaDataMutationResult addColumn(List<Mutation> tableMetaData, PTableType readOnly, List<Pair<byte[],Map<String,Object>>> families) throws SQLException {
+        return new MetaDataMutationResult(MutationCode.TABLE_ALREADY_EXISTS, 0, null);
+    }
+
+    @Override
+    public MetaDataMutationResult dropColumn(List<Mutation> tableMetadata, PTableType tableType) throws SQLException {
+        return new MetaDataMutationResult(MutationCode.TABLE_ALREADY_EXISTS, 0, null);
+    }
+
+    @Override
+    public void init(String url, Properties props) throws SQLException {
+        props = new Properties(props);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP));
+        PhoenixConnection metaConnection = new PhoenixConnection(this, url, props, PMetaDataImpl.EMPTY_META_DATA);
+        SQLException sqlE = null;
+        try {
+            try {
+                metaConnection.createStatement().executeUpdate(QueryConstants.CREATE_TABLE_METADATA);
+            } catch (TableAlreadyExistsException ignore) {
+                // Ignore, as this will happen if the SYSTEM.TABLE already exists at this fixed timestamp.
+                // A TableAlreadyExistsException is not thrown, since the table only exists *after* this fixed timestamp.
+            }
+            try {
+                metaConnection.createStatement().executeUpdate(QueryConstants.CREATE_SEQUENCE_METADATA);
+            } catch (NewerTableAlreadyExistsException ignore) {
+                // Ignore, as this will happen if the SYSTEM.SEQUENCE already exists at this fixed timestamp.
+                // A TableAlreadyExistsException is not thrown, since the table only exists *after* this fixed timestamp.
+            }
+        } catch (SQLException e) {
+            sqlE = e;
+        } finally {
+            try {
+                metaConnection.close();
+            } catch (SQLException e) {
+                if (sqlE != null) {
+                    sqlE.setNextException(e);
+                } else {
+                    sqlE = e;
+                }
+            }
+            if (sqlE != null) {
+                throw sqlE;
+            }
+        }
+    }
+
+    @Override
+    public MutationState updateData(MutationPlan plan) throws SQLException {
+        return new MutationState(0, plan.getConnection());
+    }
+
+    @Override
+    public int getLowestClusterHBaseVersion() {
+        return Integer.MAX_VALUE; // Allow everything for connectionless
+    }
+
+    @Override
+    public HBaseAdmin getAdmin() throws SQLException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public MetaDataMutationResult updateIndexState(List<Mutation> tableMetadata, String parentTableName) throws SQLException {
+        byte[][] rowKeyMetadata = new byte[3][];
+        SchemaUtil.getVarChars(tableMetadata.get(0).getRow(), rowKeyMetadata);
+        KeyValue newKV = tableMetadata.get(0).getFamilyMap().get(TABLE_FAMILY_BYTES).get(0);
+        PIndexState newState =  PIndexState.fromSerializedValue(newKV.getBuffer()[newKV.getValueOffset()]);
+        String schemaName = Bytes.toString(rowKeyMetadata[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX]);
+        String indexName = Bytes.toString(rowKeyMetadata[PhoenixDatabaseMetaData.TABLE_NAME_INDEX]);
+        String indexTableName = SchemaUtil.getTableName(schemaName, indexName);
+        PTable index = metaData.getTable(indexTableName);
+        index = PTableImpl.makePTable(index,newState == PIndexState.USABLE ? PIndexState.ACTIVE : newState == PIndexState.UNUSABLE ? PIndexState.INACTIVE : newState);
+        return new MetaDataMutationResult(MutationCode.TABLE_ALREADY_EXISTS, 0, index);
+    }
+
+    @Override
+    public HTableDescriptor getTableDescriptor(byte[] tableName) throws SQLException {
+        return null;
+    }
+
+    @Override
+    public void clearTableRegionCache(byte[] tableName) throws SQLException {
+    }
+
+    @Override
+    public boolean hasInvalidIndexConfiguration() {
+        return false;
+    }
+
+    @Override
+    public long createSequence(String tenantId, String schemaName, String sequenceName, long startWith, long incrementBy, int cacheSize, long timestamp)
+            throws SQLException {
+        SequenceKey key = new SequenceKey(tenantId, schemaName, sequenceName);
+        if (sequenceMap.get(key) != null) {
+            throw new SequenceAlreadyExistsException(schemaName, sequenceName);
+        }
+        sequenceMap.put(key, startWith);
+        return timestamp;
+    }
+
+    @Override
+    public long dropSequence(String tenantId, String schemaName, String sequenceName, long timestamp) throws SQLException {
+        SequenceKey key = new SequenceKey(tenantId, schemaName, sequenceName);
+        if (sequenceMap.remove(key) == null) {
+            throw new SequenceNotFoundException(schemaName, sequenceName);
+        }
+        return timestamp;
+    }
+
+    @Override
+    public void reserveSequenceValues(List<SequenceKey> sequenceKeys, long timestamp, long[] values,
+            SQLException[] exceptions) throws SQLException {
+        int i = 0;
+        for (SequenceKey key : sequenceKeys) {
+            Long value = sequenceMap.get(key);
+            if (value == null) {
+                exceptions[i] = new SequenceNotFoundException(key.getSchemaName(), key.getSequenceName());
+            } else {
+                values[i] = value;          
+            }
+            i++;
+        }
+    }
+
+    @Override
+    public void incrementSequenceValues(List<SequenceKey> sequenceKeys, long timestamp, long[] values,
+            SQLException[] exceptions) throws SQLException {
+        int i = 0;
+        for (SequenceKey key : sequenceKeys) {
+            Long value = sequenceMap.get(key);
+            if (value == null) {
+                exceptions[i] = new SequenceNotFoundException(key.getSchemaName(), key.getSequenceName());
+            } else {
+                values[i] = value++;
+            }
+            i++;
+        }
+        i = 0;
+        for (SQLException e : exceptions) {
+            if (e != null) {
+                sequenceMap.remove(sequenceKeys.get(i));
+            }
+            i++;
+        }
+    }
+
+    @Override
+    public long getSequenceValue(SequenceKey sequenceKey, long timestamp) throws SQLException {
+        Long value = sequenceMap.get(sequenceKey);
+        if (value == null) {
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_CALL_CURRENT_BEFORE_NEXT_VALUE)
+            .setSchemaName(sequenceKey.getSchemaName()).setTableName(sequenceKey.getSequenceName())
+            .build().buildException();
+        }
+        return value;
+    }
+
+    @Override
+    public void returnSequenceValues(List<SequenceKey> sequenceKeys, long timestamp, SQLException[] exceptions)
+            throws SQLException {
+    }
+
+    @Override
+    public void addConnection(PhoenixConnection connection) throws SQLException {
+    }
+
+    @Override
+    public void removeConnection(PhoenixConnection connection) throws SQLException {
+    }
+
+    @Override
+    public KeyValueBuilder getKeyValueBuilder() {
+        return this.kvBuilder;
+    }
+
+    @Override
+    public boolean supportsFeature(Feature feature) {
+        return false;
+    }
+}
\ No newline at end of file


[43/51] [partial] Initial commit of master branch from github

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateSequenceCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateSequenceCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateSequenceCompiler.java
new file mode 100644
index 0000000..1a22f11
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateSequenceCompiler.java
@@ -0,0 +1,213 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.compile;
+
+import java.sql.ParameterMetaData;
+import java.sql.SQLException;
+import java.util.Collections;
+
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+import org.apache.phoenix.execute.MutationState;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixStatement;
+import org.apache.phoenix.parse.BindParseNode;
+import org.apache.phoenix.parse.CreateSequenceStatement;
+import org.apache.phoenix.parse.ParseNode;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.MetaDataClient;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.PDatum;
+
+
+public class CreateSequenceCompiler {
+    private final PhoenixStatement statement;
+
+    public CreateSequenceCompiler(PhoenixStatement statement) {
+        this.statement = statement;
+    }
+    
+    private static class LongDatum implements PDatum {
+
+        @Override
+        public boolean isNullable() {
+            return false;
+        }
+
+        @Override
+        public PDataType getDataType() {
+            return PDataType.LONG;
+        }
+
+        @Override
+        public Integer getByteSize() {
+            return PDataType.LONG.getByteSize();
+        }
+
+        @Override
+        public Integer getMaxLength() {
+            return null;
+        }
+
+        @Override
+        public Integer getScale() {
+            return null;
+        }
+
+        @Override
+        public ColumnModifier getColumnModifier() {
+            return null;
+        }
+        
+    }
+    private static class IntegerDatum implements PDatum {
+
+        @Override
+        public boolean isNullable() {
+            return false;
+        }
+
+        @Override
+        public PDataType getDataType() {
+            return PDataType.INTEGER;
+        }
+
+        @Override
+        public Integer getByteSize() {
+            return PDataType.INTEGER.getByteSize();
+        }
+
+        @Override
+        public Integer getMaxLength() {
+            return null;
+        }
+
+        @Override
+        public Integer getScale() {
+            return null;
+        }
+
+        @Override
+        public ColumnModifier getColumnModifier() {
+            return null;
+        }
+        
+    }
+    private static final PDatum LONG_DATUM = new LongDatum();
+    private static final PDatum INTEGER_DATUM = new IntegerDatum();
+
+    public MutationPlan compile(final CreateSequenceStatement sequence) throws SQLException {
+        ParseNode startsWithNode = sequence.getStartWith();
+        ParseNode incrementByNode = sequence.getIncrementBy();
+        if (!startsWithNode.isStateless()) {
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode.STARTS_WITH_MUST_BE_CONSTANT)
+            .setSchemaName(sequence.getSequenceName().getSchemaName())
+            .setTableName(sequence.getSequenceName().getTableName()).build().buildException();
+        }
+        if (!incrementByNode.isStateless()) {
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode.INCREMENT_BY_MUST_BE_CONSTANT)
+            .setSchemaName(sequence.getSequenceName().getSchemaName())
+            .setTableName(sequence.getSequenceName().getTableName()).build().buildException();
+        }
+        ParseNode cacheNode = sequence.getCacheSize();
+        if (cacheNode != null && !cacheNode.isStateless()) {
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode.CACHE_MUST_BE_NON_NEGATIVE_CONSTANT)
+            .setSchemaName(sequence.getSequenceName().getSchemaName())
+            .setTableName(sequence.getSequenceName().getTableName()).build().buildException();
+        }
+        
+        final PhoenixConnection connection = statement.getConnection();
+        final ColumnResolver resolver = FromCompiler.EMPTY_TABLE_RESOLVER;
+        
+        final StatementContext context = new StatementContext(statement, resolver, statement.getParameters(), new Scan());
+        if (startsWithNode instanceof BindParseNode) {
+            context.getBindManager().addParamMetaData((BindParseNode)startsWithNode, LONG_DATUM);
+        }
+        if (incrementByNode instanceof BindParseNode) {
+            context.getBindManager().addParamMetaData((BindParseNode)incrementByNode, LONG_DATUM);
+        }
+        if (cacheNode instanceof BindParseNode) {
+            context.getBindManager().addParamMetaData((BindParseNode)cacheNode, INTEGER_DATUM);
+        }
+        ExpressionCompiler expressionCompiler = new ExpressionCompiler(context);
+        Expression startsWithExpr = startsWithNode.accept(expressionCompiler);
+        ImmutableBytesWritable ptr = context.getTempPtr();
+        startsWithExpr.evaluate(null, ptr);
+        if (ptr.getLength() == 0 || !startsWithExpr.getDataType().isCoercibleTo(PDataType.LONG)) {
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode.STARTS_WITH_MUST_BE_CONSTANT)
+            .setSchemaName(sequence.getSequenceName().getSchemaName())
+            .setTableName(sequence.getSequenceName().getTableName()).build().buildException();
+        }
+        final long startsWith = (Long)PDataType.LONG.toObject(ptr, startsWithExpr.getDataType());
+
+        Expression incrementByExpr = incrementByNode.accept(expressionCompiler);
+        incrementByExpr.evaluate(null, ptr);
+        if (ptr.getLength() == 0 || !incrementByExpr.getDataType().isCoercibleTo(PDataType.LONG)) {
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode.INCREMENT_BY_MUST_BE_CONSTANT)
+            .setSchemaName(sequence.getSequenceName().getSchemaName())
+            .setTableName(sequence.getSequenceName().getTableName()).build().buildException();
+        }
+        final long incrementBy = (Long)PDataType.LONG.toObject(ptr, incrementByExpr.getDataType());
+        
+        int cacheSizeValue = connection.getQueryServices().getProps().getInt(QueryServices.SEQUENCE_CACHE_SIZE_ATTRIB,QueryServicesOptions.DEFAULT_SEQUENCE_CACHE_SIZE);
+        if (cacheNode != null) {
+            Expression cacheSizeExpr = cacheNode.accept(expressionCompiler);
+            cacheSizeExpr.evaluate(null, ptr);
+            if (ptr.getLength() != 0 && (!cacheSizeExpr.getDataType().isCoercibleTo(PDataType.INTEGER) || (cacheSizeValue = (Integer)PDataType.INTEGER.toObject(ptr)) < 0)) {
+                throw new SQLExceptionInfo.Builder(SQLExceptionCode.CACHE_MUST_BE_NON_NEGATIVE_CONSTANT)
+                .setSchemaName(sequence.getSequenceName().getSchemaName())
+                .setTableName(sequence.getSequenceName().getTableName()).build().buildException();
+            }
+        }
+        final int cacheSize = Math.max(1, cacheSizeValue);
+        
+
+        final MetaDataClient client = new MetaDataClient(connection);        
+        return new MutationPlan() {           
+
+            @Override
+            public MutationState execute() throws SQLException {
+                return client.createSequence(sequence, startsWith, incrementBy, cacheSize);
+            }
+
+            @Override
+            public ExplainPlan getExplainPlan() throws SQLException {
+                return new ExplainPlan(Collections.singletonList("CREATE SEQUENCE"));
+            }
+
+            @Override
+            public PhoenixConnection getConnection() {
+                return connection;
+            }
+
+            @Override
+            public ParameterMetaData getParameterMetaData() {                
+                return context.getBindManager().getParameterMetaData();
+            }
+
+        };
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..12c60c8
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java
@@ -0,0 +1,245 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.compile;
+
+import java.sql.ParameterMetaData;
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+import org.apache.phoenix.execute.MutationState;
+import org.apache.phoenix.expression.AndExpression;
+import org.apache.phoenix.expression.ComparisonExpression;
+import org.apache.phoenix.expression.Expression;
+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.visitor.TraverseNoExpressionVisitor;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixStatement;
+import org.apache.phoenix.parse.CreateTableStatement;
+import org.apache.phoenix.parse.ParseNode;
+import org.apache.phoenix.parse.SQLParser;
+import org.apache.phoenix.parse.SelectStatement;
+import org.apache.phoenix.parse.TableName;
+import org.apache.phoenix.parse.WildcardParseNode;
+import org.apache.phoenix.query.DelegateConnectionQueryServices;
+import org.apache.phoenix.schema.MetaDataClient;
+import org.apache.phoenix.schema.PMetaData;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTable.ViewType;
+import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.util.ByteUtil;
+
+import com.google.common.collect.Iterators;
+
+
+public class CreateTableCompiler {
+    private static final String SELECT = "SELECT";
+    private static final String FROM = "FROM";
+    private static final String WHERE = "WHERE";
+    
+    private final PhoenixStatement statement;
+    
+    public CreateTableCompiler(PhoenixStatement statement) {
+        this.statement = statement;
+    }
+
+    public MutationPlan compile(final CreateTableStatement create) throws SQLException {
+        final PhoenixConnection connection = statement.getConnection();
+        ColumnResolver resolver = FromCompiler.getResolver(create, connection);
+        PTableType type = create.getTableType();
+        PhoenixConnection connectionToBe = connection;
+        PTable parentToBe = null;
+        ViewType viewTypeToBe = null;
+        Scan scan = new Scan();
+        final StatementContext context = new StatementContext(statement, resolver, statement.getParameters(), scan);
+        ExpressionCompiler expressionCompiler = new ExpressionCompiler(context);
+        // TODO: support any statement for a VIEW instead of just a WHERE clause
+        ParseNode whereNode = create.getWhereClause();
+        String viewStatementToBe = null;
+        if (type == PTableType.VIEW) {
+            TableRef tableRef = resolver.getTables().get(0);
+            parentToBe = tableRef.getTable();
+            viewTypeToBe = parentToBe.getViewType() == ViewType.MAPPED ? ViewType.MAPPED : ViewType.UPDATABLE;
+            if (whereNode == null) {
+                viewStatementToBe = parentToBe.getViewStatement();
+            } else {
+                whereNode = StatementNormalizer.normalize(whereNode, resolver);
+                if (whereNode.isStateless()) {
+                    throw new SQLExceptionInfo.Builder(SQLExceptionCode.VIEW_WHERE_IS_CONSTANT)
+                        .build().buildException();
+                }
+                // If our parent has a VIEW statement, combine it with this one
+                if (parentToBe.getViewStatement() != null) {
+                    SelectStatement select = new SQLParser(parentToBe.getViewStatement()).parseQuery().combine(whereNode);
+                    whereNode = select.getWhere();
+                }
+                Expression where = whereNode.accept(expressionCompiler);
+                if (where != null && !LiteralExpression.isTrue(where)) {
+                    TableName baseTableName = create.getBaseTableName();
+                    String schemaName = baseTableName.getSchemaName();
+                    // Only form we currently support for VIEWs: SELECT * FROM t WHERE ...
+                    viewStatementToBe = SELECT + " " + WildcardParseNode.NAME + " " + FROM +
+                            (schemaName == null ? "" : "\"" + schemaName + "\".") +
+                            (" \"" + baseTableName.getTableName() + "\" ") +
+                            (WHERE + " " + where.toString());
+                }
+                if (viewTypeToBe != ViewType.MAPPED) {
+                    Long scn = connection.getSCN();
+                    connectionToBe = scn != null ? connection :
+                        // If we haved no SCN on our connection, freeze the SCN at when
+                        // the base table was resolved to prevent any race condition on
+                        // the error checking we do for the base table. The only potential
+                        // issue is if the base table lives on a different region server
+                        // than the new table will, then we're relying here on the system
+                        // clocks being in sync.
+                        new PhoenixConnection(
+                            // When the new table is created, we still want to cache it
+                            // on our connection.
+                            new DelegateConnectionQueryServices(connection.getQueryServices()) {
+                                @Override
+                                public PMetaData addTable(PTable table) throws SQLException {
+                                    return connection.addTable(table);
+                                }
+                            },
+                            connection, tableRef.getTimeStamp());
+                    ViewWhereExpressionVisitor visitor = new ViewWhereExpressionVisitor();
+                    where.accept(visitor);
+                    viewTypeToBe = visitor.isUpdatable() ? ViewType.UPDATABLE : ViewType.READ_ONLY;
+                }
+            }
+        }
+        final ViewType viewType = viewTypeToBe;
+        final String viewStatement = viewStatementToBe;
+        List<ParseNode> splitNodes = create.getSplitNodes();
+        final byte[][] splits = new byte[splitNodes.size()][];
+        ImmutableBytesWritable ptr = context.getTempPtr();
+        for (int i = 0; i < splits.length; i++) {
+            ParseNode node = splitNodes.get(i);
+            if (node.isStateless()) {
+                Expression expression = node.accept(expressionCompiler);
+                if (expression.evaluate(null, ptr)) {;
+                    splits[i] = ByteUtil.copyKeyBytesIfNecessary(ptr);
+                    continue;
+                }
+            }
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode.SPLIT_POINT_NOT_CONSTANT)
+                .setMessage("Node: " + node).build().buildException();
+        }
+        final MetaDataClient client = new MetaDataClient(connectionToBe);
+        final PTable parent = parentToBe;
+        
+        return new MutationPlan() {
+
+            @Override
+            public ParameterMetaData getParameterMetaData() {
+                return context.getBindManager().getParameterMetaData();
+            }
+
+            @Override
+            public MutationState execute() throws SQLException {
+                try {
+                    return client.createTable(create, splits, parent, viewStatement, viewType);
+                } finally {
+                    if (client.getConnection() != connection) {
+                        client.getConnection().close();
+                    }
+                }
+            }
+
+            @Override
+            public ExplainPlan getExplainPlan() throws SQLException {
+                return new ExplainPlan(Collections.singletonList("CREATE TABLE"));
+            }
+
+            @Override
+            public PhoenixConnection getConnection() {
+                return connection;
+            }
+            
+        };
+    }
+    
+    private static class ViewWhereExpressionVisitor extends TraverseNoExpressionVisitor<Boolean> {
+        private boolean isUpdatable = true;
+
+        public boolean isUpdatable() {
+            return isUpdatable;
+        }
+
+        @Override
+        public Boolean defaultReturn(Expression node, List<Boolean> l) {
+            // We only hit this if we're trying to traverse somewhere
+            // in which we don't have a visitLeave that returns non null
+            isUpdatable = false;
+            return null;
+        }
+
+        @Override
+        public Iterator<Expression> visitEnter(AndExpression node) {
+            return node.getChildren().iterator();
+        }
+
+        @Override
+        public Boolean visitLeave(AndExpression node, List<Boolean> l) {
+            return l.isEmpty() ? null : Boolean.TRUE;
+        }
+
+        @Override
+        public Iterator<Expression> visitEnter(ComparisonExpression node) {
+            return node.getFilterOp() == CompareOp.EQUAL && node.getChildren().get(1).isStateless() && node.getChildren().get(1).isDeterministic() ? Iterators.singletonIterator(node.getChildren().get(0)) : super.visitEnter(node);
+        }
+
+        @Override
+        public Boolean visitLeave(ComparisonExpression node, List<Boolean> l) {
+            return l.isEmpty() ? null : Boolean.TRUE;
+        }
+
+        @Override
+        public Iterator<Expression> visitEnter(IsNullExpression node) {
+            return node.isNegate() ? super.visitEnter(node) : node.getChildren().iterator();
+        }
+        
+        @Override
+        public Boolean visitLeave(IsNullExpression node, List<Boolean> l) {
+            return l.isEmpty() ? null : Boolean.TRUE;
+        }
+        
+        @Override
+        public Boolean visit(RowKeyColumnExpression node) {
+            return Boolean.TRUE;
+        }
+
+        @Override
+        public Boolean visit(KeyValueColumnExpression node) {
+            return Boolean.TRUE;
+        }
+        
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..a73c111
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
@@ -0,0 +1,360 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.compile;
+
+import java.sql.ParameterMetaData;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.phoenix.cache.ServerCacheClient.ServerCache;
+import org.apache.phoenix.compile.GroupByCompiler.GroupBy;
+import org.apache.phoenix.compile.OrderByCompiler.OrderBy;
+import org.apache.phoenix.coprocessor.UngroupedAggregateRegionObserver;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+import org.apache.phoenix.execute.AggregatePlan;
+import org.apache.phoenix.execute.MutationState;
+import org.apache.phoenix.index.IndexMetaDataCacheClient;
+import org.apache.phoenix.index.PhoenixIndexCodec;
+import org.apache.phoenix.iterate.ResultIterator;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixResultSet;
+import org.apache.phoenix.jdbc.PhoenixStatement;
+import org.apache.phoenix.optimize.QueryOptimizer;
+import org.apache.phoenix.parse.AliasedNode;
+import org.apache.phoenix.parse.DeleteStatement;
+import org.apache.phoenix.parse.HintNode;
+import org.apache.phoenix.parse.HintNode.Hint;
+import org.apache.phoenix.parse.ParseNode;
+import org.apache.phoenix.parse.ParseNodeFactory;
+import org.apache.phoenix.parse.SelectStatement;
+import org.apache.phoenix.query.ConnectionQueryServices;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.PRow;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.schema.ReadOnlyTableException;
+import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.IndexUtil;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+public class DeleteCompiler {
+    private static ParseNodeFactory FACTORY = new ParseNodeFactory();
+    
+    private final PhoenixStatement statement;
+    
+    public DeleteCompiler(PhoenixStatement statement) {
+        this.statement = statement;
+    }
+    
+    private static MutationState deleteRows(PhoenixStatement statement, TableRef tableRef, ResultIterator iterator, RowProjector projector) throws SQLException {
+        PhoenixConnection connection = statement.getConnection();
+        byte[] tenantId = connection.getTenantId() == null ? null : connection.getTenantId().getBytes();
+        final boolean isAutoCommit = connection.getAutoCommit();
+        ConnectionQueryServices services = connection.getQueryServices();
+        final int maxSize = services.getProps().getInt(QueryServices.MAX_MUTATION_SIZE_ATTRIB,QueryServicesOptions.DEFAULT_MAX_MUTATION_SIZE);
+        final int batchSize = Math.min(connection.getMutateBatchSize(), maxSize);
+        Map<ImmutableBytesPtr,Map<PColumn,byte[]>> mutations = Maps.newHashMapWithExpectedSize(batchSize);
+        try {
+            PTable table = tableRef.getTable();
+            List<PColumn> pkColumns = table.getPKColumns();
+            boolean isMultiTenant = table.isMultiTenant() && tenantId != null;
+            int offset = (table.getBucketNum() == null ? 0 : 1) + (isMultiTenant ? 1 : 0); // Take into account salting and multi-tenant
+            byte[][] values = new byte[pkColumns.size()][];
+            if (isMultiTenant) {
+                values[offset-1] = tenantId;
+            }
+            ResultSet rs = new PhoenixResultSet(iterator, projector, statement);
+            int rowCount = 0;
+            while (rs.next()) {
+                for (int i = offset; i < values.length; i++) {
+                    byte[] byteValue = rs.getBytes(i+1-offset);
+                    // The ResultSet.getBytes() call will have inverted it - we need to invert it back.
+                    // TODO: consider going under the hood and just getting the bytes
+                    if (pkColumns.get(i).getColumnModifier() == ColumnModifier.SORT_DESC) {
+                        byte[] tempByteValue = Arrays.copyOf(byteValue, byteValue.length);
+                        byteValue = ColumnModifier.SORT_DESC.apply(byteValue, 0, tempByteValue, 0, byteValue.length);
+                    }
+                    values[i] = byteValue;
+                }
+                ImmutableBytesPtr ptr = new ImmutableBytesPtr();
+                table.newKey(ptr, values);
+                mutations.put(ptr, PRow.DELETE_MARKER);
+                if (mutations.size() > maxSize) {
+                    throw new IllegalArgumentException("MutationState size of " + mutations.size() + " is bigger than max allowed size of " + maxSize);
+                }
+                rowCount++;
+                // Commit a batch if auto commit is true and we're at our batch size
+                if (isAutoCommit && rowCount % batchSize == 0) {
+                    MutationState state = new MutationState(tableRef, mutations, 0, maxSize, connection);
+                    connection.getMutationState().join(state);
+                    connection.commit();
+                    mutations.clear();
+                }
+            }
+
+            // If auto commit is true, this last batch will be committed upon return
+            return new MutationState(tableRef, mutations, rowCount / batchSize * batchSize, maxSize, connection);
+        } finally {
+            iterator.close();
+        }
+    }
+    
+    private static class DeletingParallelIteratorFactory extends MutatingParallelIteratorFactory {
+        private RowProjector projector;
+        
+        private DeletingParallelIteratorFactory(PhoenixConnection connection, TableRef tableRef) {
+            super(connection, tableRef);
+        }
+        
+        @Override
+        protected MutationState mutate(PhoenixConnection connection, ResultIterator iterator) throws SQLException {
+            PhoenixStatement statement = new PhoenixStatement(connection);
+            return deleteRows(statement, tableRef, iterator, projector);
+        }
+        
+        public void setRowProjector(RowProjector projector) {
+            this.projector = projector;
+        }
+        
+    }
+    
+    private boolean hasImmutableIndex(TableRef tableRef) {
+        return tableRef.getTable().isImmutableRows() && !tableRef.getTable().getIndexes().isEmpty();
+    }
+    
+    private boolean hasImmutableIndexWithKeyValueColumns(TableRef tableRef) {
+        if (!hasImmutableIndex(tableRef)) {
+            return false;
+        }
+        for (PTable index : tableRef.getTable().getIndexes()) {
+            for (PColumn column : index.getPKColumns()) {
+                if (!IndexUtil.isDataPKColumn(column)) {
+                    return true;
+                }
+            }
+        }
+        return false;
+    }
+    
+    public MutationPlan compile(DeleteStatement delete) throws SQLException {
+        final PhoenixConnection connection = statement.getConnection();
+        final boolean isAutoCommit = connection.getAutoCommit();
+        final ConnectionQueryServices services = connection.getQueryServices();
+        final ColumnResolver resolver = FromCompiler.getResolver(delete, connection);
+        final TableRef tableRef = resolver.getTables().get(0);
+        PTable table = tableRef.getTable();
+        if (table.getType() == PTableType.VIEW && table.getViewType().isReadOnly()) {
+            throw new ReadOnlyTableException(table.getSchemaName().getString(),table.getTableName().getString());
+        }
+        
+        final boolean hasLimit = delete.getLimit() != null;
+        boolean noQueryReqd = !hasLimit && !hasImmutableIndex(tableRef);
+        boolean runOnServer = isAutoCommit && noQueryReqd;
+        HintNode hint = delete.getHint();
+        if (runOnServer && !delete.getHint().hasHint(Hint.USE_INDEX_OVER_DATA_TABLE)) {
+            hint = HintNode.create(hint, Hint.USE_DATA_OVER_INDEX_TABLE);
+        }
+
+        List<AliasedNode> aliasedNodes = Lists.newArrayListWithExpectedSize(table.getPKColumns().size());
+        boolean isSalted = table.getBucketNum() != null;
+        boolean isMultiTenant = connection.getTenantId() != null && table.isMultiTenant();
+        for (int i = (isSalted ? 1 : 0) + (isMultiTenant ? 1 : 0); i < table.getPKColumns().size(); i++) {
+            PColumn column = table.getPKColumns().get(i);
+            aliasedNodes.add(FACTORY.aliasedNode(null, FACTORY.column(null, '"' + column.getName().getString() + '"', null)));
+        }
+        SelectStatement select = FACTORY.select(
+                Collections.singletonList(delete.getTable()), 
+                hint, false, aliasedNodes, delete.getWhere(), 
+                Collections.<ParseNode>emptyList(), null, 
+                delete.getOrderBy(), delete.getLimit(),
+                delete.getBindCount(), false);
+        DeletingParallelIteratorFactory parallelIteratorFactory = hasLimit ? null : new DeletingParallelIteratorFactory(connection, tableRef);
+        final QueryPlan plan = new QueryOptimizer(services).optimize(select, statement, Collections.<PColumn>emptyList(), parallelIteratorFactory);
+        if (!plan.getTableRef().equals(tableRef)) {
+            runOnServer = false;
+            noQueryReqd = false;
+        }
+        
+        final int maxSize = services.getProps().getInt(QueryServices.MAX_MUTATION_SIZE_ATTRIB,QueryServicesOptions.DEFAULT_MAX_MUTATION_SIZE);
+ 
+        if (hasImmutableIndexWithKeyValueColumns(tableRef)) {
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode.NO_DELETE_IF_IMMUTABLE_INDEX).setSchemaName(tableRef.getTable().getSchemaName().getString())
+            .setTableName(tableRef.getTable().getTableName().getString()).build().buildException();
+        }
+        
+        final StatementContext context = plan.getContext();
+        // If we're doing a query for a single row with no where clause, then we don't need to contact the server at all.
+        // A simple check of the none existence of a where clause in the parse node is not sufficient, as the where clause
+        // may have been optimized out.
+        if (noQueryReqd && context.isSingleRowScan()) {
+            final ImmutableBytesPtr key = new ImmutableBytesPtr(context.getScan().getStartRow());
+            return new MutationPlan() {
+
+                @Override
+                public ParameterMetaData getParameterMetaData() {
+                    return context.getBindManager().getParameterMetaData();
+                }
+
+                @Override
+                public MutationState execute() {
+                    Map<ImmutableBytesPtr,Map<PColumn,byte[]>> mutation = Maps.newHashMapWithExpectedSize(1);
+                    mutation.put(key, PRow.DELETE_MARKER);
+                    return new MutationState(tableRef, mutation, 0, maxSize, connection);
+                }
+
+                @Override
+                public ExplainPlan getExplainPlan() throws SQLException {
+                    return new ExplainPlan(Collections.singletonList("DELETE SINGLE ROW"));
+                }
+
+                @Override
+                public PhoenixConnection getConnection() {
+                    return connection;
+                }
+            };
+        } else if (runOnServer) {
+            // TODO: better abstraction
+            Scan scan = context.getScan();
+            scan.setAttribute(UngroupedAggregateRegionObserver.DELETE_AGG, QueryConstants.TRUE);
+
+            // Build an ungrouped aggregate query: select COUNT(*) from <table> where <where>
+            // The coprocessor will delete each row returned from the scan
+            // Ignoring ORDER BY, since with auto commit on and no limit makes no difference
+            SelectStatement aggSelect = SelectStatement.create(SelectStatement.COUNT_ONE, delete.getHint());
+            final RowProjector projector = ProjectionCompiler.compile(context, aggSelect, GroupBy.EMPTY_GROUP_BY);
+            final QueryPlan aggPlan = new AggregatePlan(context, select, tableRef, projector, null, OrderBy.EMPTY_ORDER_BY, null, GroupBy.EMPTY_GROUP_BY, null);
+            return new MutationPlan() {
+
+                @Override
+                public PhoenixConnection getConnection() {
+                    return connection;
+                }
+
+                @Override
+                public ParameterMetaData getParameterMetaData() {
+                    return context.getBindManager().getParameterMetaData();
+                }
+
+                @Override
+                public MutationState execute() throws SQLException {
+                    // TODO: share this block of code with UPSERT SELECT
+                    ImmutableBytesWritable ptr = context.getTempPtr();
+                    tableRef.getTable().getIndexMaintainers(ptr);
+                    ServerCache cache = null;
+                    try {
+                        if (ptr.getLength() > 0) {
+                            IndexMetaDataCacheClient client = new IndexMetaDataCacheClient(connection, tableRef);
+                            cache = client.addIndexMetadataCache(context.getScanRanges(), ptr);
+                            byte[] uuidValue = cache.getId();
+                            context.getScan().setAttribute(PhoenixIndexCodec.INDEX_UUID, uuidValue);
+                        }
+                        ResultIterator iterator = aggPlan.iterator();
+                        try {
+                            Tuple row = iterator.next();
+                            final long mutationCount = (Long)projector.getColumnProjector(0).getValue(row, PDataType.LONG, ptr);
+                            return new MutationState(maxSize, connection) {
+                                @Override
+                                public long getUpdateCount() {
+                                    return mutationCount;
+                                }
+                            };
+                        } finally {
+                            iterator.close();
+                        }
+                    } finally {
+                        if (cache != null) {
+                            cache.close();
+                        }
+                    }
+                }
+
+                @Override
+                public ExplainPlan getExplainPlan() throws SQLException {
+                    List<String> queryPlanSteps =  aggPlan.getExplainPlan().getPlanSteps();
+                    List<String> planSteps = Lists.newArrayListWithExpectedSize(queryPlanSteps.size()+1);
+                    planSteps.add("DELETE ROWS");
+                    planSteps.addAll(queryPlanSteps);
+                    return new ExplainPlan(planSteps);
+                }
+            };
+        } else {
+            if (parallelIteratorFactory != null) {
+                parallelIteratorFactory.setRowProjector(plan.getProjector());
+            }
+            return new MutationPlan() {
+
+                @Override
+                public PhoenixConnection getConnection() {
+                    return connection;
+                }
+
+                @Override
+                public ParameterMetaData getParameterMetaData() {
+                    return context.getBindManager().getParameterMetaData();
+                }
+
+                @Override
+                public MutationState execute() throws SQLException {
+                    ResultIterator iterator = plan.iterator();
+                    if (!hasLimit) {
+                        Tuple tuple;
+                        long totalRowCount = 0;
+                        while ((tuple=iterator.next()) != null) {// Runs query
+                            KeyValue kv = tuple.getValue(0);
+                            totalRowCount += PDataType.LONG.getCodec().decodeLong(kv.getBuffer(), kv.getValueOffset(), null);
+                        }
+                        // Return total number of rows that have been delete. In the case of auto commit being off
+                        // the mutations will all be in the mutation state of the current connection.
+                        return new MutationState(maxSize, connection, totalRowCount);
+                    } else {
+                        return deleteRows(statement, tableRef, iterator, plan.getProjector());
+                    }
+                }
+
+                @Override
+                public ExplainPlan getExplainPlan() throws SQLException {
+                    List<String> queryPlanSteps =  plan.getExplainPlan().getPlanSteps();
+                    List<String> planSteps = Lists.newArrayListWithExpectedSize(queryPlanSteps.size()+1);
+                    planSteps.add("DELETE ROWS");
+                    planSteps.addAll(queryPlanSteps);
+                    return new ExplainPlan(planSteps);
+                }
+            };
+        }
+       
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/compile/DropSequenceCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/DropSequenceCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/DropSequenceCompiler.java
new file mode 100644
index 0000000..be53f6b
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/DropSequenceCompiler.java
@@ -0,0 +1,69 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.compile;
+
+import java.sql.ParameterMetaData;
+import java.sql.SQLException;
+import java.util.Collections;
+
+import org.apache.phoenix.execute.MutationState;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixParameterMetaData;
+import org.apache.phoenix.jdbc.PhoenixStatement;
+import org.apache.phoenix.parse.DropSequenceStatement;
+import org.apache.phoenix.schema.MetaDataClient;
+
+
+public class DropSequenceCompiler {
+    private final PhoenixStatement statement;
+
+    public DropSequenceCompiler(PhoenixStatement statement) {
+        this.statement = statement;
+    }
+    
+
+    public MutationPlan compile(final DropSequenceStatement sequence) throws SQLException {
+        final PhoenixConnection connection = statement.getConnection();
+        final MetaDataClient client = new MetaDataClient(connection);        
+        return new MutationPlan() {           
+
+            @Override
+            public MutationState execute() throws SQLException {
+                return client.dropSequence(sequence);
+            }
+
+            @Override
+            public ExplainPlan getExplainPlan() throws SQLException {
+                return new ExplainPlan(Collections.singletonList("DROP SEQUENCE"));
+            }
+
+            @Override
+            public PhoenixConnection getConnection() {
+                return connection;
+            }
+
+            @Override
+            public ParameterMetaData getParameterMetaData() {                
+                return PhoenixParameterMetaData.EMPTY_PARAMETER_META_DATA;
+            }
+
+        };
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/compile/ExplainPlan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/ExplainPlan.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/ExplainPlan.java
new file mode 100644
index 0000000..e1049a0
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/ExplainPlan.java
@@ -0,0 +1,39 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.compile;
+
+import java.util.Collections;
+import java.util.List;
+
+import com.google.common.collect.ImmutableList;
+
+public class ExplainPlan {
+    public static final ExplainPlan EMPTY_PLAN = new ExplainPlan(Collections.<String>emptyList());
+
+    private final List<String> planSteps;
+    
+    public ExplainPlan(List<String> planSteps) {
+        this.planSteps = ImmutableList.copyOf(planSteps);
+    }
+    
+    public List<String> getPlanSteps() {
+        return planSteps;
+    }
+}


[23/51] [partial] Initial commit of master branch from github

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
new file mode 100644
index 0000000..ac92570
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
@@ -0,0 +1,671 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.jdbc;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.io.Reader;
+import java.sql.Array;
+import java.sql.Blob;
+import java.sql.CallableStatement;
+import java.sql.Clob;
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.NClob;
+import java.sql.ParameterMetaData;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLClientInfoException;
+import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
+import java.sql.SQLWarning;
+import java.sql.SQLXML;
+import java.sql.Savepoint;
+import java.sql.Statement;
+import java.sql.Struct;
+import java.text.Format;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.concurrent.Executor;
+
+import javax.annotation.Nullable;
+
+import com.google.common.base.Strings;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import org.apache.phoenix.client.KeyValueBuilder;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+import org.apache.phoenix.execute.MutationState;
+import org.apache.phoenix.jdbc.PhoenixStatement.PhoenixStatementParser;
+import org.apache.phoenix.query.ConnectionQueryServices;
+import org.apache.phoenix.query.DelegateConnectionQueryServices;
+import org.apache.phoenix.query.MetaDataMutated;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.PArrayDataType;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.PMetaData;
+import org.apache.phoenix.schema.PMetaDataImpl;
+import org.apache.phoenix.schema.PName;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.util.DateUtil;
+import org.apache.phoenix.util.JDBCUtil;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.SQLCloseable;
+import org.apache.phoenix.util.SQLCloseables;
+
+
+/**
+ * 
+ * JDBC Connection implementation of Phoenix.
+ * Currently the following are supported:
+ * - Statement
+ * - PreparedStatement
+ * The connection may only be used with the following options:
+ * - ResultSet.TYPE_FORWARD_ONLY
+ * - Connection.TRANSACTION_READ_COMMITTED
+ * 
+ * @author jtaylor
+ * @since 0.1
+ */
+public class PhoenixConnection implements Connection, org.apache.phoenix.jdbc.Jdbc7Shim.Connection, MetaDataMutated  {
+    private final String url;
+    private final ConnectionQueryServices services;
+    private final Properties info;
+    private List<SQLCloseable> statements = new ArrayList<SQLCloseable>();
+    private final Format[] formatters = new Format[PDataType.values().length];
+    private final MutationState mutationState;
+    private final int mutateBatchSize;
+    private final Long scn;
+    private boolean isAutoCommit = false;
+    private PMetaData metaData;
+    private final PName tenantId;
+    private final String datePattern;
+    
+    private boolean isClosed = false;
+    
+    private static Properties newPropsWithSCN(long scn, Properties props) {
+        props = new Properties(props);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(scn));
+        return props;
+    }
+    
+    public PhoenixConnection(PhoenixConnection connection) throws SQLException {
+        this(connection.getQueryServices(), connection.getURL(), connection.getClientInfo(), connection.getPMetaData());
+        this.isAutoCommit = connection.isAutoCommit;
+    }
+    
+    public PhoenixConnection(PhoenixConnection connection, long scn) throws SQLException {
+        this(connection.getQueryServices(), connection, scn);
+    }
+    
+    public PhoenixConnection(ConnectionQueryServices services, PhoenixConnection connection, long scn) throws SQLException {
+        this(services, connection.getURL(), newPropsWithSCN(scn,connection.getClientInfo()), PMetaDataImpl.pruneNewerTables(scn, connection.getPMetaData()));
+        this.isAutoCommit = connection.isAutoCommit;
+    }
+    
+    @SuppressWarnings("unchecked")
+    public PhoenixConnection(ConnectionQueryServices services, String url, Properties info, PMetaData metaData) throws SQLException {
+        this.url = url;
+        // Copy so client cannot change
+        this.info = info == null ? new Properties() : new Properties(info);
+        if (this.info.isEmpty()) {
+            this.services = services;
+        } else {
+            Map<String, String> existingProps = services.getProps().asMap();
+            Map<String, String> tmpAugmentedProps = Maps.newHashMapWithExpectedSize(existingProps.size() + info.size());
+            tmpAugmentedProps.putAll(existingProps);
+            tmpAugmentedProps.putAll((Map)this.info);
+            final ReadOnlyProps augmentedProps = new ReadOnlyProps(tmpAugmentedProps);
+            this.services = new DelegateConnectionQueryServices(services) {
+    
+                @Override
+                public ReadOnlyProps getProps() {
+                    return augmentedProps;
+                }
+            };
+        }
+        this.scn = JDBCUtil.getCurrentSCN(url, this.info);
+        this.tenantId = JDBCUtil.getTenantId(url, this.info);
+        this.mutateBatchSize = JDBCUtil.getMutateBatchSize(url, this.info, services.getProps());
+        datePattern = services.getProps().get(QueryServices.DATE_FORMAT_ATTRIB, DateUtil.DEFAULT_DATE_FORMAT);
+        int maxSize = services.getProps().getInt(QueryServices.MAX_MUTATION_SIZE_ATTRIB,QueryServicesOptions.DEFAULT_MAX_MUTATION_SIZE);
+        Format dateTimeFormat = DateUtil.getDateFormatter(datePattern);
+        formatters[PDataType.DATE.ordinal()] = dateTimeFormat;
+        formatters[PDataType.TIME.ordinal()] = dateTimeFormat;
+        this.metaData = PMetaDataImpl.pruneMultiTenant(metaData);
+        this.mutationState = new MutationState(maxSize, this);
+        services.addConnection(this);
+    }
+
+    public int executeStatements(Reader reader, List<Object> binds, PrintStream out) throws IOException, SQLException {
+        int bindsOffset = 0;
+        int nStatements = 0;
+        PhoenixStatementParser parser = new PhoenixStatementParser(reader);
+        try {
+            while (true) {
+                PhoenixPreparedStatement stmt = new PhoenixPreparedStatement(this, parser);
+                ParameterMetaData paramMetaData = stmt.getParameterMetaData();
+                for (int i = 0; i < paramMetaData.getParameterCount(); i++) {
+                    stmt.setObject(i+1, binds.get(bindsOffset+i));
+                }
+                long start = System.currentTimeMillis();
+                boolean isQuery = stmt.execute();
+                if (isQuery) {
+                    ResultSet rs = stmt.getResultSet();
+                    if (!rs.next()) {
+                        if (out != null) {
+                            out.println("no rows selected");
+                        }
+                    } else {
+                        int columnCount = 0;
+                        if (out != null) {
+                            ResultSetMetaData md = rs.getMetaData();
+                            columnCount = md.getColumnCount();
+                            for (int i = 1; i <= columnCount; i++) {
+                                int displayWidth = md.getColumnDisplaySize(i);
+                                String label = md.getColumnLabel(i);
+                                if (md.isSigned(i)) {
+                                    out.print(displayWidth < label.length() ? label.substring(0,displayWidth) : Strings.padStart(label, displayWidth, ' '));
+                                    out.print(' ');
+                                } else {
+                                    out.print(displayWidth < label.length() ? label.substring(0,displayWidth) : Strings.padEnd(md.getColumnLabel(i), displayWidth, ' '));
+                                    out.print(' ');
+                                }
+                            }
+                            out.println();
+                            for (int i = 1; i <= columnCount; i++) {
+                                int displayWidth = md.getColumnDisplaySize(i);
+                                out.print(Strings.padStart("", displayWidth,'-'));
+                                out.print(' ');
+                            }
+                            out.println();
+                        }
+                        do {
+                            if (out != null) {
+                                ResultSetMetaData md = rs.getMetaData();
+                                for (int i = 1; i <= columnCount; i++) {
+                                    int displayWidth = md.getColumnDisplaySize(i);
+                                    String value = rs.getString(i);
+                                    String valueString = value == null ? QueryConstants.NULL_DISPLAY_TEXT : value;
+                                    if (md.isSigned(i)) {
+                                        out.print(Strings.padStart(valueString, displayWidth, ' '));
+                                    } else {
+                                        out.print(Strings.padEnd(valueString, displayWidth, ' '));
+                                    }
+                                    out.print(' ');
+                                }
+                                out.println();
+                            }
+                        } while (rs.next());
+                    }
+                } else if (out != null){
+                    int updateCount = stmt.getUpdateCount();
+                    if (updateCount >= 0) {
+                        out.println((updateCount == 0 ? "no" : updateCount) + (updateCount == 1 ? " row " : " rows ") + stmt.getUpdateOperation().toString());
+                    }
+                }
+                bindsOffset += paramMetaData.getParameterCount();
+                double elapsedDuration = ((System.currentTimeMillis() - start) / 1000.0);
+                out.println("Time: " + elapsedDuration + " sec(s)\n");
+                nStatements++;
+            }
+        } catch (EOFException e) {
+        }
+        return nStatements;
+    }
+
+    public @Nullable PName getTenantId() {
+        return tenantId;
+    }
+    
+    public Long getSCN() {
+        return scn;
+    }
+    
+    public int getMutateBatchSize() {
+        return mutateBatchSize;
+    }
+    
+    public PMetaData getPMetaData() {
+        return metaData;
+    }
+
+    public MutationState getMutationState() {
+        return mutationState;
+    }
+    
+    public String getDatePattern() {
+        return datePattern;
+    }
+    
+    public Format getFormatter(PDataType type) {
+        return formatters[type.ordinal()];
+    }
+    
+    public String getURL() {
+        return url;
+    }
+    
+    public ConnectionQueryServices getQueryServices() {
+        return services;
+    }
+    
+    @Override
+    public void clearWarnings() throws SQLException {
+    }
+
+    private void closeStatements() throws SQLException {
+        List<SQLCloseable> statements = this.statements;
+        // create new list to prevent close of statements
+        // from modifying this list.
+        this.statements = Lists.newArrayList();
+        try {
+            mutationState.rollback(this);
+        } finally {
+            try {
+                SQLCloseables.closeAll(statements);
+            } finally {
+                statements.clear();
+            }
+        }
+    }
+    
+    @Override
+    public void close() throws SQLException {
+        if (isClosed) {
+            return;
+        }
+        try {
+            try {
+                closeStatements();
+            } finally {
+                services.removeConnection(this);
+            }
+        } finally {
+            isClosed = true;
+        }
+    }
+
+    @Override
+    public void commit() throws SQLException {
+        mutationState.commit();
+    }
+
+    @Override
+    public Array createArrayOf(String typeName, Object[] elements) throws SQLException {
+    	PDataType arrayPrimitiveType = PDataType.fromSqlTypeName(typeName);
+    	return PArrayDataType.instantiatePhoenixArray(arrayPrimitiveType, elements);
+    }
+
+    @Override
+    public Blob createBlob() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public Clob createClob() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public NClob createNClob() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public SQLXML createSQLXML() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public Statement createStatement() throws SQLException {
+        PhoenixStatement statement = new PhoenixStatement(this);
+        statements.add(statement);
+        return statement;
+    }
+
+    /**
+     * Back-door way to inject processing into walking through a result set
+     * @param statementFactory
+     * @return PhoenixStatement
+     * @throws SQLException
+     */
+    public PhoenixStatement createStatement(PhoenixStatementFactory statementFactory) throws SQLException {
+        PhoenixStatement statement = statementFactory.newStatement(this);
+        statements.add(statement);
+        return statement;
+    }
+
+    @Override
+    public Statement createStatement(int resultSetType, int resultSetConcurrency) throws SQLException {
+        if (resultSetType != ResultSet.TYPE_FORWARD_ONLY || resultSetConcurrency != ResultSet.CONCUR_READ_ONLY) {
+            throw new SQLFeatureNotSupportedException();
+        }
+        return createStatement();
+    }
+
+    @Override
+    public Statement createStatement(int resultSetType, int resultSetConcurrency, int resultSetHoldability)
+            throws SQLException {
+        if (resultSetHoldability != ResultSet.CLOSE_CURSORS_AT_COMMIT) {
+            throw new SQLFeatureNotSupportedException();
+        }
+        return createStatement(resultSetType, resultSetConcurrency);
+    }
+
+    @Override
+    public Struct createStruct(String typeName, Object[] attributes) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public boolean getAutoCommit() throws SQLException {
+        return isAutoCommit;
+    }
+
+    @Override
+    public String getCatalog() throws SQLException {
+        return "";
+    }
+
+    @Override
+    public Properties getClientInfo() throws SQLException { 
+        // Defensive copy so client cannot change
+        return new Properties(info);
+    }
+
+    @Override
+    public String getClientInfo(String name) {
+        return info.getProperty(name);
+    }
+
+    @Override
+    public int getHoldability() throws SQLException {
+        return ResultSet.CLOSE_CURSORS_AT_COMMIT;
+    }
+
+    @Override
+    public DatabaseMetaData getMetaData() throws SQLException {
+        return new PhoenixDatabaseMetaData(this);
+    }
+
+    @Override
+    public int getTransactionIsolation() throws SQLException {
+        return Connection.TRANSACTION_READ_COMMITTED;
+    }
+
+    @Override
+    public Map<String, Class<?>> getTypeMap() throws SQLException {
+        return Collections.emptyMap();
+    }
+
+    @Override
+    public SQLWarning getWarnings() throws SQLException {
+        return null;
+    }
+
+    @Override
+    public boolean isClosed() throws SQLException {
+        return isClosed;
+    }
+
+    @Override
+    public boolean isReadOnly() throws SQLException {
+        return true;
+    }
+
+    @Override
+    public boolean isValid(int timeout) throws SQLException {
+        // TODO: run query here or ping
+        return !isClosed;
+    }
+
+    @Override
+    public String nativeSQL(String sql) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public CallableStatement prepareCall(String sql) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public CallableStatement prepareCall(String sql, int resultSetType, int resultSetConcurrency) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public CallableStatement prepareCall(String sql, int resultSetType, int resultSetConcurrency,
+            int resultSetHoldability) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public PreparedStatement prepareStatement(String sql) throws SQLException {
+        PhoenixPreparedStatement statement = new PhoenixPreparedStatement(this, sql);
+        statements.add(statement);
+        return statement;
+    }
+
+    @Override
+    public PreparedStatement prepareStatement(String sql, int autoGeneratedKeys) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public PreparedStatement prepareStatement(String sql, int[] columnIndexes) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public PreparedStatement prepareStatement(String sql, String[] columnNames) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public PreparedStatement prepareStatement(String sql, int resultSetType, int resultSetConcurrency)
+            throws SQLException {
+        if (resultSetType != ResultSet.TYPE_FORWARD_ONLY || resultSetConcurrency != ResultSet.CONCUR_READ_ONLY) {
+            throw new SQLFeatureNotSupportedException();
+        }
+        return prepareStatement(sql);
+    }
+
+    @Override
+    public PreparedStatement prepareStatement(String sql, int resultSetType, int resultSetConcurrency,
+            int resultSetHoldability) throws SQLException {
+        if (resultSetHoldability != ResultSet.CLOSE_CURSORS_AT_COMMIT) {
+            throw new SQLFeatureNotSupportedException();
+        }
+        return prepareStatement(sql, resultSetType, resultSetConcurrency);
+    }
+
+    @Override
+    public void releaseSavepoint(Savepoint savepoint) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void rollback() throws SQLException {
+        mutationState.rollback(this);
+    }
+
+    @Override
+    public void rollback(Savepoint savepoint) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void setAutoCommit(boolean isAutoCommit) throws SQLException {
+        this.isAutoCommit = isAutoCommit;
+    }
+
+    @Override
+    public void setCatalog(String catalog) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void setClientInfo(Properties properties) throws SQLClientInfoException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void setClientInfo(String name, String value) throws SQLClientInfoException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void setHoldability(int holdability) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void setReadOnly(boolean readOnly) throws SQLException {
+        if (readOnly) {
+            throw new SQLFeatureNotSupportedException();
+        }
+    }
+
+    @Override
+    public Savepoint setSavepoint() throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public Savepoint setSavepoint(String name) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public void setTransactionIsolation(int level) throws SQLException {
+        if (level != Connection.TRANSACTION_READ_COMMITTED) {
+            throw new SQLFeatureNotSupportedException();
+        }
+    }
+
+    @Override
+    public void setTypeMap(Map<String, Class<?>> map) throws SQLException {
+        throw new SQLFeatureNotSupportedException();
+    }
+
+    @Override
+    public boolean isWrapperFor(Class<?> iface) throws SQLException {
+        return iface.isInstance(this);
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public <T> T unwrap(Class<T> iface) throws SQLException {
+        if (!iface.isInstance(this)) {
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode.CLASS_NOT_UNWRAPPABLE)
+                .setMessage(this.getClass().getName() + " not unwrappable from " + iface.getName())
+                .build().buildException();
+        }
+        return (T)this;
+    }
+
+    @Override
+    public void setSchema(String schema) throws SQLException {
+        // TODO Auto-generated method stub
+        
+    }
+
+    @Override
+    public String getSchema() throws SQLException {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
+    @Override
+    public void abort(Executor executor) throws SQLException {
+        // TODO Auto-generated method stub
+        
+    }
+
+    @Override
+    public void setNetworkTimeout(Executor executor, int milliseconds) throws SQLException {
+        // TODO Auto-generated method stub
+        
+    }
+
+    @Override
+    public int getNetworkTimeout() throws SQLException {
+        // TODO Auto-generated method stub
+        return 0;
+    }
+
+    @Override
+    public PMetaData addTable(PTable table) throws SQLException {
+        // TODO: since a connection is only used by one thread at a time,
+        // we could modify this metadata in place since it's not shared.
+        if (scn == null || scn > table.getTimeStamp()) {
+            metaData = metaData.addTable(table);
+        }
+        //Cascade through to connectionQueryServices too
+        getQueryServices().addTable(table);
+        return metaData;
+    }
+
+    @Override
+    public PMetaData addColumn(String tableName, List<PColumn> columns, long tableTimeStamp, long tableSeqNum, boolean isImmutableRows)
+            throws SQLException {
+        metaData = metaData.addColumn(tableName, columns, tableTimeStamp, tableSeqNum, isImmutableRows);
+        //Cascade through to connectionQueryServices too
+        getQueryServices().addColumn(tableName, columns, tableTimeStamp, tableSeqNum, isImmutableRows);
+        return metaData;
+    }
+
+    @Override
+    public PMetaData removeTable(String tableName) throws SQLException {
+        metaData = metaData.removeTable(tableName);
+        //Cascade through to connectionQueryServices too
+        getQueryServices().removeTable(tableName);
+        return metaData;
+    }
+
+    @Override
+    public PMetaData removeColumn(String tableName, String familyName, String columnName, long tableTimeStamp,
+            long tableSeqNum) throws SQLException {
+        metaData = metaData.removeColumn(tableName, familyName, columnName, tableTimeStamp, tableSeqNum);
+        //Cascade through to connectionQueryServices too
+        getQueryServices().removeColumn(tableName, familyName, columnName, tableTimeStamp, tableSeqNum);
+        return metaData;
+    }
+
+    protected boolean removeStatement(PhoenixStatement statement) throws SQLException {
+        return statements.remove(statement);
+   }
+
+    public KeyValueBuilder getKeyValueBuilder() {
+        return this.services.getKeyValueBuilder();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..a46ab5f
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
@@ -0,0 +1,1456 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.jdbc;
+
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.ResultSet;
+import java.sql.RowIdLifetime;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.commons.lang.StringEscapeUtils;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.compile.ColumnProjector;
+import org.apache.phoenix.compile.ExpressionProjector;
+import org.apache.phoenix.compile.RowProjector;
+import org.apache.phoenix.coprocessor.MetaDataProtocol;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+import org.apache.phoenix.expression.BaseTerminalExpression;
+import org.apache.phoenix.expression.RowKeyColumnExpression;
+import org.apache.phoenix.expression.function.IndexStateNameFunction;
+import org.apache.phoenix.expression.function.SQLTableTypeFunction;
+import org.apache.phoenix.expression.function.SQLViewTypeFunction;
+import org.apache.phoenix.expression.function.SqlTypeNameFunction;
+import org.apache.phoenix.iterate.DelegateResultIterator;
+import org.apache.phoenix.iterate.MaterializedResultIterator;
+import org.apache.phoenix.iterate.ResultIterator;
+import org.apache.phoenix.parse.HintNode.Hint;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.PDatum;
+import org.apache.phoenix.schema.PName;
+import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.schema.RowKeyValueAccessor;
+import org.apache.phoenix.schema.tuple.SingleKeyValueTuple;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.KeyValueUtil;
+import org.apache.phoenix.util.MetaDataUtil;
+import org.apache.phoenix.util.SchemaUtil;
+
+import com.google.common.collect.Lists;
+
+
+/**
+ * 
+ * JDBC DatabaseMetaData implementation of Phoenix reflecting read-only nature of driver.
+ * Supported metadata methods include:
+ * {@link #getTables(String, String, String, String[])}
+ * {@link #getColumns(String, String, String, String)}
+ * {@link #getTableTypes()}
+ * {@link #getPrimaryKeys(String, String, String)}
+ * {@link #getIndexInfo(String, String, String, boolean, boolean)}
+ * {@link #getSchemas()}
+ * {@link #getSchemas(String, String)}
+ * {@link #getDatabaseMajorVersion()}
+ * {@link #getDatabaseMinorVersion()}
+ * {@link #getClientInfoProperties()}
+ * {@link #getConnection()}
+ * {@link #getDatabaseProductName()}
+ * {@link #getDatabaseProductVersion()}
+ * {@link #getDefaultTransactionIsolation()}
+ * {@link #getDriverName()}
+ * {@link #getDriverVersion()}
+ * Other ResultSet methods return an empty result set.
+ * 
+ * @author jtaylor
+ * @since 0.1
+ */
+public class PhoenixDatabaseMetaData implements DatabaseMetaData, org.apache.phoenix.jdbc.Jdbc7Shim.DatabaseMetaData {
+    public static final int INDEX_NAME_INDEX = 4; // Shared with FAMILY_NAME_INDEX
+    public static final int FAMILY_NAME_INDEX = 4;
+    public static final int COLUMN_NAME_INDEX = 3;
+    public static final int TABLE_NAME_INDEX = 2;
+    public static final int SCHEMA_NAME_INDEX = 1;
+    public static final int TENANT_ID_INDEX = 0;
+
+    public static final String TYPE_SCHEMA = "SYSTEM";
+    public static final String TYPE_TABLE = "CATALOG";
+    public static final String TYPE_SCHEMA_AND_TABLE = TYPE_SCHEMA + ".\"" + TYPE_TABLE + "\"";
+    public static final byte[] TYPE_TABLE_BYTES = TYPE_TABLE.getBytes();
+    public static final byte[] TYPE_SCHEMA_BYTES = TYPE_SCHEMA.getBytes();
+    public static final String TYPE_TABLE_NAME = SchemaUtil.getTableName(TYPE_SCHEMA, TYPE_TABLE);
+    public static final byte[] TYPE_TABLE_NAME_BYTES = SchemaUtil.getTableNameAsBytes(TYPE_SCHEMA_BYTES, TYPE_TABLE_BYTES);
+    
+    public static final String TYPE_SCHEMA_ALIAS = "SYSTEM";
+    public static final String TYPE_TABLE_ALIAS = "TABLE";
+    public static final String TYPE_SCHEMA_AND_TABLE_ALIAS = "\"" + TYPE_SCHEMA_ALIAS + "." + TYPE_TABLE_ALIAS + "\"";
+
+    public static final String TABLE_NAME_NAME = "TABLE_NAME";
+    public static final String TABLE_TYPE_NAME = "TABLE_TYPE";
+    public static final byte[] TABLE_TYPE_BYTES = Bytes.toBytes(TABLE_TYPE_NAME);
+    
+    public static final String TABLE_CAT_NAME = "TABLE_CAT";
+    public static final String TABLE_CATALOG_NAME = "TABLE_CATALOG";
+    public static final String TABLE_SCHEM_NAME = "TABLE_SCHEM";
+    public static final String REMARKS_NAME = "REMARKS";
+    public static final String TYPE_CAT_NAME = "TYPE_CAT";
+    public static final String TYPE_SCHEM_NAME = "TYPE_SCHEM";
+    public static final String TYPE_NAME_NAME = "TYPE_NAME";
+    public static final String SELF_REFERENCING_COL_NAME_NAME = "SELF_REFERENCING_COL_NAME";
+    public static final String REF_GENERATION_NAME = "REF_GENERATION";
+    public static final String PK_NAME = "PK_NAME";
+    public static final byte[] PK_NAME_BYTES = Bytes.toBytes(PK_NAME);
+    public static final String TABLE_SEQ_NUM = "TABLE_SEQ_NUM";
+    public static final byte[] TABLE_SEQ_NUM_BYTES = Bytes.toBytes(TABLE_SEQ_NUM);
+    public static final String COLUMN_COUNT = "COLUMN_COUNT";
+    public static final byte[] COLUMN_COUNT_BYTES = Bytes.toBytes(COLUMN_COUNT);
+    public static final String SALT_BUCKETS = "SALT_BUCKETS";
+    public static final byte[] SALT_BUCKETS_BYTES = Bytes.toBytes(SALT_BUCKETS);
+ 
+    public static final String DATA_TABLE_NAME = "DATA_TABLE_NAME";
+    public static final byte[] DATA_TABLE_NAME_BYTES = Bytes.toBytes(DATA_TABLE_NAME);
+    public static final String INDEX_STATE = "INDEX_STATE";
+    public static final byte[] INDEX_STATE_BYTES = Bytes.toBytes(INDEX_STATE);
+
+    public static final String TENANT_ID = "TENANT_ID";
+    public static final byte[] TENANT_ID_BYTES = Bytes.toBytes(TENANT_ID);
+    
+    public static final String COLUMN_NAME = "COLUMN_NAME";
+    public static final String DATA_TYPE = "DATA_TYPE";
+    public static final byte[] DATA_TYPE_BYTES = Bytes.toBytes(DATA_TYPE);
+    public static final String TYPE_NAME = "TYPE_NAME";
+    public static final String COLUMN_SIZE = "COLUMN_SIZE";
+    public static final String BUFFER_LENGTH = "BUFFER_LENGTH";
+    public static final String DECIMAL_DIGITS = "DECIMAL_DIGITS";
+    public static final String NUM_PREC_RADIX = "NUM_PREC_RADIX";
+    public static final String NULLABLE = "NULLABLE";
+    public static final String COLUMN_DEF = "COLUMN_DEF";
+    public static final String SQL_DATA_TYPE = "SQL_DATA_TYPE";
+    public static final String SQL_DATETIME_SUB = "SQL_DATETIME_SUB";
+    public static final String CHAR_OCTET_LENGTH = "CHAR_OCTET_LENGTH";
+    public static final String ORDINAL_POSITION = "ORDINAL_POSITION";
+    public static final String IS_NULLABLE = "IS_NULLABLE";
+    public static final String SCOPE_CATALOG = "SCOPE_CATALOG";
+    public static final String SCOPE_SCHEMA = "SCOPE_SCHEMA";
+    public static final String SCOPE_TABLE = "SCOPE_TABLE";
+    public static final String SOURCE_DATA_TYPE = "SOURCE_DATA_TYPE";
+    public static final String IS_AUTOINCREMENT = "IS_AUTOINCREMENT";
+    public static final String COLUMN_MODIFIER = "COLUMN_MODIFIER";
+    public static final String IMMUTABLE_ROWS = "IMMUTABLE_ROWS";
+    public static final byte[] IMMUTABLE_ROWS_BYTES = Bytes.toBytes(IMMUTABLE_ROWS);
+    public static final String DEFAULT_COLUMN_FAMILY_NAME = "DEFAULT_COLUMN_FAMILY";
+    public static final byte[] DEFAULT_COLUMN_FAMILY_NAME_BYTES = Bytes.toBytes(DEFAULT_COLUMN_FAMILY_NAME);
+    public static final String VIEW_STATEMENT = "VIEW_STATEMENT";
+    public static final byte[] VIEW_STATEMENT_BYTES = Bytes.toBytes(VIEW_STATEMENT);
+    public static final String DISABLE_WAL = "DISABLE_WAL";
+    public static final byte[] DISABLE_WAL_BYTES = Bytes.toBytes(DISABLE_WAL);
+    public static final String MULTI_TENANT = "MULTI_TENANT";
+    public static final byte[] MULTI_TENANT_BYTES = Bytes.toBytes(MULTI_TENANT);
+    public static final String VIEW_TYPE = "VIEW_TYPE";
+    public static final byte[] VIEW_TYPE_BYTES = Bytes.toBytes(VIEW_TYPE);
+    public static final String LINK_TYPE = "LINK_TYPE";
+    public static final byte[] LINK_TYPE_BYTES = Bytes.toBytes(LINK_TYPE);
+    public static final String ARRAY_SIZE = "ARRAY_SIZE";
+    public static final byte[] ARRAY_SIZE_BYTES = Bytes.toBytes(ARRAY_SIZE);
+
+    public static final String TABLE_FAMILY = QueryConstants.DEFAULT_COLUMN_FAMILY;
+    public static final byte[] TABLE_FAMILY_BYTES = QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES;
+    
+    public static final String TYPE_SEQUENCE = "SEQUENCE";
+    public static final byte[] SEQUENCE_FAMILY_BYTES = QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES;
+    public static final String SEQUENCE_TABLE_NAME = TYPE_SCHEMA + ".\"" + TYPE_SEQUENCE + "\"";
+    public static final byte[] SEQUENCE_TABLE_NAME_BYTES = SchemaUtil.getTableNameAsBytes(TYPE_SCHEMA, TYPE_SEQUENCE);
+    public static final String SEQUENCE_SCHEMA = "SEQUENCE_SCHEMA";
+    public static final String SEQUENCE_NAME = "SEQUENCE_NAME";
+    public static final String CURRENT_VALUE = "CURRENT_VALUE";
+    public static final byte[] CURRENT_VALUE_BYTES = Bytes.toBytes(CURRENT_VALUE);
+    public static final String START_WITH = "START_WITH";
+    public static final byte[] START_WITH_BYTES = Bytes.toBytes(START_WITH);
+    public static final String INCREMENT_BY = "INCREMENT_BY";
+    public static final byte[] INCREMENT_BY_BYTES = Bytes.toBytes(INCREMENT_BY);
+    public static final String CACHE_SIZE = "CACHE_SIZE";
+    public static final byte[] CACHE_SIZE_BYTES = Bytes.toBytes(CACHE_SIZE);
+    
+    private final PhoenixConnection connection;
+    private final ResultSet emptyResultSet;
+
+    // Version below which we should turn off essential column family.
+    public static final int ESSENTIAL_FAMILY_VERSION_THRESHOLD = MetaDataUtil.encodeVersion("0", "94", "7");
+    // Version below which we should disallow usage of mutable secondary indexing.
+    public static final int MUTABLE_SI_VERSION_THRESHOLD = MetaDataUtil.encodeVersion("0", "94", "10");
+    /** Version below which we fall back on the generic KeyValueBuilder */
+    public static final int CLIENT_KEY_VALUE_BUILDER_THRESHOLD = MetaDataUtil.encodeVersion("0", "94", "14");
+
+    PhoenixDatabaseMetaData(PhoenixConnection connection) throws SQLException {
+        this.emptyResultSet = new PhoenixResultSet(ResultIterator.EMPTY_ITERATOR, RowProjector.EMPTY_PROJECTOR, new PhoenixStatement(connection));
+        this.connection = connection;
+    }
+
+    @Override
+    public boolean allProceduresAreCallable() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean allTablesAreSelectable() throws SQLException {
+        return true;
+    }
+
+    @Override
+    public boolean autoCommitFailureClosesAllResultSets() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean dataDefinitionCausesTransactionCommit() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean dataDefinitionIgnoredInTransactions() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean deletesAreDetected(int type) throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean doesMaxRowSizeIncludeBlobs() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public ResultSet getAttributes(String catalog, String schemaPattern, String typeNamePattern,
+            String attributeNamePattern) throws SQLException {
+        return emptyResultSet;
+    }
+
+    @Override
+    public ResultSet getBestRowIdentifier(String catalog, String schema, String table, int scope, boolean nullable)
+            throws SQLException {
+        return emptyResultSet;
+    }
+
+    @Override
+    public String getCatalogSeparator() throws SQLException {
+        return ".";
+    }
+
+    @Override
+    public String getCatalogTerm() throws SQLException {
+        return "Catalog";
+    }
+
+    @Override
+    public ResultSet getCatalogs() throws SQLException {
+        return emptyResultSet;
+    }
+
+    @Override
+    public ResultSet getClientInfoProperties() throws SQLException {
+        return emptyResultSet;
+    }
+
+    @Override
+    public ResultSet getColumnPrivileges(String catalog, String schema, String table, String columnNamePattern)
+            throws SQLException {
+        return emptyResultSet;
+    }
+    
+    private String getTenantIdWhereClause() {
+        PName tenantId = connection.getTenantId();
+        return "(" + TENANT_ID + " IS NULL " + 
+                (tenantId == null
+                   ? ") "
+                   : " OR " + TENANT_ID + " = '" + StringEscapeUtils.escapeSql(tenantId.getString()) + "') ");
+    }
+
+    @Override
+    public ResultSet getColumns(String catalog, String schemaPattern, String tableNamePattern, String columnNamePattern)
+            throws SQLException {
+        StringBuilder buf = new StringBuilder("select /*+" + Hint.NO_INTRA_REGION_PARALLELIZATION + "*/" +
+                TABLE_CAT_NAME + "," + // use this column for column family name
+                TABLE_SCHEM_NAME + "," +
+                TABLE_NAME_NAME + " ," +
+                COLUMN_NAME + "," +
+                DATA_TYPE + "," +
+                SqlTypeNameFunction.NAME + "(" + DATA_TYPE + ") AS " + TYPE_NAME + "," +
+                COLUMN_SIZE + "," +
+                BUFFER_LENGTH + "," +
+                DECIMAL_DIGITS + "," +
+                NUM_PREC_RADIX + "," +
+                NULLABLE + "," +
+                COLUMN_DEF + "," +
+                SQL_DATA_TYPE + "," +
+                SQL_DATETIME_SUB + "," +
+                CHAR_OCTET_LENGTH + "," +
+                ORDINAL_POSITION + "," +
+                "CASE " + NULLABLE + " WHEN " + DatabaseMetaData.attributeNoNulls +  " THEN '" + Boolean.FALSE.toString() + "' WHEN " + DatabaseMetaData.attributeNullable + " THEN '" + Boolean.TRUE.toString() + "' END AS " + IS_NULLABLE + "," +
+                SCOPE_CATALOG + "," +
+                SCOPE_SCHEMA + "," +
+                SCOPE_TABLE + "," +
+                SOURCE_DATA_TYPE + "," +
+                IS_AUTOINCREMENT + "," + 
+                ARRAY_SIZE +
+                " from " + TYPE_SCHEMA_AND_TABLE + " " + TYPE_SCHEMA_AND_TABLE_ALIAS +
+                " where ");
+        buf.append(getTenantIdWhereClause());
+        if (schemaPattern != null) {
+            buf.append(" and " + TABLE_SCHEM_NAME + (schemaPattern.length() == 0 ? " is null" : " like '" + SchemaUtil.normalizeIdentifier(schemaPattern) + "'" ));
+        }
+        if (tableNamePattern != null && tableNamePattern.length() > 0) {
+            buf.append(" and " + TABLE_NAME_NAME + " like '" + SchemaUtil.normalizeIdentifier(tableNamePattern) + "'" );
+        }
+        if (catalog != null && catalog.length() > 0) { // if null or empty, will pick up all columns
+            // Will pick up only KV columns
+            // We supported only getting the PK columns by using catalog="", but some clients pass this through
+            // instead of null (namely SQLLine), so better to just treat these the same. If only PK columns are
+            // wanted, you can just stop the scan when you get to a non null TABLE_CAT_NAME
+            buf.append(" and " + TABLE_CAT_NAME + " like '" + SchemaUtil.normalizeIdentifier(catalog) + "'" );
+        }
+        if (columnNamePattern != null && columnNamePattern.length() > 0) {
+            buf.append(" and " + COLUMN_NAME + " like '" + SchemaUtil.normalizeIdentifier(columnNamePattern) + "'" );
+        } else {
+            buf.append(" and " + COLUMN_NAME + " is not null" );
+        }
+        buf.append(" order by " + TABLE_SCHEM_NAME + "," + TABLE_NAME_NAME + "," + ORDINAL_POSITION);
+        Statement stmt = connection.createStatement();
+        return stmt.executeQuery(buf.toString());
+    }
+
+    @Override
+    public Connection getConnection() throws SQLException {
+        return connection;
+    }
+
+    @Override
+    public ResultSet getCrossReference(String parentCatalog, String parentSchema, String parentTable,
+            String foreignCatalog, String foreignSchema, String foreignTable) throws SQLException {
+        return emptyResultSet;
+    }
+
+    @Override
+    public int getDatabaseMajorVersion() throws SQLException {
+        return MetaDataProtocol.PHOENIX_MAJOR_VERSION;
+    }
+
+    @Override
+    public int getDatabaseMinorVersion() throws SQLException {
+        return MetaDataProtocol.PHOENIX_MINOR_VERSION;
+    }
+
+    @Override
+    public String getDatabaseProductName() throws SQLException {
+        return "Phoenix";
+    }
+
+    @Override
+    public String getDatabaseProductVersion() throws SQLException {
+        return Integer.toString(getDatabaseMajorVersion()) + "." + Integer.toString(getDatabaseMinorVersion());
+    }
+
+    @Override
+    public int getDefaultTransactionIsolation() throws SQLException {
+        return connection.getTransactionIsolation();
+    }
+
+    @Override
+    public int getDriverMajorVersion() {
+        return Integer.parseInt(connection.getClientInfo(PhoenixEmbeddedDriver.MAJOR_VERSION_PROP));
+    }
+
+    @Override
+    public int getDriverMinorVersion() {
+        return Integer.parseInt(connection.getClientInfo(PhoenixEmbeddedDriver.MINOR_VERSION_PROP));
+    }
+
+    @Override
+    public String getDriverName() throws SQLException {
+        return connection.getClientInfo(PhoenixEmbeddedDriver.DRIVER_NAME_PROP);
+    }
+
+    @Override
+    public String getDriverVersion() throws SQLException {
+        return connection.getClientInfo(PhoenixEmbeddedDriver.MAJOR_VERSION_PROP) + "." + connection.getClientInfo(PhoenixEmbeddedDriver.MINOR_VERSION_PROP);
+    }
+
+    @Override
+    public ResultSet getExportedKeys(String catalog, String schema, String table) throws SQLException {
+        return emptyResultSet;
+    }
+
+    @Override
+    public String getExtraNameCharacters() throws SQLException {
+        return "";
+    }
+
+    @Override
+    public ResultSet getFunctionColumns(String catalog, String schemaPattern, String functionNamePattern,
+            String columnNamePattern) throws SQLException {
+        return emptyResultSet;
+    }
+
+    @Override
+    public ResultSet getFunctions(String catalog, String schemaPattern, String functionNamePattern) throws SQLException {
+        return emptyResultSet;
+    }
+
+    @Override
+    public String getIdentifierQuoteString() throws SQLException {
+        return "\"";
+    }
+
+    @Override
+    public ResultSet getImportedKeys(String catalog, String schema, String table) throws SQLException {
+        return emptyResultSet;
+    }
+
+    @Override
+    public ResultSet getIndexInfo(String catalog, String schema, String table, boolean unique, boolean approximate)
+            throws SQLException {
+        // Catalogs are not supported for schemas
+        if (catalog != null && catalog.length() > 0) {
+            return emptyResultSet;
+        }
+        if (unique) { // No unique indexes
+            return emptyResultSet;
+        }
+        StringBuilder buf = new StringBuilder("select /*+" + Hint.NO_INTRA_REGION_PARALLELIZATION + "*/\n" +
+                "null " + TABLE_CAT_NAME + ",\n" + // use this column for column family name
+                TABLE_SCHEM_NAME + ",\n" +
+                DATA_TABLE_NAME + " " + TABLE_NAME_NAME + ",\n" +
+                "true NON_UNIQUE,\n" +
+                "null INDEX_QUALIFIER,\n" +
+                TABLE_NAME_NAME + " INDEX_NAME,\n" +
+                DatabaseMetaData.tableIndexOther + " TYPE,\n" + 
+                ORDINAL_POSITION + ",\n" +
+                COLUMN_NAME + ",\n" +
+                "CASE WHEN " + TABLE_CAT_NAME + " IS NOT NULL THEN null WHEN " + COLUMN_MODIFIER + " = " + ColumnModifier.toSystemValue(ColumnModifier.SORT_DESC) + " THEN 'D' ELSE 'A' END ASC_OR_DESC,\n" +
+                "null CARDINALITY,\n" +
+                "null PAGES,\n" +
+                "null FILTER_CONDITION,\n" +
+                DATA_TYPE + ",\n" + // Include data type info, though not in spec
+                SqlTypeNameFunction.NAME + "(" + DATA_TYPE + ") AS " + TYPE_NAME + 
+                "\nfrom " + TYPE_SCHEMA_AND_TABLE + 
+                "\nwhere ");
+        buf.append(getTenantIdWhereClause());
+        buf.append("\nand " + TABLE_SCHEM_NAME + (schema == null || schema.length() == 0 ? " is null" : " = '" + SchemaUtil.normalizeIdentifier(schema) + "'" ));
+        buf.append("\nand " + DATA_TABLE_NAME + " = '" + SchemaUtil.normalizeIdentifier(table) + "'" );
+        buf.append("\nand " + COLUMN_NAME + " is not null" );
+        buf.append("\norder by INDEX_NAME," + ORDINAL_POSITION);
+        Statement stmt = connection.createStatement();
+        return stmt.executeQuery(buf.toString());
+    }
+
+
+    @Override
+    public int getJDBCMajorVersion() throws SQLException {
+        return 1;
+    }
+
+    @Override
+    public int getJDBCMinorVersion() throws SQLException {
+        return 0;
+    }
+
+    @Override
+    public int getMaxBinaryLiteralLength() throws SQLException {
+        return 0;
+    }
+
+    @Override
+    public int getMaxCatalogNameLength() throws SQLException {
+        return 0;
+    }
+
+    @Override
+    public int getMaxCharLiteralLength() throws SQLException {
+        return 4000;
+    }
+
+    @Override
+    public int getMaxColumnNameLength() throws SQLException {
+        return 200;
+    }
+
+    @Override
+    public int getMaxColumnsInGroupBy() throws SQLException {
+        return 1;
+    }
+
+    @Override
+    public int getMaxColumnsInIndex() throws SQLException {
+        return 0;
+    }
+
+    @Override
+    public int getMaxColumnsInOrderBy() throws SQLException {
+        return 0;
+    }
+
+    @Override
+    public int getMaxColumnsInSelect() throws SQLException {
+        return 0;
+    }
+
+    @Override
+    public int getMaxColumnsInTable() throws SQLException {
+        return 0;
+    }
+
+    @Override
+    public int getMaxConnections() throws SQLException {
+        return 0;
+    }
+
+    @Override
+    public int getMaxCursorNameLength() throws SQLException {
+        return 0;
+    }
+
+    @Override
+    public int getMaxIndexLength() throws SQLException {
+        return 0;
+    }
+
+    @Override
+    public int getMaxProcedureNameLength() throws SQLException {
+        return 0;
+    }
+
+    @Override
+    public int getMaxRowSize() throws SQLException {
+        return 0;
+    }
+
+    @Override
+    public int getMaxSchemaNameLength() throws SQLException {
+        return 0;
+    }
+
+    @Override
+    public int getMaxStatementLength() throws SQLException {
+        return 0;
+    }
+
+    @Override
+    public int getMaxStatements() throws SQLException {
+        return 0;
+    }
+
+    @Override
+    public int getMaxTableNameLength() throws SQLException {
+        return 0;
+    }
+
+    @Override
+    public int getMaxTablesInSelect() throws SQLException {
+        return 1;
+    }
+
+    @Override
+    public int getMaxUserNameLength() throws SQLException {
+        return 0;
+    }
+
+    @Override
+    public String getNumericFunctions() throws SQLException {
+        return "";
+    }
+
+    @Override
+    public ResultSet getPrimaryKeys(String catalog, String schema, String table) throws SQLException {
+        // Catalogs are not supported for schemas
+        if (catalog != null && catalog.length() > 0) {
+            return emptyResultSet;
+        }
+        if (table == null || table.length() == 0) {
+            return emptyResultSet;
+        }
+        final int keySeqPosition = 4;
+        final int pkNamePosition = 5;
+        StringBuilder buf = new StringBuilder("select /*+" + Hint.NO_INTRA_REGION_PARALLELIZATION + "*/" +
+                TABLE_CAT_NAME + "," + // use this column for column family name
+                TABLE_SCHEM_NAME + "," +
+                TABLE_NAME_NAME + " ," +
+                COLUMN_NAME + "," +
+                "null as KEY_SEQ," +
+                "PK_NAME" + "," +
+                "CASE WHEN " + COLUMN_MODIFIER + " = " + ColumnModifier.toSystemValue(ColumnModifier.SORT_DESC) + " THEN 'D' ELSE 'A' END ASC_OR_DESC," +
+                DATA_TYPE + "," + // include type info, though not in spec
+                SqlTypeNameFunction.NAME + "(" + DATA_TYPE + ") AS " + TYPE_NAME +
+                " from " + TYPE_SCHEMA_AND_TABLE + " " + TYPE_SCHEMA_AND_TABLE_ALIAS +
+                " where ");
+        buf.append(getTenantIdWhereClause());
+        buf.append(" and " + TABLE_SCHEM_NAME + (schema == null || schema.length() == 0 ? " is null" : " = '" + SchemaUtil.normalizeIdentifier(schema) + "'" ));
+        buf.append(" and " + TABLE_NAME_NAME + " = '" + SchemaUtil.normalizeIdentifier(table) + "'" );
+        buf.append(" and " + TABLE_CAT_NAME + " is null" );
+        buf.append(" order by " + ORDINAL_POSITION);
+        // Dynamically replaces the KEY_SEQ with an expression that gets incremented after each next call.
+        Statement stmt = connection.createStatement(new PhoenixStatementFactory() {
+
+            @Override
+            public PhoenixStatement newStatement(PhoenixConnection connection) {
+                final byte[] unsetValue = new byte[0];
+                final ImmutableBytesWritable pkNamePtr = new ImmutableBytesWritable(unsetValue);
+                final byte[] rowNumberHolder = new byte[PDataType.INTEGER.getByteSize()];
+                return new PhoenixStatement(connection) {
+                    @Override
+                    protected PhoenixResultSet newResultSet(ResultIterator iterator, RowProjector projector) throws SQLException {
+                        List<ColumnProjector> columns = new ArrayList<ColumnProjector>(projector.getColumnProjectors());
+                        ColumnProjector column = columns.get(keySeqPosition);
+                        
+                        columns.set(keySeqPosition, new ExpressionProjector(column.getName(), column.getTableName(), 
+                                new BaseTerminalExpression() {
+                                    @Override
+                                    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+                                        ptr.set(rowNumberHolder);
+                                        return true;
+                                    }
+
+                                    @Override
+                                    public PDataType getDataType() {
+                                        return PDataType.INTEGER;
+                                    }
+                                },
+                                column.isCaseSensitive())
+                        );
+                        column = columns.get(pkNamePosition);
+                        columns.set(pkNamePosition, new ExpressionProjector(column.getName(), column.getTableName(), 
+                                new BaseTerminalExpression() {
+                                    @Override
+                                    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+                                        if (pkNamePtr.get() == unsetValue) {
+                                            boolean b = tuple.getValue(TABLE_FAMILY_BYTES, PK_NAME_BYTES, pkNamePtr);
+                                            if (!b) {
+                                                pkNamePtr.set(ByteUtil.EMPTY_BYTE_ARRAY);
+                                            }
+                                        }
+                                        ptr.set(pkNamePtr.get(),pkNamePtr.getOffset(),pkNamePtr.getLength());
+                                        return true;
+                                    }
+
+                                    @Override
+                                    public PDataType getDataType() {
+                                        return PDataType.VARCHAR;
+                                    }
+                                },
+                                column.isCaseSensitive())
+                        );
+                        final RowProjector newProjector = new RowProjector(columns, projector.getEstimatedRowByteSize(), projector.isProjectEmptyKeyValue());
+                        ResultIterator delegate = new DelegateResultIterator(iterator) {
+                            private int rowCount = 0;
+
+                            @Override
+                            public Tuple next() throws SQLException {
+                                // Ignore first row, since it's the table row
+                                PDataType.INTEGER.toBytes(rowCount++, rowNumberHolder, 0);
+                                return super.next();
+                            }
+                        };
+                        return new PhoenixResultSet(delegate, newProjector, this);
+                    }
+                    
+                };
+            }
+            
+        });
+        ResultSet rs = stmt.executeQuery(buf.toString());
+        if (rs.next()) { // Skip table row - we just use that to get the PK_NAME
+            rs.getString(pkNamePosition+1); // Hack to cause the statement to cache this value
+        }
+        return rs;
+    }
+
+    @Override
+    public ResultSet getProcedureColumns(String catalog, String schemaPattern, String procedureNamePattern,
+            String columnNamePattern) throws SQLException {
+        return emptyResultSet;
+    }
+
+    @Override
+    public String getProcedureTerm() throws SQLException {
+        return "procedure";
+    }
+
+    @Override
+    public ResultSet getProcedures(String catalog, String schemaPattern, String procedureNamePattern)
+            throws SQLException {
+        return emptyResultSet;
+    }
+
+    @Override
+    public int getResultSetHoldability() throws SQLException {
+        return connection.getHoldability();
+    }
+
+    @Override
+    public RowIdLifetime getRowIdLifetime() throws SQLException {
+        return RowIdLifetime.ROWID_UNSUPPORTED;
+    }
+
+    @Override
+    public String getSQLKeywords() throws SQLException {
+        return "";
+    }
+
+    @Override
+    public int getSQLStateType() throws SQLException {
+        return DatabaseMetaData.sqlStateSQL99;
+    }
+
+    @Override
+    public String getSchemaTerm() throws SQLException {
+        return "schema";
+    }
+
+    @Override
+    public ResultSet getSchemas() throws SQLException {
+        return getSchemas(null, null);
+    }
+
+    @Override
+    public ResultSet getSchemas(String catalog, String schemaPattern) throws SQLException {
+        // Catalogs are not supported for schemas
+        if (catalog != null && catalog.length() > 0) {
+            return emptyResultSet;
+        }
+        StringBuilder buf = new StringBuilder("select /*+" + Hint.NO_INTRA_REGION_PARALLELIZATION + "*/ distinct " +
+                "null " + TABLE_CATALOG_NAME + "," + // no catalog for tables
+                TABLE_SCHEM_NAME +
+                " from " + TYPE_SCHEMA_AND_TABLE + " " + TYPE_SCHEMA_AND_TABLE_ALIAS +
+                " where " + COLUMN_NAME + " is null");
+        if (schemaPattern != null) {
+            buf.append(" and " + TABLE_SCHEM_NAME + " like '" + SchemaUtil.normalizeIdentifier(schemaPattern) + "'");
+        }
+        Statement stmt = connection.createStatement();
+        return stmt.executeQuery(buf.toString());
+    }
+
+    @Override
+    public String getSearchStringEscape() throws SQLException {
+        return "\\";
+    }
+
+    @Override
+    public String getStringFunctions() throws SQLException {
+        return "";
+    }
+
+    private ResultSet getSuperTables(String catalog, String schemaPattern, String tableNamePattern, String typeNamePattern) throws SQLException {
+        // Catalogs are not supported for schemas
+        // Tenant specific connections have no derived tables
+        if (catalog != null && catalog.length() > 0 || (connection.getTenantId() != null)) {
+            return emptyResultSet;
+        }
+        StringBuilder buf = new StringBuilder("select /*+" + Hint.NO_INTRA_REGION_PARALLELIZATION + "*/" +
+                TABLE_CAT_NAME + "," + // no catalog for tables
+                TABLE_SCHEM_NAME + "," +
+                TABLE_NAME_NAME + " ," +
+                SQLTableTypeFunction.NAME + "(" + TABLE_TYPE_NAME + ") AS " + TABLE_TYPE_NAME + "," +
+                REMARKS_NAME + " ," +
+                TYPE_NAME + "," +
+                SELF_REFERENCING_COL_NAME_NAME + "," +
+                REF_GENERATION_NAME + "," +
+                IndexStateNameFunction.NAME + "(" + INDEX_STATE + ") AS " + INDEX_STATE + "," +
+                IMMUTABLE_ROWS + "," +
+                SALT_BUCKETS + "," +
+                TENANT_ID + "," + 
+                VIEW_STATEMENT + "," +
+                SQLViewTypeFunction.NAME + "(" + VIEW_TYPE + ") AS " + VIEW_TYPE +
+                " from " + TYPE_SCHEMA_AND_TABLE + " " + TYPE_SCHEMA_AND_TABLE_ALIAS +
+                " where " + COLUMN_NAME + " is null" +
+                " and " + TABLE_CAT_NAME + " is null");
+        buf.append(" and " + TENANT_ID + " IS NOT NULL ");
+        if (schemaPattern != null) {
+            buf.append(" and " + TABLE_SCHEM_NAME + (schemaPattern.length() == 0 ? " is null" : " like '" + SchemaUtil.normalizeIdentifier(schemaPattern) + "'" ));
+        }
+        if (tableNamePattern != null) {
+            buf.append(" and " + TABLE_NAME_NAME + " like '" + SchemaUtil.normalizeIdentifier(tableNamePattern) + "'" );
+        }
+        if (typeNamePattern != null) {
+            buf.append(" and " + SQLTableTypeFunction.NAME + "(" + TABLE_TYPE_NAME + ") like '" + SchemaUtil.normalizeIdentifier(typeNamePattern) + "'" );
+        }
+        buf.append(" order by " + TENANT_ID + "," + TYPE_SCHEMA_AND_TABLE + "." +TABLE_TYPE_NAME + "," + TABLE_SCHEM_NAME + "," + TABLE_NAME_NAME);
+        Statement stmt = connection.createStatement();
+        return stmt.executeQuery(buf.toString());
+    }
+    
+    /**
+     * Use/abuse this to get the derived tables ordered by TENANT_ID, TABLE_TYPE, SCHEMA_NAME, TABLE_NAME
+     */
+    @Override
+    public ResultSet getSuperTables(String catalog, String schemaPattern, String tableNamePattern) throws SQLException {
+        return getSuperTables(catalog, schemaPattern, tableNamePattern, null);
+    }
+
+    @Override
+    public ResultSet getSuperTypes(String catalog, String schemaPattern, String typeNamePattern) throws SQLException {
+        return getSuperTables(catalog, schemaPattern, null, typeNamePattern);
+    }
+
+    @Override
+    public String getSystemFunctions() throws SQLException {
+        return "";
+    }
+
+    @Override
+    public ResultSet getTablePrivileges(String catalog, String schemaPattern, String tableNamePattern)
+            throws SQLException {
+        return emptyResultSet;
+    }
+
+    private static final PDatum TABLE_TYPE_DATUM = new PDatum() {
+        @Override
+        public boolean isNullable() {
+            return false;
+        }
+        @Override
+        public PDataType getDataType() {
+            return PDataType.VARCHAR;
+        }
+        @Override
+        public Integer getByteSize() {
+            return null;
+        }
+        @Override
+        public Integer getMaxLength() {
+            return null;
+        }
+        @Override
+        public Integer getScale() {
+            return null;
+        }
+		@Override
+		public ColumnModifier getColumnModifier() {
+			return null;
+		}
+    };
+    private static final RowProjector TABLE_TYPE_ROW_PROJECTOR = new RowProjector(Arrays.<ColumnProjector>asList(
+            new ExpressionProjector(TABLE_TYPE_NAME, TYPE_SCHEMA_AND_TABLE, 
+                    new RowKeyColumnExpression(TABLE_TYPE_DATUM,
+                            new RowKeyValueAccessor(Collections.<PDatum>singletonList(TABLE_TYPE_DATUM), 0)), false)
+            ), 0, true);
+    private static final Collection<Tuple> TABLE_TYPE_TUPLES = Lists.newArrayListWithExpectedSize(PTableType.values().length);
+    static {
+        for (PTableType tableType : PTableType.values()) {
+            TABLE_TYPE_TUPLES.add(new SingleKeyValueTuple(KeyValueUtil.newKeyValue(tableType.getValue().getBytes(), TABLE_FAMILY_BYTES, TABLE_TYPE_BYTES, MetaDataProtocol.MIN_TABLE_TIMESTAMP, ByteUtil.EMPTY_BYTE_ARRAY)));
+        }
+    }
+    @Override
+    public ResultSet getTableTypes() throws SQLException {
+        return new PhoenixResultSet(new MaterializedResultIterator(TABLE_TYPE_TUPLES), TABLE_TYPE_ROW_PROJECTOR, new PhoenixStatement(connection));
+    }
+
+    /**
+     * We support either:
+     * 1) A non null tableNamePattern to find an exactly match with a table name, in which case either a single
+     *    row would be returned in the ResultSet (if found) or no rows would be returned (if not
+     *    found).
+     * 2) A null tableNamePattern, in which case the ResultSet returned would have one row per
+     *    table.
+     * Note that catalog and schemaPattern must be null or an empty string and types must be null
+     * or "TABLE".  Otherwise, no rows will be returned.
+     */
+    @Override
+    public ResultSet getTables(String catalog, String schemaPattern, String tableNamePattern, String[] types)
+            throws SQLException {
+        // Catalogs are not supported for schemas
+        if (catalog != null && catalog.length() > 0) {
+            return emptyResultSet;
+        }
+        StringBuilder buf = new StringBuilder("select /*+" + Hint.NO_INTRA_REGION_PARALLELIZATION + "*/" +
+                TABLE_CAT_NAME + "," + // no catalog for tables
+                TABLE_SCHEM_NAME + "," +
+                TABLE_NAME_NAME + " ," +
+                SQLTableTypeFunction.NAME + "(" + TABLE_TYPE_NAME + ") AS " + TABLE_TYPE_NAME + "," +
+                REMARKS_NAME + " ," +
+                TYPE_NAME + "," +
+                SELF_REFERENCING_COL_NAME_NAME + "," +
+                REF_GENERATION_NAME + "," +
+                IndexStateNameFunction.NAME + "(" + INDEX_STATE + ") AS " + INDEX_STATE + "," +
+                IMMUTABLE_ROWS + "," +
+                SALT_BUCKETS + "," +
+                MULTI_TENANT + "," +
+                VIEW_STATEMENT + "," +
+                SQLViewTypeFunction.NAME + "(" + VIEW_TYPE + ") AS " + VIEW_TYPE +
+                " from " + TYPE_SCHEMA_AND_TABLE + " " + TYPE_SCHEMA_AND_TABLE_ALIAS +
+                " where " + COLUMN_NAME + " is null" +
+                " and " + TABLE_CAT_NAME + " is null");
+        buf.append(" and " + getTenantIdWhereClause());
+        if (schemaPattern != null) {
+            buf.append(" and " + TABLE_SCHEM_NAME + (schemaPattern.length() == 0 ? " is null" : " like '" + SchemaUtil.normalizeIdentifier(schemaPattern) + "'" ));
+        }
+        if (tableNamePattern != null) {
+            buf.append(" and " + TABLE_NAME_NAME + " like '" + SchemaUtil.normalizeIdentifier(tableNamePattern) + "'" );
+        }
+        if (types != null && types.length > 0) {
+            buf.append(" and " + TABLE_TYPE_NAME + " IN (");
+            // For b/w compat as table types changed in 2.2.0 TODO remove in 3.0
+            if (types[0].length() == 1) {
+                for (String type : types) {
+                    buf.append('\'');
+                    buf.append(type);
+                    buf.append('\'');
+                    buf.append(',');
+                }
+            } else {
+                for (String type : types) {
+                    buf.append('\'');
+                    buf.append(PTableType.fromValue(type).getSerializedValue());
+                    buf.append('\'');
+                    buf.append(',');
+                }
+            }
+            buf.setCharAt(buf.length()-1, ')');
+        }
+        buf.append(" order by " + TYPE_SCHEMA_AND_TABLE + "." +TABLE_TYPE_NAME + "," + TABLE_SCHEM_NAME + "," + TABLE_NAME_NAME);
+        Statement stmt = connection.createStatement();
+        return stmt.executeQuery(buf.toString());
+    }
+
+    @Override
+    public String getTimeDateFunctions() throws SQLException {
+        return "";
+    }
+
+    @Override
+    public ResultSet getTypeInfo() throws SQLException {
+        return emptyResultSet;
+    }
+
+    @Override
+    public ResultSet getUDTs(String catalog, String schemaPattern, String typeNamePattern, int[] types)
+            throws SQLException {
+        return emptyResultSet;
+    }
+
+    @Override
+    public String getURL() throws SQLException {
+        return connection.getURL();
+    }
+
+    @Override
+    public String getUserName() throws SQLException {
+        return ""; // FIXME: what should we return here?
+    }
+
+    @Override
+    public ResultSet getVersionColumns(String catalog, String schema, String table) throws SQLException {
+        return emptyResultSet;
+    }
+
+    @Override
+    public boolean insertsAreDetected(int type) throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean isCatalogAtStart() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean isReadOnly() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean locatorsUpdateCopy() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean nullPlusNonNullIsNull() throws SQLException {
+        return true;
+    }
+
+    @Override
+    public boolean nullsAreSortedAtEnd() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean nullsAreSortedAtStart() throws SQLException {
+        return true;
+    }
+
+    @Override
+    public boolean nullsAreSortedHigh() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean nullsAreSortedLow() throws SQLException {
+        return true;
+    }
+
+    @Override
+    public boolean othersDeletesAreVisible(int type) throws SQLException {
+        return true;
+    }
+
+    @Override
+    public boolean othersInsertsAreVisible(int type) throws SQLException {
+        return true;
+    }
+
+    @Override
+    public boolean othersUpdatesAreVisible(int type) throws SQLException {
+        return true;
+    }
+
+    @Override
+    public boolean ownDeletesAreVisible(int type) throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean ownInsertsAreVisible(int type) throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean ownUpdatesAreVisible(int type) throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean storesLowerCaseIdentifiers() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean storesLowerCaseQuotedIdentifiers() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean storesMixedCaseIdentifiers() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean storesMixedCaseQuotedIdentifiers() throws SQLException {
+        return true;
+    }
+
+    @Override
+    public boolean storesUpperCaseIdentifiers() throws SQLException {
+        return true;
+    }
+
+    @Override
+    public boolean storesUpperCaseQuotedIdentifiers() throws SQLException {
+        return true;
+    }
+
+    @Override
+    public boolean supportsANSI92EntryLevelSQL() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean supportsANSI92FullSQL() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean supportsANSI92IntermediateSQL() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean supportsAlterTableWithAddColumn() throws SQLException {
+        return true;
+    }
+
+    @Override
+    public boolean supportsAlterTableWithDropColumn() throws SQLException {
+        return true;
+    }
+
+    @Override
+    public boolean supportsBatchUpdates() throws SQLException {
+        return false; // FIXME?
+    }
+
+    @Override
+    public boolean supportsCatalogsInDataManipulation() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean supportsCatalogsInIndexDefinitions() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean supportsCatalogsInPrivilegeDefinitions() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean supportsCatalogsInProcedureCalls() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean supportsCatalogsInTableDefinitions() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean supportsColumnAliasing() throws SQLException {
+        return true;
+    }
+
+    @Override
+    public boolean supportsConvert() throws SQLException {
+        return true;
+    }
+
+    @Override
+    public boolean supportsConvert(int fromType, int toType) throws SQLException {
+        // TODO
+        return false;
+    }
+
+    @Override
+    public boolean supportsCoreSQLGrammar() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean supportsCorrelatedSubqueries() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean supportsDataDefinitionAndDataManipulationTransactions() throws SQLException {
+        return true;
+    }
+
+    @Override
+    public boolean supportsDataManipulationTransactionsOnly() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean supportsDifferentTableCorrelationNames() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean supportsExpressionsInOrderBy() throws SQLException {
+        return true;
+    }
+
+    @Override
+    public boolean supportsExtendedSQLGrammar() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean supportsFullOuterJoins() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean supportsGetGeneratedKeys() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean supportsGroupBy() throws SQLException {
+        return true;
+    }
+
+    @Override
+    public boolean supportsGroupByBeyondSelect() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean supportsGroupByUnrelated() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean supportsIntegrityEnhancementFacility() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean supportsLikeEscapeClause() throws SQLException {
+        return true;
+    }
+
+    @Override
+    public boolean supportsLimitedOuterJoins() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean supportsMinimumSQLGrammar() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean supportsMixedCaseIdentifiers() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean supportsMixedCaseQuotedIdentifiers() throws SQLException {
+        return true;
+    }
+
+    @Override
+    public boolean supportsMultipleOpenResults() throws SQLException {
+        return true;
+    }
+
+    @Override
+    public boolean supportsMultipleResultSets() throws SQLException {
+        return true;
+    }
+
+    @Override
+    public boolean supportsMultipleTransactions() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean supportsNamedParameters() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean supportsNonNullableColumns() throws SQLException {
+        return true;
+    }
+
+    @Override
+    public boolean supportsOpenCursorsAcrossCommit() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean supportsOpenCursorsAcrossRollback() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean supportsOpenStatementsAcrossCommit() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean supportsOpenStatementsAcrossRollback() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean supportsOrderByUnrelated() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean supportsOuterJoins() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean supportsPositionedDelete() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean supportsPositionedUpdate() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean supportsResultSetConcurrency(int type, int concurrency) throws SQLException {
+        // TODO: review
+        return type ==  ResultSet.TYPE_FORWARD_ONLY && concurrency == Connection.TRANSACTION_READ_COMMITTED;
+    }
+
+    @Override
+    public boolean supportsResultSetHoldability(int holdability) throws SQLException {
+        // TODO
+        return holdability == connection.getHoldability();
+    }
+
+    @Override
+    public boolean supportsResultSetType(int type) throws SQLException {
+        return type == ResultSet.TYPE_FORWARD_ONLY;
+    }
+
+    @Override
+    public boolean supportsSavepoints() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean supportsSchemasInDataManipulation() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean supportsSchemasInIndexDefinitions() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean supportsSchemasInPrivilegeDefinitions() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean supportsSchemasInProcedureCalls() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean supportsSchemasInTableDefinitions() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean supportsSelectForUpdate() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean supportsStatementPooling() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean supportsStoredFunctionsUsingCallSyntax() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean supportsStoredProcedures() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean supportsSubqueriesInComparisons() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean supportsSubqueriesInExists() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean supportsSubqueriesInIns() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean supportsSubqueriesInQuantifieds() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean supportsTableCorrelationNames() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean supportsTransactionIsolationLevel(int level) throws SQLException {
+        return level == connection.getTransactionIsolation();
+    }
+
+    @Override
+    public boolean supportsTransactions() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean supportsUnion() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean supportsUnionAll() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean updatesAreDetected(int type) throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean usesLocalFilePerTable() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean usesLocalFiles() throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean isWrapperFor(Class<?> iface) throws SQLException {
+        return iface.isInstance(this);
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public <T> T unwrap(Class<T> iface) throws SQLException {
+        if (!iface.isInstance(this)) {
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode.CLASS_NOT_UNWRAPPABLE)
+                .setMessage(this.getClass().getName() + " not unwrappable from " + iface.getName())
+                .build().buildException();
+        }
+        return (T)this;
+    }
+
+    @Override
+    public ResultSet getPseudoColumns(String catalog, String schemaPattern, String tableNamePattern,
+            String columnNamePattern) throws SQLException {
+        return this.emptyResultSet;
+    }
+
+    @Override
+    public boolean generatedKeyAlwaysReturned() throws SQLException {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDriver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDriver.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDriver.java
new file mode 100644
index 0000000..590b29f
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDriver.java
@@ -0,0 +1,96 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.jdbc;
+
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.Properties;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.phoenix.query.*;
+import org.apache.phoenix.util.SQLCloseables;
+
+
+/**
+ * 
+ * JDBC Driver implementation of Phoenix for production.
+ * To use this driver, specify the following URL:
+ *     jdbc:phoenix:<zookeeper quorum server name>;
+ * Only an embedded driver is currently supported (Phoenix client
+ * runs in the same JVM as the driver). Connections are lightweight
+ * and are not pooled. The last part of the URL, the hbase zookeeper
+ * quorum server name, determines the hbase cluster to which queries
+ * will be routed.
+ * 
+ * @author jtaylor
+ * @since 0.1
+ */
+public final class PhoenixDriver extends PhoenixEmbeddedDriver {
+    public static final PhoenixDriver INSTANCE;
+    static {
+        try {
+            DriverManager.registerDriver( INSTANCE = new PhoenixDriver() );
+        } catch (SQLException e) {
+            throw new IllegalStateException("Untable to register " + PhoenixDriver.class.getName() + ": "+ e.getMessage());
+        }
+    }
+    private final ConcurrentMap<ConnectionInfo,ConnectionQueryServices> connectionQueryServicesMap = new ConcurrentHashMap<ConnectionInfo,ConnectionQueryServices>(3);
+
+    public PhoenixDriver() { // for Squirrel
+        // Use production services implementation
+        super(new QueryServicesImpl());
+    }
+
+    @Override
+    public boolean acceptsURL(String url) throws SQLException {
+        // Accept the url only if test=true attribute not set
+        return super.acceptsURL(url) && !isTestUrl(url);
+    }
+
+    @Override
+    protected ConnectionQueryServices getConnectionQueryServices(String url, Properties info) throws SQLException {
+        ConnectionInfo connInfo = ConnectionInfo.create(url);
+        ConnectionInfo normalizedConnInfo = connInfo.normalize(getQueryServices().getProps());
+        ConnectionQueryServices connectionQueryServices = connectionQueryServicesMap.get(normalizedConnInfo);
+        if (connectionQueryServices == null) {
+            if (normalizedConnInfo.isConnectionless()) {
+                connectionQueryServices = new ConnectionlessQueryServicesImpl(getQueryServices());
+            } else {
+                connectionQueryServices = new ConnectionQueryServicesImpl(getQueryServices(), normalizedConnInfo);
+            }
+            connectionQueryServices.init(url, info);
+            ConnectionQueryServices prevValue = connectionQueryServicesMap.putIfAbsent(normalizedConnInfo, connectionQueryServices);
+            if (prevValue != null) {
+                connectionQueryServices = prevValue;
+            }
+        }
+        return connectionQueryServices;
+    }
+
+    @Override
+    public void close() throws SQLException {
+        try {
+            SQLCloseables.closeAll(connectionQueryServicesMap.values());
+        } finally {
+            connectionQueryServicesMap.clear();            
+        }
+    }
+}


[04/51] [partial] Initial commit of master branch from github

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/site/language_reference_source/datatypes.html
----------------------------------------------------------------------
diff --git a/phoenix-core/src/site/language_reference_source/datatypes.html b/phoenix-core/src/site/language_reference_source/datatypes.html
new file mode 100644
index 0000000..9efca10
--- /dev/null
+++ b/phoenix-core/src/site/language_reference_source/datatypes.html
@@ -0,0 +1,493 @@
+<!DOCTYPE html PUBLIC "-//W3C//DTD XHTML 1.0 Strict//EN" "http://www.w3.org/TR/xhtml1/DTD/xhtml1-strict.dtd">
+<html xmlns="http://www.w3.org/1999/xhtml" lang="en" xml:lang="en">
+<head><meta http-equiv="Content-Type" content="text/html;charset=utf-8" /><title>
+Data Types
+</title><link rel="stylesheet" type="text/css" href="stylesheet.css" />
+<!-- [search] { -->
+</head>
+<body>
+
+<!-- } -->
+<h2 id="dataTypes">Index</h2>
+<!-- syntax-start
+<p class="notranslate">
+
+    <a href="#integer_type" >INTEGER Type</a><br />
+
+    <a href="#unsigned_int_type" >UNSIGNED_INT Type</a><br />
+
+    <a href="#bigint_type" >BIGINT Type</a><br />
+
+    <a href="#unsigned_long_type" >UNSIGNED_LONG Type</a><br />
+
+    <a href="#tinyint_type" >TINYINT Type</a><br />
+
+    <a href="#unsigned_tinyint_type" >UNSIGNED_TINYINT Type</a><br />
+
+    <a href="#smallint_type" >SMALLINT Type</a><br />
+
+    <a href="#unsigned_smallint_type" >UNSIGNED_SMALLINT Type</a><br />
+
+    <a href="#float_type" >FLOAT Type</a><br />
+
+    <a href="#unsigned_float_type" >UNSIGNED_FLOAT Type</a><br />
+
+    <a href="#double_type" >DOUBLE Type</a><br />
+
+    <a href="#unsigned_double_type" >UNSIGNED_DOUBLE Type</a><br />
+
+    <a href="#decimal_type" >DECIMAL Type</a><br />
+
+    <a href="#boolean_type" >BOOLEAN Type</a><br />
+
+    <a href="#time_type" >TIME Type</a><br />
+
+    <a href="#date_type" >DATE Type</a><br />
+
+    <a href="#timestamp_type" >TIMESTAMP Type</a><br />
+
+    <a href="#varchar_type" >VARCHAR Type</a><br />
+
+    <a href="#char_type" >CHAR Type</a><br />
+
+    <a href="#binary_type" >BINARY Type</a><br />
+
+    <a href="#varbinary_type" >VARBINARY Type</a><br />
+</p>
+syntax-end -->
+<!-- railroad-start -->
+<table class="notranslate index">
+    <tr>
+        <td class="index">
+            
+                <a href="#integer_type" >INTEGER Type</a><br />
+            
+                <a href="#unsigned_int_type" >UNSIGNED_INT Type</a><br />
+            
+                <a href="#bigint_type" >BIGINT Type</a><br />
+            
+                <a href="#unsigned_long_type" >UNSIGNED_LONG Type</a><br />
+            
+                <a href="#tinyint_type" >TINYINT Type</a><br />
+            
+                <a href="#unsigned_tinyint_type" >UNSIGNED_TINYINT Type</a><br />
+            
+                <a href="#smallint_type" >SMALLINT Type</a><br />
+                    </td><td class="index">
+            
+                <a href="#unsigned_smallint_type" >UNSIGNED_SMALLINT Type</a><br />
+            
+                <a href="#float_type" >FLOAT Type</a><br />
+            
+                <a href="#unsigned_float_type" >UNSIGNED_FLOAT Type</a><br />
+            
+                <a href="#double_type" >DOUBLE Type</a><br />
+            
+                <a href="#unsigned_double_type" >UNSIGNED_DOUBLE Type</a><br />
+            
+                <a href="#decimal_type" >DECIMAL Type</a><br />
+            
+                <a href="#boolean_type" >BOOLEAN Type</a><br />
+                    </td><td class="index">
+            
+                <a href="#time_type" >TIME Type</a><br />
+            
+                <a href="#date_type" >DATE Type</a><br />
+            
+                <a href="#timestamp_type" >TIMESTAMP Type</a><br />
+            
+                <a href="#varchar_type" >VARCHAR Type</a><br />
+            
+                <a href="#char_type" >CHAR Type</a><br />
+            
+                <a href="#binary_type" >BINARY Type</a><br />
+            
+                <a href="#varbinary_type" >VARBINARY Type</a><br />
+                    </td>
+    </tr>
+</table>
+<!-- railroad-end -->
+
+
+<h3 id="integer_type" class="notranslate">INTEGER Type</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+INTEGER
+</pre>
+<div name="railroad">
+<code class="c">INTEGER</code>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+INTEGER
+</pre>
+syntax-end -->
+<p>Possible values: -2147483648 to 2147483647.</p><p>Mapped to <code>java.lang.Integer</code>. The binary representation is a 4 byte integer with the sign bit flipped (so that negative values sorts before positive values).</p>
+<p>Example:</p>
+<p class="notranslate">INTEGER</p>
+
+<h3 id="unsigned_int_type" class="notranslate">UNSIGNED_INT Type</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+UNSIGNED_INT
+</pre>
+<div name="railroad">
+<code class="c">UNSIGNED_INT</code>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+UNSIGNED_INT
+</pre>
+syntax-end -->
+<p>Possible values: 0 to 2147483647. Mapped to <code>java.lang.Integer</code>. The binary representation is a 4 byte integer, matching the <code>HBase</code> Bytes.toBytes(int) method. The purpose of this type is to map to existing <code>HBase</code> data that was serialized using this <code>HBase</code> utility method. If that is not the case, use the regular signed type instead.</p>
+<p>Example:</p>
+<p class="notranslate">UNSIGNED_INT</p>
+
+<h3 id="bigint_type" class="notranslate">BIGINT Type</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+BIGINT
+</pre>
+<div name="railroad">
+<code class="c">BIGINT</code>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+BIGINT
+</pre>
+syntax-end -->
+<p>Possible values: -9223372036854775807 to 9223372036854775807. Mapped to <code>java.lang.Long</code>. The binary representation is an 8 byte long with the sign bit flipped (so that negative values sorts before positive values).</p>
+<p>Example:</p>
+<p class="notranslate">BIGINT</p>
+
+<h3 id="unsigned_long_type" class="notranslate">UNSIGNED_LONG Type</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+UNSIGNED_LONG
+</pre>
+<div name="railroad">
+<code class="c">UNSIGNED_LONG</code>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+UNSIGNED_LONG
+</pre>
+syntax-end -->
+<p>Possible values: 0 to 9223372036854775807. Mapped to <code>java.lang.Long</code>. The binary representation is an 8 byte integer, matching the <code>HBase</code> Bytes.toBytes(long) method. The purpose of this type is to map to existing <code>HBase</code> data that was serialized using this <code>HBase</code> utility method. If that is not the case, use the regular signed type instead.</p>
+<p>Example:</p>
+<p class="notranslate">UNSIGNED_LONG</p>
+
+<h3 id="tinyint_type" class="notranslate">TINYINT Type</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+TINYINT
+</pre>
+<div name="railroad">
+<code class="c">TINYINT</code>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+TINYINT
+</pre>
+syntax-end -->
+<p>Possible values: -128 to 127. Mapped to <code>java.lang.Byte</code>. The binary representation is a single byte, with the sign bit flipped (so that negative values sorts before positive values).</p>
+<p>Example:</p>
+<p class="notranslate">TINYINT</p>
+
+<h3 id="unsigned_tinyint_type" class="notranslate">UNSIGNED_TINYINT Type</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+UNSIGNED_TINYINT
+</pre>
+<div name="railroad">
+<code class="c">UNSIGNED_TINYINT</code>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+UNSIGNED_TINYINT
+</pre>
+syntax-end -->
+<p>Possible values: 0 to 127. Mapped to <code>java.lang.Byte</code>. The binary representation is a single byte, matching the <code>HBase</code> Bytes.toBytes(byte) method. The purpose of this type is to map to existing <code>HBase</code> data that was serialized using this <code>HBase</code> utility method. If that is not the case, use the regular signed type instead.</p>
+<p>Example:</p>
+<p class="notranslate">UNSIGNED_TINYINT</p>
+
+<h3 id="smallint_type" class="notranslate">SMALLINT Type</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+SMALLINT
+</pre>
+<div name="railroad">
+<code class="c">SMALLINT</code>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+SMALLINT
+</pre>
+syntax-end -->
+<p>Possible values: -32768 to 32767. Mapped to <code>java.lang.Short</code>. The binary representation is a 2 byte short with the sign bit flipped (so that negative values sort before positive values).</p>
+<p>Example:</p>
+<p class="notranslate">SMALLINT</p>
+
+<h3 id="unsigned_smallint_type" class="notranslate">UNSIGNED_SMALLINT Type</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+UNSIGNED_SMALLINT
+</pre>
+<div name="railroad">
+<code class="c">UNSIGNED_SMALLINT</code>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+UNSIGNED_SMALLINT
+</pre>
+syntax-end -->
+<p>Possible values: 0 to 32767. Mapped to <code>java.lang.Short</code>. The binary representation is an 2 byte integer, matching the <code>HBase</code> Bytes.toBytes(short) method. The purpose of this type is to map to existing <code>HBase</code> data that was serialized using this <code>HBase</code> utility method. If that is not the case, use the regular signed type instead.</p>
+<p>Example:</p>
+<p class="notranslate">UNSIGNED_SMALLINT</p>
+
+<h3 id="float_type" class="notranslate">FLOAT Type</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+FLOAT
+</pre>
+<div name="railroad">
+<code class="c">FLOAT</code>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+FLOAT
+</pre>
+syntax-end -->
+<p>Possible values: -3.402823466 E + 38 to 3.402823466 E + 38. Mapped to <code>java.lang.Float</code>. The binary representation is an 4 byte float with the sign bit flipped (so that negative values sort before positive values).</p>
+<p>Example:</p>
+<p class="notranslate">FLOAT</p>
+
+<h3 id="unsigned_float_type" class="notranslate">UNSIGNED_FLOAT Type</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+UNSIGNED_FLOAT
+</pre>
+<div name="railroad">
+<code class="c">UNSIGNED_FLOAT</code>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+UNSIGNED_FLOAT
+</pre>
+syntax-end -->
+<p>Possible values: 0 to 3.402823466 E + 38. Mapped to <code>java.lang.Float</code>. The binary representation is an 4 byte float matching the <code>HBase</code> Bytes.toBytes(float) method. The purpose of this type is to map to existing <code>HBase</code> data that was serialized using this <code>HBase</code> utility method. If that is not the case, use the regular signed type instead.</p>
+<p>Example:</p>
+<p class="notranslate">UNSIGNED_FLOAT</p>
+
+<h3 id="double_type" class="notranslate">DOUBLE Type</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+DOUBLE
+</pre>
+<div name="railroad">
+<code class="c">DOUBLE</code>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+DOUBLE
+</pre>
+syntax-end -->
+<p>Possible values: -1.7976931348623158 E + 308 to 1.7976931348623158 E + 308. Mapped to <code>java.lang.Double</code>. The binary representation is an 8 byte double with the sign bit flipped (so that negative values sort before positive value).</p>
+<p>Example:</p>
+<p class="notranslate">DOUBLE</p>
+
+<h3 id="unsigned_double_type" class="notranslate">UNSIGNED_DOUBLE Type</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+UNSIGNED_DOUBLE
+</pre>
+<div name="railroad">
+<code class="c">UNSIGNED_DOUBLE</code>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+UNSIGNED_DOUBLE
+</pre>
+syntax-end -->
+<p>Possible values: 0 to &nbsp;1.7976931348623158 E + 308. Mapped to <code>java.lang.Double</code>. The binary representation is an 8 byte double matching the <code>HBase</code> Bytes.toBytes(double) method. The purpose of this type is to map to existing <code>HBase</code> data that was serialized using this <code>HBase</code> utility method. If that is not the case, use the regular signed type instead.</p>
+<p>Example:</p>
+<p class="notranslate">UNSIGNED_DOUBLE</p>
+
+<h3 id="decimal_type" class="notranslate">DECIMAL Type</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+DECIMAL
+</pre>
+<div name="railroad">
+<code class="c">DECIMAL</code>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+DECIMAL
+</pre>
+syntax-end -->
+<p>Data type with fixed precision and scale. The maximum precision is 18 digits. Mapped to <code>java.math.BigDecimal</code>. The binary representation is binary comparable, variable length format. When used in a row key, it is terminated with a null byte unless it is the last column.</p>
+<p>Example:</p>
+<p class="notranslate">DECIMAL</p>
+
+<h3 id="boolean_type" class="notranslate">BOOLEAN Type</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+BOOLEAN
+</pre>
+<div name="railroad">
+<code class="c">BOOLEAN</code>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+BOOLEAN
+</pre>
+syntax-end -->
+<p>Possible values: <code>TRUE</code> and <code>FALSE</code>.</p><p>Mapped to <code>java.lang.Boolean</code>. The binary representation is a single byte with 0 for false and 1 for true</p>
+<p>Example:</p>
+<p class="notranslate">BOOLEAN</p>
+
+<h3 id="time_type" class="notranslate">TIME Type</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+TIME
+</pre>
+<div name="railroad">
+<code class="c">TIME</code>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+TIME
+</pre>
+syntax-end -->
+<p>The time data type. The format is yyyy-<code>MM</code>-dd hh:mm:ss, with both the date and time parts maintained. Mapped to <code>java.sql.Time</code>. The binary representation is an 8 byte long (the number of milliseconds from the epoch).</p>
+<p>Example:</p>
+<p class="notranslate">TIME</p>
+
+<h3 id="date_type" class="notranslate">DATE Type</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+DATE
+</pre>
+<div name="railroad">
+<code class="c">DATE</code>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+DATE
+</pre>
+syntax-end -->
+<p>The date data type. The format is yyyy-<code>MM</code>-dd hh:mm:ss, with both the date and time parts maintained to a millisecond accuracy. Mapped to <code>java.sql.Date</code>. The binary representation is an 8 byte long (the number of milliseconds from the epoch).</p>
+<p>Example:</p>
+<p class="notranslate">DATE</p>
+
+<h3 id="timestamp_type" class="notranslate">TIMESTAMP Type</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+TIMESTAMP
+</pre>
+<div name="railroad">
+<code class="c">TIMESTAMP</code>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+TIMESTAMP
+</pre>
+syntax-end -->
+<p>The timestamp data type. The format is yyyy-<code>MM</code>-dd hh:mm:ss[.nnnnnnnnn]. Mapped to <code>java.sql.Timestamp</code> with an internal representation of the number of nanos from the epoch. The binary representation is 12 bytes: an 8 byte long for the epoch time plus a 4 byte integer for the nanos.</p>
+<p>Example:</p>
+<p class="notranslate">TIMESTAMP</p>
+
+<h3 id="varchar_type" class="notranslate">VARCHAR Type</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+VARCHAR  [ ( <a href="index.html#int">precisionInt</a> ) ]
+</pre>
+<div name="railroad">
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c">VARCHAR</code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">( <a href="index.html#int">precisionInt</a> )</code></td></tr></table></td><td class="le"></td></tr></table></td></tr></table>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+VARCHAR  [ ( <a href="index.html#int">precisionInt</a> ) ]
+</pre>
+syntax-end -->
+<p>A variable length String with an optional max byte length. The binary representation is <code>UTF8</code> matching the <code>HBase</code> Bytes.toBytes(String) method. When used in a row key, it is terminated with a null byte unless it is the last column.</p><p>Mapped to <code>java.lang.String</code>.</p>
+<p>Example:</p>
+<p class="notranslate">VARCHAR<br />VARCHAR(255)</p>
+
+<h3 id="char_type" class="notranslate">CHAR Type</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+CHAR ( <a href="index.html#int">precisionInt</a> )
+</pre>
+<div name="railroad">
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c">CHAR ( <a href="index.html#int">precisionInt</a> )</code></td></tr></table>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+CHAR ( <a href="index.html#int">precisionInt</a> )
+</pre>
+syntax-end -->
+<p>A fixed length String with single-byte characters. The binary representation is <code>UTF8</code> matching the <code>HBase</code> Bytes.toBytes(String) method.</p><p>Mapped to <code>java.lang.String</code>.</p>
+<p>Example:</p>
+<p class="notranslate">CHAR(10)</p>
+
+<h3 id="binary_type" class="notranslate">BINARY Type</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+BINARY ( <a href="index.html#int">precisionInt</a> )
+</pre>
+<div name="railroad">
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c">BINARY ( <a href="index.html#int">precisionInt</a> )</code></td></tr></table>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+BINARY ( <a href="index.html#int">precisionInt</a> )
+</pre>
+syntax-end -->
+<p>Raw fixed length byte array.</p><p>Mapped to <code>byte[]</code>.</p>
+<p>Example:</p>
+<p class="notranslate">BINARY</p>
+
+<h3 id="varbinary_type" class="notranslate">VARBINARY Type</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+VARBINARY
+</pre>
+<div name="railroad">
+<code class="c">VARBINARY</code>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+VARBINARY
+</pre>
+syntax-end -->
+<p>Raw variable length byte array.</p><p>Mapped to <code>byte[]</code>.</p>
+<p>Example:</p>
+<p class="notranslate">VARBINARY</p>
+
+<!-- [close] { -->
+
+<!-- } --></body></html>
+

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/site/language_reference_source/functions.html
----------------------------------------------------------------------
diff --git a/phoenix-core/src/site/language_reference_source/functions.html b/phoenix-core/src/site/language_reference_source/functions.html
new file mode 100644
index 0000000..5305629
--- /dev/null
+++ b/phoenix-core/src/site/language_reference_source/functions.html
@@ -0,0 +1,740 @@
+<!DOCTYPE html PUBLIC "-//W3C//DTD XHTML 1.0 Strict//EN" "http://www.w3.org/TR/xhtml1/DTD/xhtml1-strict.dtd">
+<html xmlns="http://www.w3.org/1999/xhtml" lang="en" xml:lang="en">
+<head><meta http-equiv="Content-Type" content="text/html;charset=utf-8" /><title>
+Functions
+</title><link rel="stylesheet" type="text/css" href="stylesheet.css" />
+<!-- [search] { -->
+</head>
+<body>
+
+<!-- } -->
+<h3 id="functions">Aggregate Functions</h3>
+<!-- syntax-start
+<p class="notranslate">
+
+    <a href="#avg" >AVG</a><br />
+
+    <a href="#count" >COUNT</a><br />
+
+    <a href="#max" >MAX</a><br />
+
+    <a href="#min" >MIN</a><br />
+
+    <a href="#sum" >SUM</a><br />
+
+    <a href="#percentile_cont" >PERCENTILE_CONT</a><br />
+
+    <a href="#percentile_disc" >PERCENTILE_DISC</a><br />
+
+    <a href="#percent_rank" >PERCENT_RANK</a><br />
+
+    <a href="#stddev_pop" >STDDEV_POP</a><br />
+
+    <a href="#stddev_samp" >STDDEV_SAMP</a><br />
+</p>
+syntax-end -->
+<!-- railroad-start -->
+<table class="notranslate index">
+    <tr>
+        <td class="index">
+            
+                <a href="#avg" >AVG</a><br />
+            
+                <a href="#count" >COUNT</a><br />
+            
+                <a href="#max" >MAX</a><br />
+            
+                <a href="#min" >MIN</a><br />
+                    </td><td class="index">
+            
+                <a href="#sum" >SUM</a><br />
+            
+                <a href="#percentile_cont" >PERCENTILE_CONT</a><br />
+            
+                <a href="#percentile_disc" >PERCENTILE_DISC</a><br />
+            
+                <a href="#percent_rank" >PERCENT_RANK</a><br />
+                    </td><td class="index">
+            
+                <a href="#stddev_pop" >STDDEV_POP</a><br />
+            
+                <a href="#stddev_samp" >STDDEV_SAMP</a><br />
+                    </td>
+    </tr>
+</table>
+<!-- railroad-end -->
+
+<h3>String Functions</h3>
+<!-- syntax-start
+<p class="notranslate">
+
+    <a href="#substr" >SUBSTR</a><br />
+
+    <a href="#trim" >TRIM</a><br />
+
+    <a href="#ltrim" >LTRIM</a><br />
+
+    <a href="#rtrim" >RTRIM</a><br />
+
+    <a href="#length" >LENGTH</a><br />
+
+    <a href="#regexp_substr" >REGEXP_SUBSTR</a><br />
+
+    <a href="#regexp_replace" >REGEXP_REPLACE</a><br />
+
+    <a href="#upper" >UPPER</a><br />
+
+    <a href="#lower" >LOWER</a><br />
+
+    <a href="#reverse" >REVERSE</a><br />
+
+    <a href="#to_char" >TO_CHAR</a><br />
+</p>
+syntax-end -->
+<!-- railroad-start -->
+<table class="notranslate index">
+    <tr>
+        <td class="index">
+            
+                <a href="#substr" >SUBSTR</a><br />
+            
+                <a href="#trim" >TRIM</a><br />
+            
+                <a href="#ltrim" >LTRIM</a><br />
+            
+                <a href="#rtrim" >RTRIM</a><br />
+                    </td><td class="index">
+            
+                <a href="#length" >LENGTH</a><br />
+            
+                <a href="#regexp_substr" >REGEXP_SUBSTR</a><br />
+            
+                <a href="#regexp_replace" >REGEXP_REPLACE</a><br />
+            
+                <a href="#upper" >UPPER</a><br />
+                    </td><td class="index">
+            
+                <a href="#lower" >LOWER</a><br />
+            
+                <a href="#reverse" >REVERSE</a><br />
+            
+                <a href="#to_char" >TO_CHAR</a><br />
+                    </td>
+    </tr>
+</table>
+<!-- railroad-end -->
+
+<h3>Time and Date Functions</h3>
+<!-- syntax-start
+<p class="notranslate">
+
+    <a href="#round" >ROUND</a><br />
+
+    <a href="#truncate" >TRUNCATE</a><br />
+
+    <a href="#to_date" >TO_DATE</a><br />
+
+    <a href="#current_date" >CURRENT_DATE</a><br />
+
+    <a href="#current_time" >CURRENT_TIME</a><br />
+</p>
+syntax-end -->
+<!-- railroad-start -->
+<table class="notranslate index">
+    <tr>
+        <td class="index">
+            
+                <a href="#round" >ROUND</a><br />
+            
+                <a href="#truncate" >TRUNCATE</a><br />
+                    </td><td class="index">
+            
+                <a href="#to_date" >TO_DATE</a><br />
+            
+                <a href="#current_date" >CURRENT_DATE</a><br />
+                    </td><td class="index">
+            
+                <a href="#current_time" >CURRENT_TIME</a><br />
+                    </td>
+    </tr>
+</table>
+<!-- railroad-end -->
+
+<h3>Other Functions</h3>
+<!-- syntax-start
+<p class="notranslate">
+
+    <a href="#md5" >MD5</a><br />
+
+    <a href="#invert" >INVERT</a><br />
+
+    <a href="#to_number" >TO_NUMBER</a><br />
+
+    <a href="#coalesce" >COALESCE</a><br />
+</p>
+syntax-end -->
+<!-- railroad-start -->
+<table class="notranslate index">
+    <tr>
+        <td class="index">
+            
+                <a href="#md5" >MD5</a><br />
+            
+                <a href="#invert" >INVERT</a><br />
+                    </td><td class="index">
+            
+                <a href="#to_number" >TO_NUMBER</a><br />
+            
+                <a href="#coalesce" >COALESCE</a><br />
+                    </td><td class="index">
+                    </td>
+    </tr>
+</table>
+<!-- railroad-end -->
+
+
+
+<h3 id="avg" class="notranslate">AVG</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+AVG ( { <a href="index.html#term">numericTerm</a> } )
+</pre>
+<div name="railroad">
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c">AVG ( <a href="index.html#term">numericTerm</a> )</code></td></tr></table>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+AVG ( { <a href="index.html#term">numericTerm</a> } )
+</pre>
+syntax-end -->
+<p>The average (mean) value. If no rows are selected, the result is <code>NULL</code>. Aggregates are only allowed in select statements. The returned value is of the same data type as the parameter.</p>
+<p>Example:</p>
+<p class="notranslate">AVG(X)</p>
+
+<h3 id="count" class="notranslate">COUNT</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+COUNT( [ DISTINCT ] { * | { <a href="index.html#term">term</a> } } )
+</pre>
+<div name="railroad">
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c">COUNT (</code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c">DISTINCT</code></td><td class="le"></td></tr></table></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d"><code class="c">*</code></td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c"><a href="index.html#term">term</a></code></td><td class="le"></td></tr></table></td><td class="d"><code class="c">)</code></td></tr></table>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+COUNT( [ DISTINCT ] { * | { <a href="index.html#term">term</a> } } )
+</pre>
+syntax-end -->
+<p>The count of all row, or of the non-null values. This method returns a long. When <code>DISTINCT</code> is used, it counts only distinct values. If no rows are selected, the result is 0. Aggregates are only allowed in select statements.</p>
+<p>Example:</p>
+<p class="notranslate">COUNT(*)</p>
+
+<h3 id="max" class="notranslate">MAX</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+MAX(<a href="index.html#term">term</a>)
+</pre>
+<div name="railroad">
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c">MAX ( <a href="index.html#term">term</a> )</code></td></tr></table>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+MAX(<a href="index.html#term">term</a>)
+</pre>
+syntax-end -->
+<p>The highest value. If no rows are selected, the result is <code>NULL</code>. Aggregates are only allowed in select statements. The returned value is of the same data type as the parameter.</p>
+<p>Example:</p>
+<p class="notranslate">MAX(NAME)</p>
+
+<h3 id="min" class="notranslate">MIN</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+MIN(<a href="index.html#term">term</a>)
+</pre>
+<div name="railroad">
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c">MIN ( <a href="index.html#term">term</a> )</code></td></tr></table>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+MIN(<a href="index.html#term">term</a>)
+</pre>
+syntax-end -->
+<p>The lowest value. If no rows are selected, the result is <code>NULL</code>. Aggregates are only allowed in select statements. The returned value is of the same data type as the parameter.</p>
+<p>Example:</p>
+<p class="notranslate">MIN(NAME)</p>
+
+<h3 id="sum" class="notranslate">SUM</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+SUM( { <a href="index.html#term">numericTerm</a> } )
+</pre>
+<div name="railroad">
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c">SUM ( <a href="index.html#term">numericTerm</a> )</code></td></tr></table>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+SUM( { <a href="index.html#term">numericTerm</a> } )
+</pre>
+syntax-end -->
+<p>The sum of all values. If no rows are selected, the result is <code>NULL</code>. Aggregates are only allowed in select statements. The returned value is of the same data type as the parameter.</p>
+<p>Example:</p>
+<p class="notranslate">SUM(X)</p>
+
+<h3 id="percentile_cont" class="notranslate">PERCENTILE_CONT</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+PERCENTILE_CONT( { <a href="index.html#numeric">numeric</a> } ) WITHIN GROUP (ORDER BY { <a href="index.html#term">numericTerm</a> } { ASC | DESC } )
+</pre>
+<div name="railroad">
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c">PERCENTILE_CONT ( <a href="index.html#numeric">numeric</a> ) WITHIN GROUP ( ORDER BY <a href="index.html#term">numericTerm</a></code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d"><code class="c">ASC</code></td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c">DESC</code></td><td class="le"></td></tr></table></td><td class="d"><code class="c">)</code></td></tr></table>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+PERCENTILE_CONT( { <a href="index.html#numeric">numeric</a> } ) WITHIN GROUP (ORDER BY { <a href="index.html#term">numericTerm</a> } { ASC | DESC } )
+</pre>
+syntax-end -->
+<p>The nth percentile of values in the column. The percentile value can be between 0 and 1 inclusive. Aggregates are only allowed in select statements. The returned value is of decimal data type.</p>
+<p>Example:</p>
+<p class="notranslate">PERCENTILE_CONT( 0.9 ) WITHIN GROUP (ORDER BY X ASC)</p>
+
+<h3 id="percentile_disc" class="notranslate">PERCENTILE_DISC</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+PERCENTILE_DIST( { <a href="index.html#numeric">numeric</a> } ) WITHIN GROUP (ORDER BY { <a href="index.html#term">numericTerm</a> } { ASC | DESC } )
+</pre>
+<div name="railroad">
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c">PERCENTILE_DIST ( <a href="index.html#numeric">numeric</a> ) WITHIN GROUP ( ORDER BY <a href="index.html#term">numericTerm</a></code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d"><code class="c">ASC</code></td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c">DESC</code></td><td class="le"></td></tr></table></td><td class="d"><code class="c">)</code></td></tr></table>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+PERCENTILE_DIST( { <a href="index.html#numeric">numeric</a> } ) WITHIN GROUP (ORDER BY { <a href="index.html#term">numericTerm</a> } { ASC | DESC } )
+</pre>
+syntax-end -->
+<p><code>PERCENTILE_DISC</code> is an inverse distribution function that assumes a discrete distribution model. It takes a percentile value and a sort specification and returns an element from the set. Nulls are ignored in the calculation.</p>
+<p>Example:</p>
+<p class="notranslate">PERCENTILE_DISC( 0.9 ) WITHIN GROUP (ORDER BY X DESC)</p>
+
+<h3 id="percent_rank" class="notranslate">PERCENT_RANK</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+PERCENT_RANK( { <a href="index.html#numeric">numeric</a> } ) WITHIN GROUP (ORDER BY { <a href="index.html#term">numericTerm</a> } { ASC | DESC } )
+</pre>
+<div name="railroad">
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c">PERCENT_RANK ( <a href="index.html#numeric">numeric</a> ) WITHIN GROUP ( ORDER BY <a href="index.html#term">numericTerm</a></code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d"><code class="c">ASC</code></td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><code class="c">DESC</code></td><td class="le"></td></tr></table></td><td class="d"><code class="c">)</code></td></tr></table>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+PERCENT_RANK( { <a href="index.html#numeric">numeric</a> } ) WITHIN GROUP (ORDER BY { <a href="index.html#term">numericTerm</a> } { ASC | DESC } )
+</pre>
+syntax-end -->
+<p>The percentile rank for a hypothetical value, if inserted into the column. Aggregates are only allowed in select statements. The returned value is of decimal data type.</p>
+<p>Example:</p>
+<p class="notranslate">PERCENT_RANK( 100 ) WITHIN GROUP (ORDER BY X ASC)</p>
+
+<h3 id="stddev_pop" class="notranslate">STDDEV_POP</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+STDDEV_POP( { <a href="index.html#term">numericTerm</a> } )
+</pre>
+<div name="railroad">
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c">STDDEV_POP ( <a href="index.html#term">numericTerm</a> )</code></td></tr></table>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+STDDEV_POP( { <a href="index.html#term">numericTerm</a> } )
+</pre>
+syntax-end -->
+<p>The population standard deviation of all values. Aggregates are only allowed in select statements. The returned value is of decimal data type.</p>
+<p>Example:</p>
+<p class="notranslate">STDDEV_POP( X )</p>
+
+<h3 id="stddev_samp" class="notranslate">STDDEV_SAMP</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+STDDEV_SAMP( { <a href="index.html#term">numericTerm</a> } )
+</pre>
+<div name="railroad">
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c">STDDEV_SAMP ( <a href="index.html#term">numericTerm</a> )</code></td></tr></table>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+STDDEV_SAMP( { <a href="index.html#term">numericTerm</a> } )
+</pre>
+syntax-end -->
+<p>The sample standard deviation of all values. Aggregates are only allowed in select statements. The returned value is of decimal data type.</p>
+<p>Example:</p>
+<p class="notranslate">STDDEV_SAMP( X )</p>
+
+<h3 id="md5" class="notranslate">MD5</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+MD5( <a href="index.html#term">term</a> )
+</pre>
+<div name="railroad">
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c">MD5 ( <a href="index.html#term">term</a> )</code></td></tr></table>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+MD5( <a href="index.html#term">term</a> )
+</pre>
+syntax-end -->
+<p>Computes the <code>MD5</code> hash of the argument, returning the result as a <code>BINARY</code>(16).</p>
+<p>Example:</p>
+<p class="notranslate">MD5(my_column)</p>
+
+<h3 id="invert" class="notranslate">INVERT</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+INVERT( <a href="index.html#term">term</a> )
+</pre>
+<div name="railroad">
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c">INVERT ( <a href="index.html#term">term</a> )</code></td></tr></table>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+INVERT( <a href="index.html#term">term</a> )
+</pre>
+syntax-end -->
+<p>Inverts the bits of the argument. The return type will be the same as the argument.</p>
+<p>Example:</p>
+<p class="notranslate">INVERT(my_column)</p>
+
+<h3 id="to_number" class="notranslate">TO_NUMBER</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+TO_NUMBER( <a href="index.html#term">stringTerm</a> | <a href="index.html#term">timeStampTerm</a> [, <a href="index.html#string">formatString</a>] )
+</pre>
+<div name="railroad">
+<table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">TO_NUMBER ( <a href="index.html#term">stringTerm</a></code></td></tr></table></td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c"><a href="index.html#term">timeStampTerm</a></code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">, <a href="index.html#string">formatString</a></code></td></tr></table></td><td class="le"></td></tr></table></td><td class="d"><code class="c">)</code></td></tr></table></td><td class="le"></td></tr></table>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+TO_NUMBER( <a href="index.html#term">stringTerm</a> | <a href="index.html#term">timeStampTerm</a> [, <a href="index.html#string">formatString</a>] )
+</pre>
+syntax-end -->
+<p>Formats a string or date/time/timeStamp as a number, optionally accepting a format string. For details on the format, see <code>java.text.DecimalFormat</code>. For date, time, and timeStamp terms, the result is the time in milliseconds since the epoch. This method returns a decimal number.</p>
+<p>Example:</p>
+<p class="notranslate">TO_NUMBER(&#39;&#36;123.33&#39;, &#39;\u00A4###.##&#39;)</p>
+
+<h3 id="coalesce" class="notranslate">COALESCE</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+COALESCE( <a href="index.html#term">firstTerm</a>, <a href="index.html#term">secondTerm</a> )
+</pre>
+<div name="railroad">
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c">COALESCE ( <a href="index.html#term">firstTerm</a> , <a href="index.html#term">secondTerm</a> )</code></td></tr></table>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+COALESCE( <a href="index.html#term">firstTerm</a>, <a href="index.html#term">secondTerm</a> )
+</pre>
+syntax-end -->
+<p>Returns the value of the first argument if not null and the second argument otherwise. Useful to guarantee that a column in an <code>UPSERT SELECT</code> command will evaluate to a non null value.</p>
+<p>Example:</p>
+<p class="notranslate">COALESCE(last_update_date, CURRENT_DATE())</p>
+
+<h3 id="substr" class="notranslate">SUBSTR</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+SUBSTR( <a href="index.html#term">stringTerm</a>, <a href="index.html#int">startInt</a> [, <a href="index.html#int">lengthInt</a> ] )
+</pre>
+<div name="railroad">
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c">SUBSTR ( <a href="index.html#term">stringTerm</a> , <a href="index.html#int">startInt</a></code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">, <a href="index.html#int">lengthInt</a></code></td></tr></table></td><td class="le"></td></tr></table></td><td class="d"><code class="c">)</code></td></tr></table>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+SUBSTR( <a href="index.html#term">stringTerm</a>, <a href="index.html#int">startInt</a> [, <a href="index.html#int">lengthInt</a> ] )
+</pre>
+syntax-end -->
+<p>Returns a substring of a string starting at the one-based position. If zero is used, the position is zero-based. If the start index is negative, then the start index is relative to the end of the string. The length is optional and if not supplied, the rest of the string will be returned.</p>
+<p>Example:</p>
+<p class="notranslate">SUBSTR(&#39;[Hello]&#39;, 2, 5)<br />SUBSTR(&#39;Hello World&#39;, -5)</p>
+
+<h3 id="trim" class="notranslate">TRIM</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+TRIM( <a href="index.html#term">stringTerm</a> )
+</pre>
+<div name="railroad">
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c">TRIM ( <a href="index.html#term">stringTerm</a> )</code></td></tr></table>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+TRIM( <a href="index.html#term">stringTerm</a> )
+</pre>
+syntax-end -->
+<p>Removes leading and trailing spaces from the input string.</p>
+<p>Example:</p>
+<p class="notranslate">TRIM(&#39; &nbsp;Hello &nbsp;&#39;)</p>
+
+<h3 id="ltrim" class="notranslate">LTRIM</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+LTRIM( <a href="index.html#term">stringTerm</a> )
+</pre>
+<div name="railroad">
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c">LTRIM ( <a href="index.html#term">stringTerm</a> )</code></td></tr></table>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+LTRIM( <a href="index.html#term">stringTerm</a> )
+</pre>
+syntax-end -->
+<p>Removes leading spaces from the input string.</p>
+<p>Example:</p>
+<p class="notranslate">LTRIM(&#39; &nbsp;Hello&#39;)</p>
+
+<h3 id="rtrim" class="notranslate">RTRIM</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+RTRIM( <a href="index.html#term">stringTerm</a> )
+</pre>
+<div name="railroad">
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c">RTRIM ( <a href="index.html#term">stringTerm</a> )</code></td></tr></table>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+RTRIM( <a href="index.html#term">stringTerm</a> )
+</pre>
+syntax-end -->
+<p>Removes trailing spaces from the input string.</p>
+<p>Example:</p>
+<p class="notranslate">RTRIM(&#39;Hello &nbsp;&nbsp;&#39;)</p>
+
+<h3 id="length" class="notranslate">LENGTH</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+LENGTH( <a href="index.html#term">stringTerm</a> )
+</pre>
+<div name="railroad">
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c">LENGTH ( <a href="index.html#term">stringTerm</a> )</code></td></tr></table>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+LENGTH( <a href="index.html#term">stringTerm</a> )
+</pre>
+syntax-end -->
+<p>Returns the length of the string in characters.</p>
+<p>Example:</p>
+<p class="notranslate">LENGTH(&#39;Hello&#39;)</p>
+
+<h3 id="regexp_substr" class="notranslate">REGEXP_SUBSTR</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+REGEXP_SUBSTR( <a href="index.html#term">stringTerm</a>, <a href="index.html#string">patternString</a> [, <a href="index.html#int">startInt</a> ] )
+</pre>
+<div name="railroad">
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c">REGEXP_SUBSTR ( <a href="index.html#term">stringTerm</a> , <a href="index.html#string">patternString</a></code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">, <a href="index.html#int">startInt</a></code></td></tr></table></td><td class="le"></td></tr></table></td><td class="d"><code class="c">)</code></td></tr></table>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+REGEXP_SUBSTR( <a href="index.html#term">stringTerm</a>, <a href="index.html#string">patternString</a> [, <a href="index.html#int">startInt</a> ] )
+</pre>
+syntax-end -->
+<p>Returns a substring of a string by applying a regular expression start from the offset of a one-based position. Just like with <code>SUBSTR</code>, if the start index is negative, then it is relative to the end of the string. If not specified, the start index defaults to 1.</p>
+<p>Example:</p>
+<p class="notranslate">REGEXP_SUBSTR(&#39;na1-appsrv35-sj35&#39;, &#39;[^-]+&#39;) evaluates to &#39;na1&#39;</p>
+
+<h3 id="regexp_replace" class="notranslate">REGEXP_REPLACE</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+REGEXP_REPLACE( <a href="index.html#term">stringTerm</a>, <a href="index.html#string">patternString</a> [, <a href="index.html#string">replacementString</a> ] )
+</pre>
+<div name="railroad">
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c">REGEXP_REPLACE ( <a href="index.html#term">stringTerm</a> , <a href="index.html#string">patternString</a></code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">, <a href="index.html#string">replacementString</a></code></td></tr></table></td><td class="le"></td></tr></table></td><td class="d"><code class="c">)</code></td></tr></table>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+REGEXP_REPLACE( <a href="index.html#term">stringTerm</a>, <a href="index.html#string">patternString</a> [, <a href="index.html#string">replacementString</a> ] )
+</pre>
+syntax-end -->
+<p>Returns a string by applying a regular expression and replacing the matches with the replacement string. If the replacement string is not specified, it defaults to an empty string.</p>
+<p>Example:</p>
+<p class="notranslate">REGEXP_REPLACE(&#39;abc123ABC&#39;, &#39;[0-9]+&#39;, &#39;#&#39;) evaluates to &#39;abc#ABC&#39;</p>
+
+<h3 id="upper" class="notranslate">UPPER</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+UPPER( <a href="index.html#term">stringTerm</a> )
+</pre>
+<div name="railroad">
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c">UPPER ( <a href="index.html#term">stringTerm</a> )</code></td></tr></table>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+UPPER( <a href="index.html#term">stringTerm</a> )
+</pre>
+syntax-end -->
+<p>Returns upper case string of the string argument.</p>
+<p>Example:</p>
+<p class="notranslate">UPPER(&#39;Hello&#39;)</p>
+
+<h3 id="lower" class="notranslate">LOWER</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+LOWER( <a href="index.html#term">stringTerm</a> )
+</pre>
+<div name="railroad">
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c">LOWER ( <a href="index.html#term">stringTerm</a> )</code></td></tr></table>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+LOWER( <a href="index.html#term">stringTerm</a> )
+</pre>
+syntax-end -->
+<p>Returns lower case string of the string argument.</p>
+<p>Example:</p>
+<p class="notranslate">LOWER(&#39;HELLO&#39;)</p>
+
+<h3 id="reverse" class="notranslate">REVERSE</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+REVERSE( <a href="index.html#term">stringTerm</a> )
+</pre>
+<div name="railroad">
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c">REVERSE ( <a href="index.html#term">stringTerm</a> )</code></td></tr></table>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+REVERSE( <a href="index.html#term">stringTerm</a> )
+</pre>
+syntax-end -->
+<p>Returns reversed string of the string argument.</p>
+<p>Example:</p>
+<p class="notranslate">REVERSE(&#39;Hello&#39;)</p>
+
+<h3 id="to_char" class="notranslate">TO_CHAR</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+TO_CHAR( <a href="index.html#term">timestampTerm</a> | <a href="index.html#term">numberTerm</a> [, <a href="index.html#string">formatString</a>] )
+</pre>
+<div name="railroad">
+<table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">TO_CHAR ( <a href="index.html#term">timestampTerm</a></code></td></tr></table></td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c"><a href="index.html#term">numberTerm</a></code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">, <a href="index.html#string">formatString</a></code></td></tr></table></td><td class="le"></td></tr></table></td><td class="d"><code class="c">)</code></td></tr></table></td><td class="le"></td></tr></table>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+TO_CHAR( <a href="index.html#term">timestampTerm</a> | <a href="index.html#term">numberTerm</a> [, <a href="index.html#string">formatString</a>] )
+</pre>
+syntax-end -->
+<p>Formats a date, time, timestamp, or number as a string. The default date format is <code>yyyy-MM-dd HH:mm:ss</code> and the default number format is <code>#,##0.###</code>. For details, see <code>java.text.SimpleDateFormat</code> for date/time values and <code>java.text.DecimalFormat</code> for numbers. This method returns a string.</p>
+<p>Example:</p>
+<p class="notranslate">TO_CHAR(myDate, &#39;2001-02-03 04:05:06&#39;)<br />TO_CHAR(myDecimal, &#39;#,##0.###&#39;)</p>
+
+<h3 id="round" class="notranslate">ROUND</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+ROUND(<a href="index.html#term">timestampTerm</a>, {&apos;DAY&apos; | &apos;HOUR&apos; | &apos;MINUTE&apos; | &apos;SECOND&apos; | &apos;MILLISECOND&apos;} [, <a href="index.html#number">multiplierNumber</a>])
+</pre>
+<div name="railroad">
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c">ROUND ( <a href="index.html#term">timestampTerm</a> ,</code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">&apos; DAY &apos;</code></td></tr></table></td><td class="te"></td></tr><tr class="railroad"><td class="ks"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">&apos; HOUR &apos;</code></td></tr></table></td><td class="ke"></td></tr><tr class="railroad"><td class="ks"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">&apos; MINUTE &apos;</code></td></tr></table></td><td class="ke"></td></tr><tr class="railroad"><td class="ks"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">&apos; SECOND &apos;</code></td></tr></table></td><td class="ke"></td></tr><tr 
 class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">&apos; MILLISECOND &apos;</code></td></tr></table></td><td class="le"></td></tr></table></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">, <a href="index.html#number">multiplierNumber</a></code></td></tr></table></td><td class="le"></td></tr></table></td><td class="d"><code class="c">)</code></td></tr></table>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+ROUND(<a href="index.html#term">timestampTerm</a>, {&apos;DAY&apos; | &apos;HOUR&apos; | &apos;MINUTE&apos; | &apos;SECOND&apos; | &apos;MILLISECOND&apos;} [, <a href="index.html#number">multiplierNumber</a>])
+</pre>
+syntax-end -->
+<p>Rounds the timestamp to the nearest time unit specified. The multiplier is used to round to a multiple of a time unit (i.e. 10 minute) and defaults to 1 if not specified. This method returns a date.</p>
+<p>Example:</p>
+<p class="notranslate">ROUND(date, &#39;MINUTE&#39;, 30)<br />ROUND(time, &#39;HOUR&#39;)</p>
+
+<h3 id="truncate" class="notranslate">TRUNCATE</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+TRUNC(<a href="index.html#term">timestampTerm</a>, {&apos;DAY&apos; | &apos;HOUR&apos; | &apos;MINUTE&apos; | &apos;SECOND&apos; | &apos;MILLISECOND&apos;} [, <a href="index.html#int">multiplierInt</a>])
+</pre>
+<div name="railroad">
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c">TRUNC ( <a href="index.html#term">timestampTerm</a> ,</code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">&apos; DAY &apos;</code></td></tr></table></td><td class="te"></td></tr><tr class="railroad"><td class="ks"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">&apos; HOUR &apos;</code></td></tr></table></td><td class="ke"></td></tr><tr class="railroad"><td class="ks"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">&apos; MINUTE &apos;</code></td></tr></table></td><td class="ke"></td></tr><tr class="railroad"><td class="ks"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">&apos; SECOND &apos;</code></td></tr></table></td><td class="ke"></td></tr><tr 
 class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">&apos; MILLISECOND &apos;</code></td></tr></table></td><td class="le"></td></tr></table></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">, <a href="index.html#int">multiplierInt</a></code></td></tr></table></td><td class="le"></td></tr></table></td><td class="d"><code class="c">)</code></td></tr></table>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+TRUNC(<a href="index.html#term">timestampTerm</a>, {&apos;DAY&apos; | &apos;HOUR&apos; | &apos;MINUTE&apos; | &apos;SECOND&apos; | &apos;MILLISECOND&apos;} [, <a href="index.html#int">multiplierInt</a>])
+</pre>
+syntax-end -->
+<p>Truncates the timestamp to the next time unit closer to 0. The multiplier is used to truncate to a multiple of a time unit (i.e. 10 minute) and defaults to 1 if not specified. This method returns a date.</p>
+<p>Example:</p>
+<p class="notranslate">TRUNCATE(timestamp, &#39;SECOND&#39;, 30)<br />TRUNCATE(date, &#39;DAY&#39;, 7)</p>
+
+<h3 id="to_date" class="notranslate">TO_DATE</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+TO_DATE( <a href="index.html#term">stringTerm</a> [, <a href="index.html#string">formatString</a>] )
+</pre>
+<div name="railroad">
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c">TO_DATE ( <a href="index.html#term">stringTerm</a></code></td><td class="d"><table class="railroad"><tr class="railroad"><td class="ts"></td><td class="d">&nbsp;</td><td class="te"></td></tr><tr class="railroad"><td class="ls"></td><td class="d"><table class="railroad"><tr class="railroad"><td class="d"><code class="c">, <a href="index.html#string">formatString</a></code></td></tr></table></td><td class="le"></td></tr></table></td><td class="d"><code class="c">)</code></td></tr></table>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+TO_DATE( <a href="index.html#term">stringTerm</a> [, <a href="index.html#string">formatString</a>] )
+</pre>
+syntax-end -->
+<p>Parses a string and returns a date. The most important format characters are: y year, M month, d day, H hour, m minute, s second. The default format string is <code>yyyy-MM-dd HH:mm:ss</code>. For details of the format, see <code>java.text.SimpleDateFormat</code>.</p>
+<p>Example:</p>
+<p class="notranslate">TO_DATE(&#39;Sat, 3 Feb 2001 03:05:06 GMT&#39;, &#39;EEE, d MMM yyyy HH:mm:ss z&#39;)</p>
+
+<h3 id="current_date" class="notranslate">CURRENT_DATE</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+CURRENT_DATE()
+</pre>
+<div name="railroad">
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c">CURRENT_DATE ( )</code></td></tr></table>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+CURRENT_DATE()
+</pre>
+syntax-end -->
+<p>Returns the current server-side date, bound at the start of the execution of a query based on the current time on the region server owning the metadata of the table being queried.</p>
+<p>Example:</p>
+<p class="notranslate">CURRENT_DATE()</p>
+
+<h3 id="current_time" class="notranslate">CURRENT_TIME</h3>
+<!-- railroad-start -->
+<pre name="bnf" style="display: none">
+CURRENT_TIME()
+</pre>
+<div name="railroad">
+<table class="railroad"><tr class="railroad"><td class="d"><code class="c">CURRENT_TIME ( )</code></td></tr></table>
+</div>
+<!-- railroad-end -->
+<!-- syntax-start
+<pre>
+CURRENT_TIME()
+</pre>
+syntax-end -->
+<p>Same as <code>CURRENT_DATE</code>(), except returns a value of type <code>TIME</code>. In either case, the underlying representation is the epoch time as a long value.</p>
+<p>Example:</p>
+<p class="notranslate">CURRENT_TIME()</p>
+
+<!-- [close] { -->
+
+<!-- } --></body></html>
+


[41/51] [partial] Initial commit of master branch from github

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..92d8995
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
@@ -0,0 +1,472 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.compile;
+
+import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.phoenix.coprocessor.MetaDataProtocol.MetaDataMutationResult;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.parse.BindTableNode;
+import org.apache.phoenix.parse.ColumnDef;
+import org.apache.phoenix.parse.CreateTableStatement;
+import org.apache.phoenix.parse.DerivedTableNode;
+import org.apache.phoenix.parse.JoinTableNode;
+import org.apache.phoenix.parse.NamedTableNode;
+import org.apache.phoenix.parse.SelectStatement;
+import org.apache.phoenix.parse.SingleTableSQLStatement;
+import org.apache.phoenix.parse.TableName;
+import org.apache.phoenix.parse.TableNode;
+import org.apache.phoenix.parse.TableNodeVisitor;
+import org.apache.phoenix.query.ConnectionQueryServices;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.AmbiguousColumnException;
+import org.apache.phoenix.schema.AmbiguousTableException;
+import org.apache.phoenix.schema.ColumnFamilyNotFoundException;
+import org.apache.phoenix.schema.ColumnNotFoundException;
+import org.apache.phoenix.schema.ColumnRef;
+import org.apache.phoenix.schema.MetaDataClient;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PColumnFamily;
+import org.apache.phoenix.schema.PColumnFamilyImpl;
+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.PTableImpl;
+import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.schema.TableNotFoundException;
+import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.util.SchemaUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ListMultimap;
+import com.google.common.collect.Lists;
+import com.google.common.io.Closeables;
+
+/**
+ * Validates FROM clause and builds a ColumnResolver for resolving column references
+ * 
+ * @author jtaylor
+ * @since 0.1
+ */
+public class FromCompiler {
+    private static final Logger logger = LoggerFactory.getLogger(FromCompiler.class);
+
+    public static final ColumnResolver EMPTY_TABLE_RESOLVER = new ColumnResolver() {
+
+        @Override
+        public List<TableRef> getTables() {
+            return Collections.emptyList();
+        }
+
+        @Override
+        public ColumnRef resolveColumn(String schemaName, String tableName, String colName) throws SQLException {
+            throw new UnsupportedOperationException();
+        }
+    };
+
+    public static ColumnResolver getResolver(final CreateTableStatement statement, final PhoenixConnection connection)
+            throws SQLException {
+        TableName baseTable = statement.getBaseTableName();
+        if (baseTable == null) {
+            return EMPTY_TABLE_RESOLVER;
+        }
+        NamedTableNode tableNode = NamedTableNode.create(null, baseTable, Collections.<ColumnDef>emptyList());
+        // Always use non-tenant-specific connection here
+        try {
+            SingleTableColumnResolver visitor = new SingleTableColumnResolver(connection, tableNode, false, true);
+            return visitor;
+        } catch (TableNotFoundException e) {
+            // Used for mapped VIEW, since we won't be able to resolve that.
+            // Instead, we create a table with just the dynamic columns.
+            // A tenant-specific connection may not create a mapped VIEW.
+            if (connection.getTenantId() == null && statement.getTableType() == PTableType.VIEW) {
+                ConnectionQueryServices services = connection.getQueryServices();
+                byte[] fullTableName = SchemaUtil.getTableNameAsBytes(baseTable.getSchemaName(), baseTable.getTableName());
+                HTableInterface htable = null;
+                try {
+                    htable = services.getTable(fullTableName);
+                } catch (UnsupportedOperationException ignore) {
+                    throw e; // For Connectionless
+                } finally {
+                    if (htable != null) Closeables.closeQuietly(htable);
+                }
+                tableNode = NamedTableNode.create(null, baseTable, statement.getColumnDefs());
+                return new SingleTableColumnResolver(connection, tableNode, e.getTimeStamp());
+            }
+            throw e;
+        }
+    }
+
+    /**
+     * Iterate through the nodes in the FROM clause to build a column resolver used to lookup a column given the name
+     * and alias.
+     * 
+     * @param statement
+     *            the select statement
+     * @return the column resolver
+     * @throws SQLException
+     * @throws SQLFeatureNotSupportedException
+     *             if unsupported constructs appear in the FROM clause. Currently only a single table name is supported.
+     * @throws TableNotFoundException
+     *             if table name not found in schema
+     */
+    public static ColumnResolver getResolver(SelectStatement statement, PhoenixConnection connection)
+    		throws SQLException {
+    	List<TableNode> fromNodes = statement.getFrom();
+    	if (fromNodes.size() > 1) { throw new SQLFeatureNotSupportedException("Joins not supported"); }
+    	SingleTableColumnResolver visitor = new SingleTableColumnResolver(connection, (NamedTableNode)fromNodes.get(0), false, false);
+    	return visitor;
+    }
+    
+    public static ColumnResolver getMultiTableResolver(SelectStatement statement, PhoenixConnection connection)
+            throws SQLException {
+        List<TableNode> fromNodes = statement.getFrom();
+        MultiTableColumnResolver visitor = new MultiTableColumnResolver(connection);
+        for (TableNode node : fromNodes) {
+            node.accept(visitor);
+        }
+        return visitor;
+    }
+
+    public static ColumnResolver getResolver(NamedTableNode tableNode, PhoenixConnection connection) throws SQLException {
+        SingleTableColumnResolver visitor = new SingleTableColumnResolver(connection, tableNode, false, false);
+        return visitor;
+    }
+    
+    public static ColumnResolver getResolver(SingleTableSQLStatement statement, PhoenixConnection connection,
+            List<ColumnDef> dyn_columns) throws SQLException {
+        SingleTableColumnResolver visitor = new SingleTableColumnResolver(connection, statement.getTable(), true, false);
+        return visitor;
+    }
+
+    public static ColumnResolver getResolver(SingleTableSQLStatement statement, PhoenixConnection connection)
+            throws SQLException {
+        return getResolver(statement, connection, Collections.<ColumnDef>emptyList());
+    }
+
+    private static class SingleTableColumnResolver extends BaseColumnResolver {
+        	private final List<TableRef> tableRefs;
+        	private final String alias;
+    	
+       public SingleTableColumnResolver(PhoenixConnection connection, NamedTableNode table, long timeStamp) throws SQLException  {
+           super(connection);
+           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()));
+               }
+           }
+           Long scn = connection.getSCN();
+           PTable theTable = new PTableImpl(table.getName().getSchemaName(), table.getName().getTableName(), scn == null ? HConstants.LATEST_TIMESTAMP : scn, families);
+           theTable = this.addDynamicColumns(table.getDynamicColumns(), theTable);
+           alias = null;
+           tableRefs = ImmutableList.of(new TableRef(alias, theTable, timeStamp, !table.getDynamicColumns().isEmpty()));
+       }
+       
+        public SingleTableColumnResolver(PhoenixConnection connection, NamedTableNode table, boolean updateCacheOnlyIfAutoCommit, boolean allowMultiTenant) throws SQLException {
+            super(connection);
+            alias = table.getAlias();
+            TableName tableNameNode = table.getName();
+            String schemaName = tableNameNode.getSchemaName();
+            String tableName = tableNameNode.getTableName();
+            SQLException sqlE = null;
+            long timeStamp = QueryConstants.UNSET_TIMESTAMP;
+            TableRef tableRef = null;
+            boolean retry = true;
+            boolean didRetry = false;
+            MetaDataMutationResult result = null;
+            String fullTableName = SchemaUtil.getTableName(schemaName, tableName);
+            while (true) {
+                try {
+                    if (!updateCacheOnlyIfAutoCommit || connection.getAutoCommit()) {
+                        retry = false; // No reason to retry after this
+                        result = client.updateCache(schemaName, tableName);
+                        timeStamp = result.getMutationTime();
+                    }
+                    PTable theTable;
+                    try {
+                        theTable = connection.getPMetaData().getTable(fullTableName);
+                    } catch (TableNotFoundException e) {
+                        if (allowMultiTenant && result != null && result.getTable() != null) {
+                            theTable = result.getTable();
+                        } else {
+                            throw e;
+                        }
+                    }
+                    // If dynamic columns have been specified add them to the table declaration
+                    if (!table.getDynamicColumns().isEmpty()) {
+                        theTable = this.addDynamicColumns(table.getDynamicColumns(), theTable);
+                    }
+                    tableRef = new TableRef(alias, theTable, timeStamp, !table.getDynamicColumns().isEmpty());
+                    if (didRetry && logger.isDebugEnabled()) {
+                        logger.debug("Re-resolved stale table " + fullTableName + " with seqNum " + tableRef.getTable().getSequenceNumber() + " at timestamp " + tableRef.getTable().getTimeStamp() + " with " + tableRef.getTable().getColumns().size() + " columns: " + tableRef.getTable().getColumns());
+                    }
+                    break;
+                } catch (TableNotFoundException e) {
+                    sqlE = new TableNotFoundException(e,timeStamp);
+                }
+                // If we haven't already tried, update our cache and retry
+                // Only loop back if the cache was updated or we got a result back and we're allowed to multi-tenant tables
+                if (retry && ((result = client.updateCache(schemaName, tableName)).wasUpdated() || (result.getTable() != null && allowMultiTenant))) {
+                    timeStamp = result.getMutationTime();
+                    retry = false;
+                    didRetry = true;
+                    continue;
+                }
+                throw sqlE;
+            }
+            tableRefs = ImmutableList.of(tableRef);
+        }
+
+		@Override
+		public List<TableRef> getTables() {
+			return tableRefs;
+		}
+
+		@Override
+		public ColumnRef resolveColumn(String schemaName, String tableName,
+				String colName) throws SQLException {
+			TableRef tableRef = tableRefs.get(0);
+			boolean resolveCF = false;
+			if (schemaName != null || tableName != null) {
+			    String resolvedTableName = tableRef.getTable().getTableName().getString();
+			    String resolvedSchemaName = tableRef.getTable().getSchemaName().getString();
+			    if (schemaName != null && tableName != null) {
+                    if ( ! ( schemaName.equals(resolvedSchemaName)  &&
+                             tableName.equals(resolvedTableName) )) {
+                        if (!(resolveCF = schemaName.equals(alias))) {
+                            throw new ColumnNotFoundException(schemaName, tableName, null, colName);
+                        }
+                    }
+			    } else { // schemaName == null && tableName != null
+                    if (tableName != null && !tableName.equals(alias) && (!tableName.equals(resolvedTableName) || !resolvedSchemaName.equals(""))) {
+                        resolveCF = true;
+                   }
+			    }
+			    
+			}
+        	PColumn column = resolveCF
+        	        ? tableRef.getTable().getColumnFamily(tableName).getColumn(colName)
+        			: tableRef.getTable().getColumn(colName);
+            return new ColumnRef(tableRef, column.getPosition());
+		}
+
+    }
+
+    private static abstract class BaseColumnResolver implements ColumnResolver {
+        protected final PhoenixConnection connection;
+        protected final MetaDataClient client;
+        
+        private BaseColumnResolver(PhoenixConnection connection) {
+        	this.connection = connection;
+            this.client = new MetaDataClient(connection);
+        }
+
+        protected PTable addDynamicColumns(List<ColumnDef> dynColumns, PTable theTable)
+                throws SQLException {
+            if (!dynColumns.isEmpty()) {
+                List<PColumn> allcolumns = new ArrayList<PColumn>();
+                allcolumns.addAll(theTable.getColumns());
+                int position = allcolumns.size();
+                PName defaultFamilyName = PNameFactory.newName(SchemaUtil.getEmptyColumnFamily(theTable.getColumnFamilies()));
+                for (ColumnDef dynColumn : dynColumns) {
+                    PName familyName = defaultFamilyName;
+                    PName name = PNameFactory.newName(dynColumn.getColumnDefName().getColumnName());
+                    String family = dynColumn.getColumnDefName().getFamilyName();
+                    if (family != null) {
+                        theTable.getColumnFamily(family); // Verifies that column family exists
+                        familyName = PNameFactory.newName(family);
+                    }
+                    allcolumns.add(new PColumnImpl(name, familyName, dynColumn.getDataType(), dynColumn.getMaxLength(),
+                            dynColumn.getScale(), dynColumn.isNull(), position, dynColumn.getColumnModifier(), dynColumn.getArraySize()));
+                    position++;
+                }
+                theTable = PTableImpl.makePTable(theTable, allcolumns);
+            }
+            return theTable;
+        }
+    }
+    
+    // TODO: unused, but should be used for joins - make private once used
+    public static class MultiTableColumnResolver extends BaseColumnResolver implements TableNodeVisitor {
+        private final ListMultimap<String, TableRef> tableMap;
+        private final List<TableRef> tables;
+
+        private MultiTableColumnResolver(PhoenixConnection connection) {
+        	super(connection);
+            tableMap = ArrayListMultimap.<String, TableRef> create();
+            tables = Lists.newArrayList();
+        }
+
+        @Override
+        public List<TableRef> getTables() {
+            return tables;
+        }
+
+        @Override
+        public void visit(BindTableNode boundTableNode) throws SQLException {
+            throw new SQLFeatureNotSupportedException();
+        }
+
+        @Override
+        public void visit(JoinTableNode joinNode) throws SQLException {
+            joinNode.getTable().accept(this);
+        }
+
+        private TableRef createTableRef(String alias, String schemaName, String tableName,
+                List<ColumnDef> dynamicColumnDefs) throws SQLException {
+            MetaDataMutationResult result = client.updateCache(schemaName, tableName);
+            long timeStamp = result.getMutationTime();
+            PTable theTable = connection.getPMetaData().getTable(SchemaUtil.getTableName(schemaName, tableName));
+
+            // If dynamic columns have been specified add them to the table declaration
+            if (!dynamicColumnDefs.isEmpty()) {
+                theTable = this.addDynamicColumns(dynamicColumnDefs, theTable);
+            }
+            TableRef tableRef = new TableRef(alias, theTable, timeStamp, !dynamicColumnDefs.isEmpty());
+            return tableRef;
+        }
+
+
+        @Override
+        public void visit(NamedTableNode namedTableNode) throws SQLException {
+            String tableName = namedTableNode.getName().getTableName();
+            String schemaName = namedTableNode.getName().getSchemaName();
+
+            String alias = namedTableNode.getAlias();
+            List<ColumnDef> dynamicColumnDefs = namedTableNode.getDynamicColumns();
+
+            TableRef tableRef = createTableRef(alias, schemaName, tableName, dynamicColumnDefs);
+            PTable theTable = tableRef.getTable();
+
+            if (alias != null) {
+                tableMap.put(alias, tableRef);
+            }
+
+            String name = theTable.getName().getString();
+            //avoid having one name mapped to two identical TableRef.
+            if (alias == null || !alias.equals(name)) {
+            	tableMap.put(name, tableRef);
+            }
+            tables.add(tableRef);
+        }
+
+        @Override
+        public void visit(DerivedTableNode subselectNode) throws SQLException {
+            throw new SQLFeatureNotSupportedException();
+        }
+
+        private static class ColumnFamilyRef {
+            private final TableRef tableRef;
+            private final PColumnFamily family;
+
+            ColumnFamilyRef(TableRef tableRef, PColumnFamily family) {
+                this.tableRef = tableRef;
+                this.family = family;
+            }
+
+            public TableRef getTableRef() {
+                return tableRef;
+            }
+
+            public PColumnFamily getFamily() {
+                return family;
+            }
+        }
+
+        private TableRef resolveTable(String schemaName, String tableName) throws SQLException {
+            String fullTableName = SchemaUtil.getTableName(schemaName, tableName);
+            List<TableRef> tableRefs = tableMap.get(fullTableName);
+            if (tableRefs.size() == 0) {
+                throw new TableNotFoundException(fullTableName);
+            } else if (tableRefs.size() > 1) {
+                throw new AmbiguousTableException(tableName);
+            } else {
+                return tableRefs.get(0);
+            }
+        }
+
+        private ColumnFamilyRef resolveColumnFamily(String tableName, String cfName) throws SQLException {
+            if (tableName == null) {
+                ColumnFamilyRef theColumnFamilyRef = null;
+                Iterator<TableRef> iterator = tables.iterator();
+                while (iterator.hasNext()) {
+                    TableRef tableRef = iterator.next();
+                    try {
+                        PColumnFamily columnFamily = tableRef.getTable().getColumnFamily(cfName);
+                        if (theColumnFamilyRef != null) { throw new TableNotFoundException(cfName); }
+                        theColumnFamilyRef = new ColumnFamilyRef(tableRef, columnFamily);
+                    } catch (ColumnFamilyNotFoundException e) {}
+                }
+                if (theColumnFamilyRef != null) { return theColumnFamilyRef; }
+                throw new TableNotFoundException(cfName);
+            } else {
+                TableRef tableRef = resolveTable(null, tableName);
+                PColumnFamily columnFamily = tableRef.getTable().getColumnFamily(cfName);
+                return new ColumnFamilyRef(tableRef, columnFamily);
+            }
+        }
+
+        @Override
+        public ColumnRef resolveColumn(String schemaName, String tableName, String colName) throws SQLException {
+            if (tableName == null) {
+                int theColumnPosition = -1;
+                TableRef theTableRef = null;
+                Iterator<TableRef> iterator = tables.iterator();
+                while (iterator.hasNext()) {
+                    TableRef tableRef = iterator.next();
+                    try {
+                        PColumn column = tableRef.getTable().getColumn(colName);
+                        if (theTableRef != null) { throw new AmbiguousColumnException(colName); }
+                        theTableRef = tableRef;
+                        theColumnPosition = column.getPosition();
+                    } catch (ColumnNotFoundException e) {
+
+                    }
+                }
+                if (theTableRef != null) { return new ColumnRef(theTableRef, theColumnPosition); }
+                throw new ColumnNotFoundException(colName);
+            } else {
+                try {
+                    TableRef tableRef = resolveTable(schemaName, tableName);
+                    PColumn column = tableRef.getTable().getColumn(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);
+                    return new ColumnRef(cfRef.getTableRef(), column.getPosition());
+                }
+            }
+        }
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/compile/GroupByCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/GroupByCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/GroupByCompiler.java
new file mode 100644
index 0000000..e9b08f8
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/GroupByCompiler.java
@@ -0,0 +1,282 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.compile;
+
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+
+import org.apache.http.annotation.Immutable;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import org.apache.phoenix.compile.TrackOrderPreservingExpressionCompiler.Entry;
+import org.apache.phoenix.compile.TrackOrderPreservingExpressionCompiler.Ordering;
+import org.apache.phoenix.coprocessor.GroupedAggregateRegionObserver;
+import org.apache.phoenix.coprocessor.UngroupedAggregateRegionObserver;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+import org.apache.phoenix.expression.CoerceExpression;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.parse.AliasedNode;
+import org.apache.phoenix.parse.ParseNode;
+import org.apache.phoenix.parse.SelectStatement;
+import org.apache.phoenix.schema.AmbiguousColumnException;
+import org.apache.phoenix.schema.ColumnNotFoundException;
+import org.apache.phoenix.schema.PDataType;
+
+
+/**
+ * 
+ * Validates GROUP BY clause and builds a {@link GroupBy} instance to encapsulate the
+ * group by expressions.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class GroupByCompiler {
+    @Immutable
+    public static class GroupBy {
+        private final List<Expression> expressions;
+        private final List<Expression> keyExpressions;
+        private final String scanAttribName;
+        public static final GroupByCompiler.GroupBy EMPTY_GROUP_BY = new GroupBy(new GroupByBuilder());
+        
+        private GroupBy(GroupByBuilder builder) {
+            this.expressions = ImmutableList.copyOf(builder.expressions);
+            this.keyExpressions = ImmutableList.copyOf(builder.keyExpressions);
+            this.scanAttribName = builder.scanAttribName;
+            assert(expressions.size() == keyExpressions.size());
+        }
+        
+        public List<Expression> getExpressions() {
+            return expressions;
+        }
+        
+        public List<Expression> getKeyExpressions() {
+            return keyExpressions;
+        }
+        
+        public String getScanAttribName() {
+            return scanAttribName;
+        }
+        
+        public boolean isEmpty() {
+            return expressions.isEmpty();
+        }
+        
+        public static class GroupByBuilder {
+            private String scanAttribName;
+            private List<Expression> expressions = Collections.emptyList();
+            private List<Expression> keyExpressions = Collections.emptyList();
+
+            public GroupByBuilder() {
+            }
+            
+            public GroupByBuilder setScanAttribName(String scanAttribName) {
+                this.scanAttribName = scanAttribName;
+                return this;
+            }
+            
+            public GroupByBuilder setExpressions(List<Expression> expressions) {
+                this.expressions = expressions;
+                return this;
+            }
+            
+            public GroupByBuilder setKeyExpressions(List<Expression> keyExpressions) {
+                this.keyExpressions = keyExpressions;
+                return this;
+            }
+            
+            public GroupBy build() {
+                return new GroupBy(this);
+            }
+        }
+
+        public boolean isOrderPreserving() {
+            return !GroupedAggregateRegionObserver.UNORDERED_GROUP_BY_EXPRESSIONS.equals(scanAttribName);
+        }
+        
+        public void explain(List<String> planSteps) {
+            if (scanAttribName != null) {
+                if (UngroupedAggregateRegionObserver.UNGROUPED_AGG.equals(scanAttribName)) {
+                    planSteps.add("    SERVER AGGREGATE INTO SINGLE ROW");
+                } else if (GroupedAggregateRegionObserver.UNORDERED_GROUP_BY_EXPRESSIONS.equals(scanAttribName)) {
+                    planSteps.add("    SERVER AGGREGATE INTO DISTINCT ROWS BY " + getExpressions());                    
+                } else {
+                    planSteps.add("    SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY " + getExpressions());                    
+                }
+            }
+        }
+    }
+
+    /**
+     * Get list of columns in the GROUP BY clause.
+     * @param context query context kept between compilation of different query clauses
+     * @param statement SQL statement being compiled
+     * @return the {@link GroupBy} instance encapsulating the group by clause
+     * @throws ColumnNotFoundException if column name could not be resolved
+     * @throws AmbiguousColumnException if an unaliased column name is ambiguous across multiple tables
+     */
+    public static GroupBy compile(StatementContext context, SelectStatement statement) throws SQLException {
+        List<ParseNode> groupByNodes = statement.getGroupBy();
+        /**
+         * Distinct can use an aggregate plan if there's no group by.
+         * Otherwise, we need to insert a step after the Merge that dedups.
+         * Order by only allowed on columns in the select distinct
+         */
+        if (groupByNodes.isEmpty()) {
+            if (statement.isAggregate()) {
+                return new GroupBy.GroupByBuilder().setScanAttribName(UngroupedAggregateRegionObserver.UNGROUPED_AGG).build();
+            }
+            if (!statement.isDistinct()) {
+                return GroupBy.EMPTY_GROUP_BY;
+            }
+            
+            groupByNodes = Lists.newArrayListWithExpectedSize(statement.getSelect().size());
+            for (AliasedNode aliasedNode : statement.getSelect()) {
+                groupByNodes.add(aliasedNode.getNode());
+            }
+        }
+
+       // Accumulate expressions in GROUP BY
+        TrackOrderPreservingExpressionCompiler groupByVisitor =
+                new TrackOrderPreservingExpressionCompiler(context, 
+                        GroupBy.EMPTY_GROUP_BY, groupByNodes.size(), 
+                        Ordering.UNORDERED);
+        for (ParseNode node : groupByNodes) {
+            Expression expression = node.accept(groupByVisitor);
+            if (groupByVisitor.isAggregate()) {
+                throw new SQLExceptionInfo.Builder(SQLExceptionCode.AGGREGATE_IN_GROUP_BY)
+                    .setMessage(expression.toString()).build().buildException();
+            }
+            if (!expression.isStateless()) {
+                groupByVisitor.addEntry(expression);
+            }
+            groupByVisitor.reset();
+        }
+        
+        List<Entry> groupByEntries = groupByVisitor.getEntries();
+        if (groupByEntries.isEmpty()) {
+            return GroupBy.EMPTY_GROUP_BY;
+        }
+        
+        boolean isRowKeyOrderedGrouping = groupByVisitor.isOrderPreserving();
+        List<Expression> expressions = Lists.newArrayListWithCapacity(groupByEntries.size());
+        List<Expression> keyExpressions = expressions;
+        String groupExprAttribName;
+        // This is true if the GROUP BY is composed of only PK columns. We further check here that
+        // there are no "gaps" in the PK columns positions used (i.e. we start with the first PK
+        // column and use each subsequent one in PK order).
+        if (isRowKeyOrderedGrouping) {
+            groupExprAttribName = GroupedAggregateRegionObserver.KEY_ORDERED_GROUP_BY_EXPRESSIONS;
+            for (Entry groupByEntry : groupByEntries) {
+                expressions.add(groupByEntry.getExpression());
+            }
+        } else {
+            /*
+             * Otherwise, our coprocessor needs to collect all distinct groups within a region, sort them, and
+             * hold on to them until the scan completes.
+             */
+            groupExprAttribName = GroupedAggregateRegionObserver.UNORDERED_GROUP_BY_EXPRESSIONS;
+            /*
+             * Put fixed length nullables at the end, so that we can represent null by the absence of the trailing
+             * value in the group by key. If there is more than one, we'll need to convert the ones not at the end
+             * into a Decimal so that we can use an empty byte array as our representation for null (which correctly
+             * maintains the sort order). We convert the Decimal back to the appropriate type (Integer or Long) when
+             * it's retrieved from the result set.
+             * 
+             * More specifically, order into the following buckets:
+             *   1) non nullable fixed width
+             *   2) variable width
+             *   3) nullable fixed width
+             * Within each bucket, order based on the column position in the schema. Putting the fixed width values
+             * in the beginning optimizes access to subsequent values.
+             */
+            Collections.sort(groupByEntries, new Comparator<Entry>() {
+                @Override
+                public int compare(Entry o1, Entry o2) {
+                    Expression e1 = o1.getExpression();
+                    Expression e2 = o2.getExpression();
+                    boolean isFixed1 = e1.getDataType().isFixedWidth();
+                    boolean isFixed2 = e2.getDataType().isFixedWidth();
+                    boolean isFixedNullable1 = e1.isNullable() &&isFixed1;
+                    boolean isFixedNullable2 = e2.isNullable() && isFixed2;
+                    if (isFixedNullable1 == isFixedNullable2) {
+                        if (isFixed1 == isFixed2) {
+                            // Not strictly necessary, but forces the order to match the schema
+                            // column order (with PK columns before value columns).
+                            return o1.getColumnPosition() - o2.getColumnPosition();
+                        } else if (isFixed1) {
+                            return -1;
+                        } else {
+                            return 1;
+                        }
+                    } else if (isFixedNullable1) {
+                        return 1;
+                    } else {
+                        return -1;
+                    }
+                }
+            });
+            for (Entry groupByEntry : groupByEntries) {
+                expressions.add(groupByEntry.getExpression());
+            }
+            for (int i = expressions.size()-2; i >= 0; i--) {
+                Expression expression = expressions.get(i);
+                PDataType keyType = getKeyType(expression);
+                if (keyType == expression.getDataType()) {
+                    continue;
+                }
+                // Copy expressions only when keyExpressions will be different than expressions
+                if (keyExpressions == expressions) {
+                    keyExpressions = new ArrayList<Expression>(expressions);
+                }
+                // Wrap expression in an expression that coerces the expression to the required type..
+                // This is done so that we have a way of expressing null as an empty key when more
+                // than one fixed and nullable types are used in a group by clause
+                keyExpressions.set(i, CoerceExpression.create(expression, keyType));
+            }
+        }
+
+        // Set attribute with serialized expressions for coprocessor
+        // FIXME: what if group by is empty (i.e. only literals)?
+        GroupedAggregateRegionObserver.serializeIntoScan(context.getScan(), groupExprAttribName, keyExpressions);
+        GroupBy groupBy = new GroupBy.GroupByBuilder().setScanAttribName(groupExprAttribName).setExpressions(expressions).setKeyExpressions(keyExpressions).build();
+        return groupBy;
+    }
+    
+    private static PDataType getKeyType(Expression expression) {
+        PDataType type = expression.getDataType();
+        if (!expression.isNullable() || !type.isFixedWidth()) {
+            return type;
+        }
+        if (type.isCoercibleTo(PDataType.DECIMAL)) {
+            return PDataType.DECIMAL;
+        }
+        // Should never happen
+        throw new IllegalStateException("Multiple occurrences of type " + type + " may not occur in a GROUP BY clause");
+    }
+    
+    private GroupByCompiler() {
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/compile/HavingCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/HavingCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/HavingCompiler.java
new file mode 100644
index 0000000..857d48a
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/HavingCompiler.java
@@ -0,0 +1,241 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.compile;
+
+import java.sql.SQLException;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.phoenix.compile.GroupByCompiler.GroupBy;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.LiteralExpression;
+import org.apache.phoenix.parse.AddParseNode;
+import org.apache.phoenix.parse.AndParseNode;
+import org.apache.phoenix.parse.BetweenParseNode;
+import org.apache.phoenix.parse.CaseParseNode;
+import org.apache.phoenix.parse.ColumnParseNode;
+import org.apache.phoenix.parse.ComparisonParseNode;
+import org.apache.phoenix.parse.DivideParseNode;
+import org.apache.phoenix.parse.FunctionParseNode;
+import org.apache.phoenix.parse.IsNullParseNode;
+import org.apache.phoenix.parse.MultiplyParseNode;
+import org.apache.phoenix.parse.OrParseNode;
+import org.apache.phoenix.parse.ParseNode;
+import org.apache.phoenix.parse.SelectStatement;
+import org.apache.phoenix.parse.SelectStatementRewriter;
+import org.apache.phoenix.parse.SubtractParseNode;
+import org.apache.phoenix.parse.TraverseNoParseNodeVisitor;
+import org.apache.phoenix.schema.ColumnRef;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.TypeMismatchException;
+
+
+public class HavingCompiler {
+
+    private HavingCompiler() {
+    }
+
+    public static Expression compile(StatementContext context, SelectStatement statement, GroupBy groupBy) throws SQLException {
+        ParseNode having = statement.getHaving();
+        if (having == null) {
+            return null;
+        }
+        ExpressionCompiler expressionBuilder = new ExpressionCompiler(context, groupBy);
+        Expression expression = having.accept(expressionBuilder);
+        if (expression.getDataType() != PDataType.BOOLEAN) {
+            throw TypeMismatchException.newException(PDataType.BOOLEAN, expression.getDataType(), expression.toString());
+        }
+        if (LiteralExpression.isFalse(expression)) {
+            context.setScanRanges(ScanRanges.NOTHING);
+            return null;
+        } else if (LiteralExpression.isTrue(expression)) {
+            return null;
+        }
+        if (!expressionBuilder.isAggregate()) {
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode.ONLY_AGGREGATE_IN_HAVING_CLAUSE).build().buildException();
+        }
+        return expression;
+    }
+
+    public static SelectStatement rewrite(StatementContext context, SelectStatement statement, GroupBy groupBy) throws SQLException {
+        ParseNode having = statement.getHaving();
+        if (having == null) {
+            return statement;
+        }
+        HavingClauseVisitor visitor = new HavingClauseVisitor(context, groupBy);
+        having.accept(visitor);
+        statement = SelectStatementRewriter.moveFromHavingToWhereClause(statement, visitor.getMoveToWhereClauseExpressions());
+        return statement;
+    }
+
+    /**
+     * 
+     * Visitor that figures out if an expression can be moved from the HAVING clause to
+     * the WHERE clause, since it's more optimal to pre-filter instead of post-filter.
+     * 
+     * The visitor traverses through AND expressions only and into comparison expresssions.
+     * If a comparison expression uses a GROUP BY column and does not use any aggregate
+     * functions, then it's moved. For example, these HAVING expressions would be moved:
+     * 
+     * select count(1) from atable group by a_string having a_string = 'foo'
+     * select count(1) from atable group by a_date having round(a_date,'hour') > ?
+     * select count(1) from atable group by a_date,a_string having a_date > ? or a_string = 'a'
+     * select count(1) from atable group by a_string,b_string having a_string = 'a' and b_string = 'b'
+     * 
+     * while these would not be moved:
+     * 
+     * select count(1) from atable having min(a_integer) < 5
+     * select count(1) from atable group by a_string having count(a_string) >= 1
+     * select count(1) from atable group by a_date,a_string having a_date > ? or min(a_string) = 'a'
+     * select count(1) from atable group by a_date having round(min(a_date),'hour') < ?
+     *
+     * @author jtaylor
+     * @since 0.1
+     */
+    private static class HavingClauseVisitor extends TraverseNoParseNodeVisitor<Void> {
+        private ParseNode topNode = null;
+        private boolean hasNoAggregateFunctions = true;
+        private Boolean hasOnlyAggregateColumns;
+        private final StatementContext context;
+        private final GroupBy groupBy;
+        private final Set<ParseNode> moveToWhereClause = new LinkedHashSet<ParseNode>();
+        
+        HavingClauseVisitor(StatementContext context, GroupBy groupBy) {
+            this.context = context;
+            this.groupBy = groupBy;
+        }
+        
+        public Set<ParseNode> getMoveToWhereClauseExpressions() {
+            return moveToWhereClause;
+        }
+        
+        @Override
+        public boolean visitEnter(AndParseNode node) throws SQLException {
+            return true;
+        }
+        
+        @Override
+        public boolean visitEnter(OrParseNode node) throws SQLException {
+            enterBooleanNode(node);
+            return true;
+        }
+        
+        @Override
+        public boolean visitEnter(ComparisonParseNode node) throws SQLException {
+            enterBooleanNode(node);
+            return true;
+        }
+        
+        @Override
+        public boolean visitEnter(IsNullParseNode node) throws SQLException {
+            enterBooleanNode(node);
+            return true;
+        }
+        
+        private void enterBooleanNode(ParseNode node) {
+            if (topNode == null) {
+                topNode = node;
+            }
+        }
+        
+        private void leaveBooleanNode(ParseNode node) {
+            if (topNode == node) {
+                if ( hasNoAggregateFunctions && !Boolean.FALSE.equals(hasOnlyAggregateColumns)) {
+                    moveToWhereClause.add(node);
+                }
+                hasNoAggregateFunctions = true;
+                hasOnlyAggregateColumns = null;
+                topNode = null;
+            }
+        }
+
+        @Override
+        public Void visitLeave(OrParseNode node, List<Void> l) throws SQLException {
+            leaveBooleanNode(node);
+            return null;
+        }
+
+        @Override
+        public Void visitLeave(ComparisonParseNode node, List<Void> l) throws SQLException {
+            leaveBooleanNode(node);
+            return null;
+        }
+
+        @Override
+        public Void visitLeave(IsNullParseNode node, List<Void> l) throws SQLException {
+            leaveBooleanNode(node);
+            return null;
+        }
+
+        @Override
+        public boolean visitEnter(FunctionParseNode node) throws SQLException {
+            boolean isAggregate = node.isAggregate();
+            this.hasNoAggregateFunctions = this.hasNoAggregateFunctions && !isAggregate;
+            return !isAggregate;
+        }
+
+        @Override
+        public boolean visitEnter(CaseParseNode node) throws SQLException {
+            return true;
+        }
+
+        @Override
+        public Void visit(ColumnParseNode node) throws SQLException {
+            ColumnRef ref = context.getResolver().resolveColumn(node.getSchemaName(), node.getTableName(), node.getName());
+            boolean isAggregateColumn = groupBy.getExpressions().indexOf(ref.newColumnExpression()) >= 0;
+            if (hasOnlyAggregateColumns == null) {
+                hasOnlyAggregateColumns = isAggregateColumn;
+            } else {
+                hasOnlyAggregateColumns &= isAggregateColumn;
+            }
+            
+            return null;
+        }
+
+        @Override
+        public boolean visitEnter(SubtractParseNode node) throws SQLException {
+            return true;
+        }
+
+        @Override
+        public boolean visitEnter(AddParseNode node) throws SQLException {
+            return true;
+        }
+
+        @Override
+        public boolean visitEnter(MultiplyParseNode node) throws SQLException {
+            return true;
+        }
+
+        @Override
+        public boolean visitEnter(DivideParseNode node) throws SQLException {
+            return true;
+        }
+
+        @Override
+        public boolean visitEnter(BetweenParseNode node) throws SQLException {
+            return true;
+        }
+		
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/compile/IndexStatementRewriter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/IndexStatementRewriter.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/IndexStatementRewriter.java
new file mode 100644
index 0000000..84ac0a1
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/IndexStatementRewriter.java
@@ -0,0 +1,103 @@
+package org.apache.phoenix.compile;
+
+import java.sql.SQLException;
+import java.util.Map;
+
+import org.apache.phoenix.parse.ColumnParseNode;
+import org.apache.phoenix.parse.FamilyWildcardParseNode;
+import org.apache.phoenix.parse.ParseNode;
+import org.apache.phoenix.parse.ParseNodeFactory;
+import org.apache.phoenix.parse.ParseNodeRewriter;
+import org.apache.phoenix.parse.SelectStatement;
+import org.apache.phoenix.parse.TableName;
+import org.apache.phoenix.parse.WildcardParseNode;
+import org.apache.phoenix.schema.ColumnRef;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.util.IndexUtil;
+
+public class IndexStatementRewriter extends ParseNodeRewriter {
+    private static final ParseNodeFactory FACTORY = new ParseNodeFactory();
+    
+    private Map<TableRef, TableRef> multiTableRewriteMap;
+    
+    public IndexStatementRewriter(ColumnResolver dataResolver, Map<TableRef, TableRef> multiTableRewriteMap) {
+        super(dataResolver);
+        this.multiTableRewriteMap = multiTableRewriteMap;
+    }
+    
+    /**
+     * Rewrite the select statement by translating all data table column references to
+     * references to the corresponding index column.
+     * @param statement the select statement
+     * @param dataResolver the column resolver
+     * @return new select statement or the same one if nothing was rewritten.
+     * @throws SQLException 
+     */
+    public static SelectStatement translate(SelectStatement statement, ColumnResolver dataResolver) throws SQLException {
+        return translate(statement, dataResolver, null);
+    }
+    
+    /**
+     * Rewrite the select statement containing multiple tables by translating all 
+     * data table column references to references to the corresponding index column.
+     * @param statement the select statement
+     * @param dataResolver the column resolver
+     * @param multiTableRewriteMap the data table to index table map
+     * @return new select statement or the same one if nothing was rewritten.
+     * @throws SQLException 
+     */
+    public static SelectStatement translate(SelectStatement statement, ColumnResolver dataResolver, Map<TableRef, TableRef> multiTableRewriteMap) throws SQLException {
+        return rewrite(statement, new IndexStatementRewriter(dataResolver, multiTableRewriteMap));
+    }
+
+    @Override
+    public ParseNode visit(ColumnParseNode node) throws SQLException {
+        ColumnRef dataColRef = getResolver().resolveColumn(node.getSchemaName(), node.getTableName(), node.getName());
+        TableName tName = null;
+        if (multiTableRewriteMap != null) {
+            TableRef origRef = dataColRef.getTableRef();
+            TableRef tableRef = multiTableRewriteMap.get(origRef);
+            if (tableRef == null)
+                return node;
+            
+            if (origRef.getTableAlias() != null) {
+                tName = FACTORY.table(null, origRef.getTableAlias());
+            } else {
+                String schemaName = tableRef.getTable().getSchemaName().getString();
+                schemaName = schemaName.length() == 0 ? null : '"' + schemaName + '"';
+                String tableName = '"' + tableRef.getTable().getTableName().getString() + '"';
+                tName = FACTORY.table(schemaName, tableName);
+            }
+        }
+        String indexColName = IndexUtil.getIndexColumnName(dataColRef.getColumn());
+        // Same alias as before, but use the index column name instead of the data column name
+        ParseNode indexColNode = new ColumnParseNode(tName, indexColName, node.getAlias());
+        PDataType indexColType = IndexUtil.getIndexColumnDataType(dataColRef.getColumn());
+        PDataType dataColType = dataColRef.getColumn().getDataType();
+
+        // Coerce index column reference back to same type as data column so that
+        // expression behave exactly the same. No need to invert, as this will be done
+        // automatically as needed. If node is used at the top level, do not convert, as
+        // otherwise the wrapper gets in the way in the group by clause. For example,
+        // an INTEGER column in a GROUP BY gets doubly wrapped like this:
+        //     CAST CAST int_col AS INTEGER AS DECIMAL
+        // This is unnecessary and problematic in the case of a null value.
+        // TODO: test case for this
+        if (!isTopLevel() && indexColType != dataColType) {
+            indexColNode = FACTORY.cast(indexColNode, dataColType);
+        }
+        return indexColNode;
+    }
+
+    @Override
+    public ParseNode visit(WildcardParseNode node) throws SQLException {
+        return WildcardParseNode.REWRITE_INSTANCE;
+    }
+
+    @Override
+    public ParseNode visit(FamilyWildcardParseNode node) throws SQLException {
+        return new FamilyWildcardParseNode(node, true);
+    }
+    
+}


[28/51] [partial] Initial commit of master branch from github

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RegexpSubstrFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RegexpSubstrFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RegexpSubstrFunction.java
new file mode 100644
index 0000000..2308581
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RegexpSubstrFunction.java
@@ -0,0 +1,175 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.LiteralExpression;
+import org.apache.phoenix.parse.FunctionParseNode.Argument;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.ByteUtil;
+
+
+/**
+ * 
+ * Implementation of REGEXP_SUBSTR(<source>, <pattern>, <offset>) built-in function,
+ * where <offset> is the offset from the start of <string>. Positive offset is treated as 1-based,
+ * a zero offset is treated as 0-based, and a negative offset starts from the end of the string 
+ * working backwards. The <pattern> is the pattern we would like to search for in the <source> string.
+ * The function returns the first occurrence of any substring in the <source> string that matches
+ * the <pattern> input as a VARCHAR. 
+ * 
+ * @author zhuang
+ * @since 0.1
+ */
+@BuiltInFunction(name=RegexpSubstrFunction.NAME, args={
+    @Argument(allowedTypes={PDataType.VARCHAR}),
+    @Argument(allowedTypes={PDataType.VARCHAR}),
+    @Argument(allowedTypes={PDataType.LONG}, defaultValue="1")} )
+public class RegexpSubstrFunction extends PrefixFunction {
+    public static final String NAME = "REGEXP_SUBSTR";
+
+    private Pattern pattern;
+    private boolean isOffsetConstant;
+    private Integer byteSize;
+
+    public RegexpSubstrFunction() { }
+
+    public RegexpSubstrFunction(List<Expression> children) {
+        super(children);
+        init();
+    }
+
+    private void init() {
+        Object patternString = ((LiteralExpression)children.get(1)).getValue();
+        if (patternString != null) {
+            pattern = Pattern.compile((String)patternString);
+        }
+        // If the source string has a fixed width, then the max length would be the length 
+        // of the source string minus the offset, or the absolute value of the offset if 
+        // it's negative. Offset number is a required argument. However, if the source string
+        // is not fixed width, the maxLength would be null.
+        isOffsetConstant = getOffsetExpression() instanceof LiteralExpression;
+        Number offsetNumber = (Number)((LiteralExpression)getOffsetExpression()).getValue();
+        if (offsetNumber != null) {
+            int offset = offsetNumber.intValue();
+            if (getSourceStrExpression().getDataType().isFixedWidth()) {
+                if (offset >= 0) {
+                    byteSize = getSourceStrExpression().getByteSize() - offset - (offset == 0 ? 0 : 1);
+                } else {
+                    byteSize = -offset;
+                }
+            }
+        }
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        if (pattern == null) {
+            return false;
+        }
+        if (!getSourceStrExpression().evaluate(tuple, ptr)) {
+            return false;
+        }
+        String sourceStr = (String)PDataType.VARCHAR.toObject(ptr, getSourceStrExpression().getColumnModifier());
+        if (sourceStr == null) {
+            return false;
+        }
+
+        Expression offsetExpression = getOffsetExpression();
+        if (!offsetExpression.evaluate(tuple, ptr)) {
+            return false;
+        }
+        int offset = offsetExpression.getDataType().getCodec().decodeInt(ptr, offsetExpression.getColumnModifier());
+
+        int strlen = sourceStr.length();
+        // Account for 1 versus 0-based offset
+        offset = offset - (offset <= 0 ? 0 : 1);
+        if (offset < 0) { // Offset < 0 means get from end
+            offset = strlen + offset;
+        }
+        if (offset < 0 || offset >= strlen) {
+            return false;
+        }
+
+        Matcher matcher = pattern.matcher(sourceStr);
+        boolean hasSubString = matcher.find(offset);
+        if (!hasSubString) {
+            ptr.set(ByteUtil.EMPTY_BYTE_ARRAY);
+            return true;
+        }
+        String subString = matcher.group();
+        ptr.set(PDataType.VARCHAR.toBytes(subString));
+        return true;
+    }
+
+    @Override
+    public Integer getByteSize() {
+        return byteSize;
+    }
+
+    @Override
+    public OrderPreserving preservesOrder() {
+        if (isOffsetConstant) {
+            LiteralExpression literal = (LiteralExpression) getOffsetExpression();
+            Number offsetNumber = (Number) literal.getValue();
+            if (offsetNumber != null) { 
+                int offset = offsetNumber.intValue();
+                if (offset == 0 || offset == 1) {
+                    return OrderPreserving.YES_IF_LAST;
+                }
+            }
+        }
+        return OrderPreserving.NO;
+    }
+
+    @Override
+    public int getKeyFormationTraversalIndex() {
+        return preservesOrder() == OrderPreserving.NO ? NO_TRAVERSAL : 0;
+    }
+
+    private Expression getOffsetExpression() {
+        return children.get(2);
+    }
+
+    private Expression getSourceStrExpression() {
+        return children.get(0);
+    }
+
+    @Override
+    public PDataType getDataType() {
+        // ALways VARCHAR since we do not know in advanced how long the 
+        // matched string will be.
+        return PDataType.VARCHAR;
+    }
+
+    @Override
+    public String getName() {
+        return NAME;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ReverseFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ReverseFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ReverseFunction.java
new file mode 100644
index 0000000..b90fd7c
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ReverseFunction.java
@@ -0,0 +1,70 @@
+package org.apache.phoenix.expression.function;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.parse.FunctionParseNode.Argument;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.StringUtil;
+
+@BuiltInFunction(name=ReverseFunction.NAME,  args={
+        @Argument(allowedTypes={PDataType.VARCHAR})} )
+public class ReverseFunction extends ScalarFunction {
+    public static final String NAME = "REVERSE";
+    
+    public ReverseFunction() {
+    }
+
+    public ReverseFunction(List<Expression> children) throws SQLException {
+        super(children);
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        Expression arg = getChildren().get(0);
+        if (!arg.evaluate(tuple, ptr)) {
+            return false;
+        }
+
+        int targetOffset = ptr.getLength();
+        if (targetOffset == 0) {
+            return true;
+        }
+
+        byte[] source = ptr.get();
+        byte[] target = new byte[targetOffset];
+        int sourceOffset = ptr.getOffset(); 
+        int endOffset = sourceOffset + ptr.getLength();
+        ColumnModifier modifier = arg.getColumnModifier();
+        while (sourceOffset < endOffset) {
+            int nBytes = StringUtil.getBytesInChar(source[sourceOffset], modifier);
+            targetOffset -= nBytes;
+            System.arraycopy(source, sourceOffset, target, targetOffset, nBytes);
+            sourceOffset += nBytes;
+        }
+        ptr.set(target);
+        return true;
+    }
+
+    @Override
+    public ColumnModifier getColumnModifier() {
+        return getChildren().get(0).getColumnModifier();
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return PDataType.VARCHAR;
+    }
+
+    @Override
+    public String getName() {
+        return NAME;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RoundDateExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RoundDateExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RoundDateExpression.java
new file mode 100644
index 0000000..eab4b28
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RoundDateExpression.java
@@ -0,0 +1,274 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.sql.Date;
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.io.WritableUtils;
+
+import com.google.common.collect.Lists;
+import org.apache.phoenix.compile.KeyPart;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.LiteralExpression;
+import org.apache.phoenix.query.KeyRange;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.PDataType.PDataCodec;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.ByteUtil;
+
+/**
+ * Function used to bucketize date/time values by rounding them to
+ * an even increment.  Usage:
+ * ROUND(<date/time col ref>,<'day'|'hour'|'minute'|'second'|'millisecond'>,<optional integer multiplier>)
+ * The integer multiplier is optional and is used to do rollups to a partial time unit (i.e. 10 minute rollup)
+ * The function returns a {@link org.apache.phoenix.schema.PDataType#DATE}
+
+ * @author jtaylor, samarth.jain
+ * @since 0.1
+ */
+public class RoundDateExpression extends ScalarFunction {
+    
+    long divBy;
+    
+    public static final String NAME = "ROUND";
+    
+    private static final long[] TIME_UNIT_MS = new long[] {
+        24 * 60 * 60 * 1000,
+        60 * 60 * 1000,
+        60 * 1000,
+        1000,
+        1
+    };
+    
+    public RoundDateExpression() {}
+    
+    /**
+     * @param timeUnit - unit of time to round up to.
+     * Creates a {@link RoundDateExpression} with default multiplier of 1.
+     */
+    public static Expression create(Expression expr, TimeUnit timeUnit) throws SQLException {
+        return create(expr, timeUnit, 1);
+    }
+    
+    /**
+     * @param timeUnit - unit of time to round up to
+     * @param multiplier - determines the roll up window size.
+     * Create a {@link RoundDateExpression}. 
+     */
+    public static Expression create(Expression expr, TimeUnit timeUnit, int multiplier) throws SQLException {
+        Expression timeUnitExpr = getTimeUnitExpr(timeUnit);
+        Expression defaultMultiplierExpr = getMultiplierExpr(multiplier);
+        List<Expression> expressions = Lists.newArrayList(expr, timeUnitExpr, defaultMultiplierExpr);
+        return create(expressions);
+    }
+    
+    public static Expression create(List<Expression> children) throws SQLException {
+        return new RoundDateExpression(children);
+    }
+    
+    static Expression getTimeUnitExpr(TimeUnit timeUnit) throws SQLException {
+        return LiteralExpression.newConstant(timeUnit.name(), PDataType.VARCHAR, true);
+    }
+    
+    static Expression getMultiplierExpr(int multiplier) throws SQLException {
+        return LiteralExpression.newConstant(multiplier, PDataType.INTEGER, true);
+    }
+    
+    RoundDateExpression(List<Expression> children) {
+        super(children.subList(0, 1));
+        int numChildren = children.size();
+        if(numChildren < 2 || numChildren > 3) {
+            throw new IllegalArgumentException("Wrong number of arguments : " + numChildren);
+        }
+        Object timeUnitValue = ((LiteralExpression)children.get(1)).getValue();
+        Object multiplierValue = numChildren > 2 ? ((LiteralExpression)children.get(2)).getValue() : null;
+        int multiplier = multiplierValue == null ? 1 :((Number)multiplierValue).intValue();
+        TimeUnit timeUnit = TimeUnit.getTimeUnit(timeUnitValue != null ? timeUnitValue.toString() : null); 
+        divBy = multiplier * TIME_UNIT_MS[timeUnit.ordinal()];
+    }
+    
+    
+    protected long getRoundUpAmount() {
+        return divBy/2;
+    }
+    
+    
+    protected long roundTime(long time) {
+        long value;
+        long roundUpAmount = getRoundUpAmount();
+        if (time <= Long.MAX_VALUE - roundUpAmount) { // If no overflow, add
+            value = (time + roundUpAmount) / divBy;
+        } else { // Else subtract and add one
+            value = (time - roundUpAmount) / divBy + 1;
+        }
+        return value * divBy;
+    }
+    
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        if (children.get(0).evaluate(tuple, ptr)) {
+            PDataType dataType = getDataType();
+            long time = dataType.getCodec().decodeLong(ptr, children.get(0).getColumnModifier());
+            long value = roundTime(time);
+            
+            Date d = new Date(value);
+            byte[] byteValue = dataType.toBytes(d);
+            ptr.set(byteValue);
+            return true;
+        }
+        return false;
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        long roundUpAmount = this.getRoundUpAmount();
+        result = prime * result + (int)(divBy ^ (divBy >>> 32));
+        result = prime * result + (int)(roundUpAmount ^ (roundUpAmount >>> 32));
+        result = prime * result + children.get(0).hashCode();
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) return true;
+        if (obj == null) return false;
+        if (getClass() != obj.getClass()) return false;
+        RoundDateExpression other = (RoundDateExpression)obj;
+        if (divBy != other.divBy) return false;
+        if (getRoundUpAmount() != other.getRoundUpAmount()) return false;
+        return children.get(0).equals(other.children.get(0));
+    }
+    
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        super.readFields(input);
+        divBy = WritableUtils.readVLong(input);
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+        super.write(output);
+        WritableUtils.writeVLong(output, divBy);
+    }
+    
+    @Override
+    public PDataType getDataType() {
+        return children.get(0).getDataType();
+    }
+    
+    @Override
+    public Integer getByteSize() {
+        return children.get(0).getByteSize();
+    }
+
+    @Override
+    public boolean isNullable() {
+        return children.get(0).isNullable() || divBy == 0;
+    }
+    
+    protected PDataCodec getKeyRangeCodec(PDataType columnDataType) {
+        return columnDataType.getCodec();
+    }
+    
+    /**
+     * Form the key range from the key to the key right before or at the
+     * next rounded value.
+     */
+    @Override
+    public KeyPart newKeyPart(final KeyPart childPart) {
+        return new KeyPart() {
+            private final List<Expression> extractNodes = Collections.<Expression>singletonList(RoundDateExpression.this);
+
+            @Override
+            public PColumn getColumn() {
+                return childPart.getColumn();
+            }
+
+            @Override
+            public List<Expression> getExtractNodes() {
+                return extractNodes;
+            }
+
+            @Override
+            public KeyRange getKeyRange(CompareOp op, Expression rhs) {
+                PDataType type = getColumn().getDataType();
+                ImmutableBytesWritable ptr = new ImmutableBytesWritable();
+                rhs.evaluate(null, ptr);
+                byte[] key = ByteUtil.copyKeyBytesIfNecessary(ptr);
+                // No need to take into account column modifier, because ROUND
+                // always forces the value to be in ascending order
+                PDataCodec codec = getKeyRangeCodec(type);
+                int offset = ByteUtil.isInclusive(op) ? 1 : 0;
+                long value = codec.decodeLong(key, 0, null);
+                byte[] nextKey = new byte[type.getByteSize()];
+                switch (op) {
+                case EQUAL:
+                    // If the value isn't evenly divisible by the div amount, then it
+                    // can't possibly be equal to any rounded value. For example, if you
+                    // had ROUND(dateCol,'DAY') = TO_DATE('2013-01-01 23:00:00')
+                    // it could never be equal, since date constant isn't at a day
+                    // boundary.
+                    if (value % divBy != 0) {
+                        return KeyRange.EMPTY_RANGE;
+                    }
+                    codec.encodeLong(value + divBy, nextKey, 0);
+                    return type.getKeyRange(key, true, nextKey, false);
+                case GREATER:
+                case GREATER_OR_EQUAL:
+                    codec.encodeLong((value + divBy - offset)/divBy*divBy, nextKey, 0);
+                    return type.getKeyRange(nextKey, true, KeyRange.UNBOUND, false);
+                case LESS:
+                case LESS_OR_EQUAL:
+                    codec.encodeLong((value + divBy - (1 -offset))/divBy*divBy, nextKey, 0);
+                    return type.getKeyRange(KeyRange.UNBOUND, false, nextKey, false);
+                default:
+                    return childPart.getKeyRange(op, rhs);
+                }
+            }
+        };
+    }
+
+
+    @Override
+    public String getName() {
+        return NAME;
+    }
+    
+    @Override
+    public OrderPreserving preservesOrder() {
+        return OrderPreserving.YES;
+    }
+
+    @Override
+    public int getKeyFormationTraversalIndex() {
+        return 0;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RoundDecimalExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RoundDecimalExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RoundDecimalExpression.java
new file mode 100644
index 0000000..85e7b75
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RoundDecimalExpression.java
@@ -0,0 +1,136 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.math.RoundingMode;
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.io.WritableUtils;
+
+import com.google.common.collect.Lists;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.LiteralExpression;
+import org.apache.phoenix.schema.IllegalDataException;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+/**
+ * 
+ * Class encapsulating the process for rounding off a column/literal of 
+ * type {@link org.apache.phoenix.schema.PDataType#DECIMAL}
+ *
+ * @author samarth.jain
+ * @since 3.0.0
+ */
+
+public class RoundDecimalExpression extends ScalarFunction {
+    
+    private int scale;
+    
+    /**
+     * Creates a {@link RoundDecimalExpression} with rounding scale given by @param scale. 
+     *
+     */
+    public static Expression create(Expression expr, int scale) throws SQLException {
+        if (expr.getDataType().isCoercibleTo(PDataType.LONG)) {
+            return expr;
+        }
+        Expression scaleExpr = LiteralExpression.newConstant(scale, PDataType.INTEGER, true);
+        List<Expression> expressions = Lists.newArrayList(expr, scaleExpr);
+        return new RoundDecimalExpression(expressions);
+    }
+    
+    /**
+     * Creates a {@link RoundDecimalExpression} with a default scale of 0 used for rounding. 
+     *
+     */
+    public static Expression create(Expression expr) throws SQLException {
+        return create(expr, 0);
+    }
+    
+    public RoundDecimalExpression() {}
+    
+    public RoundDecimalExpression(List<Expression> children) {
+        super(children);
+        LiteralExpression scaleChild = (LiteralExpression)children.get(1);
+        PDataType scaleType = scaleChild.getDataType();
+        Object scaleValue = scaleChild.getValue();
+        if(scaleValue != null) {
+            if (scaleType.isCoercibleTo(PDataType.INTEGER, scaleValue)) {
+                int scale = (Integer)PDataType.INTEGER.toObject(scaleValue, scaleType);
+                if (scale >=0 && scale <= PDataType.MAX_PRECISION) {
+                    this.scale = scale;
+                    return;
+                }
+            }
+            throw new IllegalDataException("Invalid second argument for scale: " + scaleValue + ". The scale must be between 0 and " + PDataType.MAX_PRECISION + " inclusive.");
+        } 
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        Expression childExpr = children.get(0);
+        if(childExpr.evaluate(tuple, ptr)) {
+            BigDecimal value = (BigDecimal)PDataType.DECIMAL.toObject(ptr, childExpr.getColumnModifier());
+            BigDecimal scaledValue = value.setScale(scale, getRoundingMode());
+            ptr.set(getDataType().toBytes(scaledValue));
+            return true;
+        }
+        return false;
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return children.get(0).getDataType();
+    }
+    
+    protected RoundingMode getRoundingMode() {
+        return RoundingMode.HALF_UP;
+    }
+    
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        super.readFields(input);
+        scale = WritableUtils.readVInt(input);
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+        super.write(output);
+        WritableUtils.writeVInt(output, scale);
+    }
+
+    @Override
+    public String getName() {
+        return RoundFunction.NAME;
+    }
+    
+    @Override
+    public OrderPreserving preservesOrder() {
+        return OrderPreserving.YES;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RoundFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RoundFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RoundFunction.java
new file mode 100644
index 0000000..1b0782b
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RoundFunction.java
@@ -0,0 +1,57 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import java.util.List;
+
+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.RoundParseNode;
+import org.apache.phoenix.schema.PDataType;
+
+
+/**
+ * Base class for RoundFunction.
+ * 
+ * @author jtaylor, samarth.jain
+ * @since 0.1
+ */
+@BuiltInFunction(name = RoundFunction.NAME, 
+                 nodeClass = RoundParseNode.class,
+                 args = {
+                        @Argument(allowedTypes={PDataType.TIMESTAMP, PDataType.DECIMAL}),
+                        @Argument(allowedTypes={PDataType.VARCHAR, PDataType.INTEGER}, defaultValue = "null", isConstant=true),
+                        @Argument(allowedTypes={PDataType.INTEGER}, defaultValue="1", isConstant=true)
+                        } 
+                )
+public abstract class RoundFunction extends ScalarFunction {
+    
+    public static final String NAME = "ROUND";
+    
+    public RoundFunction(List<Expression> children) {
+        super(children);
+    }
+    
+    @Override
+    public String getName() {
+        return NAME;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RoundTimestampExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RoundTimestampExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RoundTimestampExpression.java
new file mode 100644
index 0000000..0146b19
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RoundTimestampExpression.java
@@ -0,0 +1,106 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import java.sql.SQLException;
+import java.sql.Timestamp;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import com.google.common.collect.Lists;
+import org.apache.phoenix.expression.CoerceExpression;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.LiteralExpression;
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.PDataType.PDataCodec;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+/**
+ * 
+ * Class encapsulating the process for rounding off a column/literal of 
+ * type {@link org.apache.phoenix.schema.PDataType#TIMESTAMP}
+ * This class only supports rounding off the milliseconds that is for
+ * {@link TimeUnit#MILLISECOND}. If you want more options of rounding like 
+ * using {@link TimeUnit#HOUR} use {@link RoundDateExpression}
+ *
+ * @author samarth.jain
+ * @since 3.0.0
+ */
+
+public class RoundTimestampExpression extends RoundDateExpression {
+    
+    private static final long HALF_OF_NANOS_IN_MILLI = java.util.concurrent.TimeUnit.MILLISECONDS.toNanos(1)/2;
+
+    public RoundTimestampExpression() {}
+    
+    private RoundTimestampExpression(List<Expression> children) {
+        super(children);
+    }
+    
+    public static Expression create (List<Expression> children) throws SQLException {
+        Expression firstChild = children.get(0);
+        PDataType firstChildDataType = firstChild.getDataType();
+        String timeUnit = (String)((LiteralExpression)children.get(1)).getValue();
+        LiteralExpression multiplierExpr = (LiteralExpression)children.get(2);
+        
+        /*
+         * When rounding off timestamp to milliseconds, nanos play a part only when the multiplier value
+         * is equal to 1. This is because for cases when multiplier value is greater than 1, number of nanos/multiplier
+         * will always be less than half the nanos in a millisecond. 
+         */
+        if((timeUnit == null || TimeUnit.MILLISECOND.toString().equalsIgnoreCase(timeUnit)) && ((Number)multiplierExpr.getValue()).intValue() == 1) {
+            return new RoundTimestampExpression(children);
+        }
+        // Coerce TIMESTAMP to DATE, as the nanos has no affect
+        List<Expression> newChildren = Lists.newArrayListWithExpectedSize(children.size());
+        newChildren.add(CoerceExpression.create(firstChild, firstChildDataType == PDataType.TIMESTAMP ? PDataType.DATE : PDataType.UNSIGNED_DATE));
+        newChildren.addAll(children.subList(1, children.size()));
+        return RoundDateExpression.create(newChildren);
+    }
+    
+    @Override
+    protected PDataCodec getKeyRangeCodec(PDataType columnDataType) {
+        return columnDataType == PDataType.TIMESTAMP 
+                ? PDataType.DATE.getCodec() 
+                : columnDataType == PDataType.UNSIGNED_TIMESTAMP 
+                    ? PDataType.UNSIGNED_DATE.getCodec() 
+                    : super.getKeyRangeCodec(columnDataType);
+    }
+    
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        if (children.get(0).evaluate(tuple, ptr)) {
+            ColumnModifier columnModifier = children.get(0).getColumnModifier();
+            PDataType dataType = getDataType();
+            int nanos = dataType.getNanos(ptr, columnModifier);
+            if(nanos >= HALF_OF_NANOS_IN_MILLI) {
+                long timeMillis = dataType.getMillis(ptr, columnModifier);
+                Timestamp roundedTs = new Timestamp(timeMillis + 1);
+                byte[] byteValue = dataType.toBytes(roundedTs);
+                ptr.set(byteValue);
+            }
+            return true; // for timestamp we only support rounding up the milliseconds. 
+        }
+        return false;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SQLTableTypeFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SQLTableTypeFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SQLTableTypeFunction.java
new file mode 100644
index 0000000..04465ea
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SQLTableTypeFunction.java
@@ -0,0 +1,80 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.parse.FunctionParseNode.Argument;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+
+/**
+ * 
+ * Function used to get the SQL table type name from the serialized table type.
+ * Usage:
+ * SqlTableType('v') will return 'VIEW' based on
+ * {@link java.sql.DatabaseMetaData#getTableTypes()}
+ * 
+ * @author jtaylor
+ * @since 2.2
+ */
+@BuiltInFunction(name=SQLTableTypeFunction.NAME, args= {
+    @Argument(allowedTypes=PDataType.CHAR)} )
+public class SQLTableTypeFunction extends ScalarFunction {
+    public static final String NAME = "SQLTableType";
+
+    public SQLTableTypeFunction() {
+    }
+    
+    public SQLTableTypeFunction(List<Expression> children) throws SQLException {
+        super(children);
+    }
+    
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        Expression child = children.get(0);
+        if (!child.evaluate(tuple, ptr)) {
+            return false;
+        }
+        if (ptr.getLength() == 0) {
+            return true;
+        }
+        PTableType tableType = PTableType.fromSerializedValue(ptr.get()[ptr.getOffset()]);
+        ptr.set(tableType.getValue().getBytes());
+        return true;
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return PDataType.VARCHAR;
+    }
+    
+    @Override
+    public String getName() {
+        return NAME;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SQLViewTypeFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SQLViewTypeFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SQLViewTypeFunction.java
new file mode 100644
index 0000000..e70ec8e
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SQLViewTypeFunction.java
@@ -0,0 +1,80 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.parse.FunctionParseNode.Argument;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.PTable.ViewType;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+
+/**
+ * 
+ * Function used to get the SQL view type name from the serialized view type.
+ * Usage:
+ * SQLViewType('v') will return 'VIEW' based on
+ * {@link java.sql.DatabaseMetaData#getTableTypes()}
+ * 
+ * @author jtaylor
+ * @since 2.2
+ */
+@BuiltInFunction(name=SQLViewTypeFunction.NAME, args= {
+    @Argument(allowedTypes=PDataType.UNSIGNED_TINYINT)} )
+public class SQLViewTypeFunction extends ScalarFunction {
+    public static final String NAME = "SQLViewType";
+
+    public SQLViewTypeFunction() {
+    }
+    
+    public SQLViewTypeFunction(List<Expression> children) throws SQLException {
+        super(children);
+    }
+    
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        Expression child = children.get(0);
+        if (!child.evaluate(tuple, ptr)) {
+            return false;
+        }
+        if (ptr.getLength() == 0) {
+            return true;
+        }
+        ViewType viewType = ViewType.fromSerializedValue(ptr.get()[ptr.getOffset()]);
+        ptr.set(viewType.getBytes());
+        return true;
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return PDataType.VARCHAR;
+    }
+    
+    @Override
+    public String getName() {
+        return NAME;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ScalarFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ScalarFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ScalarFunction.java
new file mode 100644
index 0000000..1d1059b
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ScalarFunction.java
@@ -0,0 +1,72 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import java.util.List;
+
+import org.apache.phoenix.compile.KeyPart;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.visitor.ExpressionVisitor;
+
+
+public abstract class ScalarFunction extends FunctionExpression {
+    public static final int NO_TRAVERSAL = -1;
+    
+    public ScalarFunction() {
+    }
+    
+    public ScalarFunction(List<Expression> children) {
+        super(children);
+    }
+    
+    @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;
+    }
+    
+    /**
+     * Determines whether or not a function may be used to form
+     * the start/stop key of a scan
+     * @return the zero-based position of the argument to traverse
+     *  into to look for a primary key column reference, or
+     *  {@value #NO_TRAVERSAL} if the function cannot be used to
+     *  form the scan key.
+     */
+    public int getKeyFormationTraversalIndex() {
+        return NO_TRAVERSAL;
+    }
+
+    /**
+     * Manufactures a KeyPart used to construct the KeyRange given
+     * a constant and a comparison operator.
+     * @param childPart the KeyPart formulated for the child expression
+     *  at the {@link #getKeyFormationTraversalIndex()} position.
+     * @return the KeyPart for constructing the KeyRange for this
+     *  function.
+     */
+    public KeyPart newKeyPart(KeyPart childPart) {
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SingleAggregateFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SingleAggregateFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SingleAggregateFunction.java
new file mode 100644
index 0000000..f02c4e7
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SingleAggregateFunction.java
@@ -0,0 +1,173 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import java.io.DataInput;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.LiteralExpression;
+import org.apache.phoenix.expression.aggregator.Aggregator;
+import org.apache.phoenix.expression.visitor.ExpressionVisitor;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+
+/**
+ * 
+ * Base class for aggregate functions that calculate an aggregation
+ * using a single {{@link Aggregator}
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+abstract public class SingleAggregateFunction extends AggregateFunction {
+    private static final List<Expression> DEFAULT_EXPRESSION_LIST = Arrays.<Expression>asList(LiteralExpression.newConstant(1, true));
+    protected boolean isConstant;
+    private Aggregator aggregator;
+    
+    /**
+     * Sort aggregate functions with nullable fields last. This allows us not to have to store trailing null values.
+     * Within non-nullable/nullable groups, put fixed width values first since we can access those more efficiently
+     * (i.e. we can skip over groups of them in-mass instead of reading the length of each one to skip over as
+     * required by a variable length value).
+     */
+    public static final Comparator<SingleAggregateFunction> SCHEMA_COMPARATOR = new Comparator<SingleAggregateFunction>() {
+
+        @Override
+        public int compare(SingleAggregateFunction o1, SingleAggregateFunction o2) {
+            boolean isNullable1 = o1.isNullable();
+            boolean isNullable2 = o2.isNullable();
+            if (isNullable1 != isNullable2) {
+                return isNullable1 ? 1 : -1;
+            }
+            isNullable1 = o1.getAggregatorExpression().isNullable();
+            isNullable2 = o2.getAggregatorExpression().isNullable();
+            if (isNullable1 != isNullable2) {
+                return isNullable1 ? 1 : -1;
+            }
+            // Ensures COUNT(1) sorts first TODO: unit test for this
+            boolean isConstant1 = o1.isConstantExpression();
+            boolean isConstant2 = o2.isConstantExpression();
+            if (isConstant1 != isConstant2) {
+                return isConstant1 ? 1 : -1;
+            }
+            PDataType r1 = o1.getAggregator().getDataType();
+            PDataType r2 = o2.getAggregator().getDataType();
+            if (r1.isFixedWidth() != r2.isFixedWidth()) {
+                return r1.isFixedWidth() ? -1 : 1;
+            }
+            return r1.compareTo(r2);
+        }
+    };
+    
+    protected SingleAggregateFunction() {
+        this(DEFAULT_EXPRESSION_LIST, true);
+    }
+
+    public SingleAggregateFunction(List<Expression> children) {
+        this(children, children.get(0) instanceof LiteralExpression);
+    }
+    
+    private SingleAggregateFunction(List<Expression> children, boolean isConstant) {
+        super(children);
+        this.isConstant = children.get(0) instanceof LiteralExpression;
+        this.aggregator = newClientAggregator();
+    }
+
+    public boolean isConstantExpression() {
+        return isConstant;
+    }
+    
+    @Override
+    public PDataType getDataType() {
+        return children.get(0).getDataType();
+    }
+    
+    public Expression getAggregatorExpression() {
+        return children.get(0);
+    }
+    
+    public Aggregator getAggregator() {
+        return aggregator;
+    }
+    
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        return getAggregator().evaluate(tuple, ptr);
+    }
+
+    /**
+     * Create the aggregator to do server-side aggregation.
+     * The data type of the returned Aggregator must match
+     * the data type returned by {@link #newClientAggregator()}
+     * @param conf HBase configuration.
+     * @return the aggregator to use on the server-side
+     */
+    abstract public Aggregator newServerAggregator(Configuration conf);
+    /**
+     * Create the aggregator to do client-side aggregation
+     * based on the results returned from the aggregating
+     * coprocessor. The data type of the returned Aggregator
+     * must match the data type returned by {@link #newServerAggregator(Configuration)}
+     * @return the aggregator to use on the client-side
+     */
+    public Aggregator newClientAggregator() {
+        return newServerAggregator(null);
+    }
+
+    public Aggregator newServerAggregator(Configuration config, ImmutableBytesWritable ptr) {
+        Aggregator agg = newServerAggregator(config);
+        agg.aggregate(null, ptr);
+        return agg;
+    }
+    
+    public void readFields(DataInput input, Configuration conf) throws IOException {
+        super.readFields(input);
+        aggregator = newServerAggregator(conf);
+    }
+
+    @Override
+    public boolean isNullable() {
+        return true;
+    }
+    
+    protected SingleAggregateFunction getDelegate() {
+        return this;
+    }
+
+    @Override
+    public final <T> T accept(ExpressionVisitor<T> visitor) {
+        SingleAggregateFunction function = getDelegate();
+        List<T> l = acceptChildren(visitor, visitor.visitEnter(function));
+        T t = visitor.visitLeave(function, l);
+        if (t == null) {
+            t = visitor.defaultReturn(function, l);
+        }
+        return t;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SqlTypeNameFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SqlTypeNameFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SqlTypeNameFunction.java
new file mode 100644
index 0000000..5d39123
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SqlTypeNameFunction.java
@@ -0,0 +1,86 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.parse.FunctionParseNode.Argument;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.schema.IllegalDataException;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.ByteUtil;
+
+
+/**
+ * 
+ * Function used to get the SQL type name from the SQL type integer.
+ * Usage:
+ * SqlTypeName(12)
+ * will return 'VARCHAR' based on {@link java.sql.Types#VARCHAR} being 12
+ * 
+ * @author jtaylor
+ * @since 0.1
+ */
+@BuiltInFunction(name=SqlTypeNameFunction.NAME, args= {
+    @Argument(allowedTypes=PDataType.INTEGER)} )
+public class SqlTypeNameFunction extends ScalarFunction {
+    public static final String NAME = "SqlTypeName";
+
+    public SqlTypeNameFunction() {
+    }
+    
+    public SqlTypeNameFunction(List<Expression> children) throws SQLException {
+        super(children);
+    }
+    
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        Expression child = children.get(0);
+        if (!child.evaluate(tuple, ptr)) {
+            return false;
+        }
+        if (ptr.getLength() == 0) {
+            return true;
+        }
+        int sqlType = child.getDataType().getCodec().decodeInt(ptr, child.getColumnModifier());
+        try {
+            byte[] sqlTypeNameBytes = PDataType.fromTypeId(sqlType).getSqlTypeNameBytes();
+            ptr.set(sqlTypeNameBytes);
+        } catch (IllegalDataException e) {
+            ptr.set(ByteUtil.EMPTY_BYTE_ARRAY);
+        }
+        return true;
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return PDataType.VARCHAR;
+    }
+    
+    @Override
+    public String getName() {
+        return NAME;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/StddevPopFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/StddevPopFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/StddevPopFunction.java
new file mode 100644
index 0000000..4a4e7b8
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/StddevPopFunction.java
@@ -0,0 +1,78 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.aggregator.Aggregator;
+import org.apache.phoenix.expression.aggregator.DecimalStddevPopAggregator;
+import org.apache.phoenix.expression.aggregator.DistinctValueWithCountClientAggregator;
+import org.apache.phoenix.expression.aggregator.DistinctValueWithCountServerAggregator;
+import org.apache.phoenix.expression.aggregator.StddevPopAggregator;
+import org.apache.phoenix.parse.FunctionParseNode.Argument;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.schema.PDataType;
+
+/**
+ * 
+ * Built-in function for STDDEV_POP(<expression>) aggregate function
+ * 
+ * @author anoopsjohn
+ * @since 1.2.1
+ */
+@BuiltInFunction(name = StddevPopFunction.NAME, args = { @Argument(allowedTypes={PDataType.DECIMAL})})
+public class StddevPopFunction extends DistinctValueWithCountAggregateFunction {
+    public static final String NAME = "STDDEV_POP";
+
+    public StddevPopFunction() {
+
+    }
+
+    public StddevPopFunction(List<Expression> childern) {
+        super(childern);
+    }
+
+    @Override
+    public Aggregator newServerAggregator(Configuration conf) {
+        return new DistinctValueWithCountServerAggregator(conf);
+    }
+
+    @Override
+    public DistinctValueWithCountClientAggregator newClientAggregator() {
+        if (children.get(0).getDataType() == PDataType.DECIMAL) {
+            // Special Aggregators for DECIMAL datatype for more precision than double
+            return new DecimalStddevPopAggregator(children, getAggregatorExpression().getColumnModifier());
+        }
+        return new StddevPopAggregator(children, getAggregatorExpression().getColumnModifier());
+    }
+    
+    @Override
+    public String getName() {
+        return NAME;
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return PDataType.DECIMAL;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/StddevSampFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/StddevSampFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/StddevSampFunction.java
new file mode 100644
index 0000000..0e931c6
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/StddevSampFunction.java
@@ -0,0 +1,78 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.aggregator.Aggregator;
+import org.apache.phoenix.expression.aggregator.DecimalStddevSampAggregator;
+import org.apache.phoenix.expression.aggregator.DistinctValueWithCountClientAggregator;
+import org.apache.phoenix.expression.aggregator.DistinctValueWithCountServerAggregator;
+import org.apache.phoenix.expression.aggregator.StddevSampAggregator;
+import org.apache.phoenix.parse.FunctionParseNode.Argument;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.schema.PDataType;
+
+/**
+ * 
+ * Built-in function for STDDEV_SAMP(<expression>) aggregate function
+ * 
+ * @author anoopsjohn
+ * @since 1.2.1
+ */
+@BuiltInFunction(name = StddevSampFunction.NAME, args = { @Argument(allowedTypes={PDataType.DECIMAL})})
+public class StddevSampFunction extends DistinctValueWithCountAggregateFunction {
+    public static final String NAME = "STDDEV_SAMP";
+
+    public StddevSampFunction() {
+
+    }
+
+    public StddevSampFunction(List<Expression> childern) {
+        super(childern);
+    }
+
+    @Override
+    public Aggregator newServerAggregator(Configuration conf) {
+        return new DistinctValueWithCountServerAggregator(conf);
+    }
+
+    @Override
+    public DistinctValueWithCountClientAggregator newClientAggregator() {
+        if (children.get(0).getDataType() == PDataType.DECIMAL) {
+            // Special Aggregators for DECIMAL datatype for more precision than double
+            return new DecimalStddevSampAggregator(children, getAggregatorExpression().getColumnModifier());
+        }
+        return new StddevSampAggregator(children, getAggregatorExpression().getColumnModifier());
+    }
+    
+    @Override
+    public String getName() {
+        return NAME;
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return PDataType.DECIMAL;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SubstrFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SubstrFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SubstrFunction.java
new file mode 100644
index 0000000..90aa3c8
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SubstrFunction.java
@@ -0,0 +1,217 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import java.io.DataInput;
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.LiteralExpression;
+import org.apache.phoenix.parse.FunctionParseNode.Argument;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.StringUtil;
+
+
+/**
+ * 
+ * Implementation of the SUBSTR(<string>,<offset>[,<length>]) built-in function
+ * where <offset> is the offset from the start of <string>. A positive offset
+ * is treated as 1-based, a zero offset is treated as 0-based, and a negative
+ * offset starts from the end of the string working backwards. The optional
+ * <length> argument is the number of characters to return. In the absence of the
+ * <length> argument, the rest of the string starting from <offset> is returned.
+ * If <length> is less than 1, null is returned.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+@BuiltInFunction(name=SubstrFunction.NAME,  args={
+    @Argument(allowedTypes={PDataType.VARCHAR}),
+    @Argument(allowedTypes={PDataType.LONG}), // These are LONG because negative numbers end up as longs
+    @Argument(allowedTypes={PDataType.LONG},defaultValue="null")} )
+public class SubstrFunction extends PrefixFunction {
+    public static final String NAME = "SUBSTR";
+    private boolean hasLengthExpression;
+    private boolean isOffsetConstant;
+    private boolean isLengthConstant;
+    private boolean isFixedWidth;
+    private Integer byteSize;
+
+    public SubstrFunction() {
+    }
+
+    public SubstrFunction(List<Expression> children) {
+        super(children);
+        init();
+    }
+
+    private void init() {
+        // TODO: when we have ColumnModifier.REVERSE, we'll need to negate offset,
+        // since the bytes are reverse and we'll want to work from the end.
+        isOffsetConstant = getOffsetExpression() instanceof LiteralExpression;
+        isLengthConstant = getLengthExpression() instanceof LiteralExpression;
+        hasLengthExpression = !isLengthConstant || ((LiteralExpression)getLengthExpression()).getValue() != null;
+        isFixedWidth = getStrExpression().getDataType().isFixedWidth() && ((hasLengthExpression && isLengthConstant) || (!hasLengthExpression && isOffsetConstant));
+        if (hasLengthExpression && isLengthConstant) {
+            Integer maxLength = ((Number)((LiteralExpression)getLengthExpression()).getValue()).intValue();
+            this.byteSize = maxLength >= 0 ? maxLength : 0;
+        } else if (isOffsetConstant) {
+            Number offsetNumber = (Number)((LiteralExpression)getOffsetExpression()).getValue();
+            if (offsetNumber != null) {
+                int offset = offsetNumber.intValue();
+                if (getStrExpression().getDataType().isFixedWidth()) {
+                    if (offset >= 0) {
+                        byteSize = getStrExpression().getByteSize() - offset + (offset == 0 ? 0 : 1);
+                    } else {
+                        byteSize = -offset;
+                    }
+                }
+            }
+        }
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        Expression offsetExpression = getOffsetExpression();
+        if (!offsetExpression.evaluate(tuple,  ptr)) {
+            return false;
+        }
+        int offset = offsetExpression.getDataType().getCodec().decodeInt(ptr, offsetExpression.getColumnModifier());
+        
+        int length = -1;
+        if (hasLengthExpression) {
+            Expression lengthExpression = getLengthExpression();
+            if (!lengthExpression.evaluate(tuple, ptr)) {
+                return false;
+            }
+            length = lengthExpression.getDataType().getCodec().decodeInt(ptr, lengthExpression.getColumnModifier());
+            if (length <= 0) {
+                return false;
+            }
+        }
+        
+        if (!getStrExpression().evaluate(tuple, ptr)) {
+            return false;
+        }
+        
+        try {
+            boolean isCharType = getStrExpression().getDataType() == PDataType.CHAR;
+            ColumnModifier columnModifier = getStrExpression().getColumnModifier();
+            int strlen = isCharType ? ptr.getLength() : StringUtil.calculateUTF8Length(ptr.get(), ptr.getOffset(), ptr.getLength(), columnModifier);
+            
+            // Account for 1 versus 0-based offset
+            offset = offset - (offset <= 0 ? 0 : 1);
+            if (offset < 0) { // Offset < 0 means get from end
+                offset = strlen + offset;
+            }
+            if (offset < 0 || offset >= strlen) {
+                return false;
+            }
+            int maxLength = strlen - offset;
+            length = length == -1 ? maxLength : Math.min(length,maxLength);
+            
+            int byteOffset = isCharType ? offset : StringUtil.getByteLengthForUtf8SubStr(ptr.get(), ptr.getOffset(), offset, columnModifier);
+            int byteLength = isCharType ? length : StringUtil.getByteLengthForUtf8SubStr(ptr.get(), ptr.getOffset() + byteOffset, length, columnModifier);
+            ptr.set(ptr.get(), ptr.getOffset() + byteOffset, byteLength);
+            return true;
+        } catch (UnsupportedEncodingException e) {
+            return false;
+        }
+    }
+
+    @Override
+    public PDataType getDataType() {
+        // If fixed width, then return child expression type.
+        // If not fixed width, then we don't know how big this will be across the board
+        return isFixedWidth ? getStrExpression().getDataType() : PDataType.VARCHAR;
+    }
+
+    @Override
+    public boolean isNullable() {
+        return getStrExpression().isNullable() || !isFixedWidth || getOffsetExpression().isNullable();
+    }
+
+    @Override
+    public Integer getByteSize() {
+        return byteSize;
+    }
+    
+    // TODO: we shouldn't need both getByteSize() and getMaxLength()
+    @Override
+    public Integer getMaxLength() {
+        return byteSize;
+    }
+    
+    @Override
+    public ColumnModifier getColumnModifier() {
+        return getStrExpression().getColumnModifier();
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        super.readFields(input);
+        init();
+    }
+
+    private Expression getStrExpression() {
+        return children.get(0);
+    }
+
+    private Expression getOffsetExpression() {
+        return children.get(1);
+    }
+
+    private Expression getLengthExpression() {
+        return children.get(2);
+    }
+
+    @Override
+    public OrderPreserving preservesOrder() {
+        if (isOffsetConstant) {
+            LiteralExpression literal = (LiteralExpression) getOffsetExpression();
+            Number offsetNumber = (Number) literal.getValue();
+            if (offsetNumber != null) { 
+                int offset = offsetNumber.intValue();
+                if ((offset == 0 || offset == 1) && (!hasLengthExpression || isLengthConstant)) {
+                    return OrderPreserving.YES_IF_LAST;
+                }
+            }
+        }
+        return OrderPreserving.NO;
+    }
+
+    @Override
+    protected boolean extractNode() {
+        return true;
+    }
+
+    @Override
+    public String getName() {
+        return NAME;
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SumAggregateFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SumAggregateFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SumAggregateFunction.java
new file mode 100644
index 0000000..ab64ce8
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/SumAggregateFunction.java
@@ -0,0 +1,146 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import java.math.BigDecimal;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.LiteralExpression;
+import org.apache.phoenix.expression.aggregator.Aggregator;
+import org.apache.phoenix.expression.aggregator.DecimalSumAggregator;
+import org.apache.phoenix.expression.aggregator.DoubleSumAggregator;
+import org.apache.phoenix.expression.aggregator.NumberSumAggregator;
+import org.apache.phoenix.parse.FunctionParseNode.Argument;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.parse.SumAggregateParseNode;
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+
+/**
+ * 
+ * Built-in function for SUM aggregation function.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+@BuiltInFunction(name=SumAggregateFunction.NAME, nodeClass=SumAggregateParseNode.class, args= {@Argument(allowedTypes={PDataType.DECIMAL})} )
+public class SumAggregateFunction extends DelegateConstantToCountAggregateFunction {
+    public static final String NAME = "SUM";
+    
+    public SumAggregateFunction() {
+    }
+    
+    // TODO: remove when not required at built-in func register time
+    public SumAggregateFunction(List<Expression> childExpressions){
+        super(childExpressions, null);
+    }
+    
+    public SumAggregateFunction(List<Expression> childExpressions, CountAggregateFunction delegate){
+        super(childExpressions, delegate);
+    }
+    
+    private Aggregator newAggregator(final PDataType type, ColumnModifier columnModifier, ImmutableBytesWritable ptr) {
+        switch( type ) {
+            case DECIMAL:
+                return new DecimalSumAggregator(columnModifier, ptr);
+            case UNSIGNED_DOUBLE:
+            case UNSIGNED_FLOAT:
+            case DOUBLE:
+            case FLOAT:
+                return new DoubleSumAggregator(columnModifier, ptr) {
+                    @Override
+                    protected PDataType getInputDataType() {
+                        return type;
+                    }
+                };
+            default:
+                return new NumberSumAggregator(columnModifier, ptr) {
+                    @Override
+                    protected PDataType getInputDataType() {
+                        return type;
+                    }
+                };
+        }
+    }
+
+    @Override
+    public Aggregator newClientAggregator() {
+        return newAggregator(getDataType(), null, null);
+    }
+    
+    @Override
+    public Aggregator newServerAggregator(Configuration conf) {
+        Expression child = getAggregatorExpression();
+        return newAggregator(child.getDataType(), child.getColumnModifier(), null);
+    }
+    
+    @Override
+    public Aggregator newServerAggregator(Configuration conf, ImmutableBytesWritable ptr) {
+        Expression child = getAggregatorExpression();
+        return newAggregator(child.getDataType(), child.getColumnModifier(), ptr);
+    }
+    
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        if (!super.evaluate(tuple, ptr)) {
+            return false;
+        }
+        if (isConstantExpression()) {
+            PDataType type = getDataType();
+            Object constantValue = ((LiteralExpression)children.get(0)).getValue();
+            if (type == PDataType.DECIMAL) {
+                BigDecimal value = ((BigDecimal)constantValue).multiply((BigDecimal)PDataType.DECIMAL.toObject(ptr, PDataType.LONG));
+                ptr.set(PDataType.DECIMAL.toBytes(value));
+            } else {
+                long constantLongValue = ((Number)constantValue).longValue();
+                long value = constantLongValue * type.getCodec().decodeLong(ptr, null);
+                ptr.set(new byte[type.getByteSize()]);
+                type.getCodec().encodeLong(value, ptr);
+            }
+        }
+        return true;
+    }
+
+    @Override
+    public PDataType getDataType() {
+        switch(super.getDataType()) {
+        case DECIMAL:
+            return PDataType.DECIMAL;
+        case UNSIGNED_FLOAT:
+        case UNSIGNED_DOUBLE:
+        case FLOAT:
+        case DOUBLE:
+            return PDataType.DOUBLE;
+        default:
+            return PDataType.LONG;
+        }
+    }
+
+    @Override
+    public String getName() {
+        return NAME;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/TimeUnit.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/TimeUnit.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/TimeUnit.java
new file mode 100644
index 0000000..7ea5161
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/TimeUnit.java
@@ -0,0 +1,50 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import com.google.common.base.Joiner;
+
+public enum TimeUnit {
+    DAY("day"), 
+    HOUR("hour"), 
+    MINUTE("minute"), 
+    SECOND("second"), 
+    MILLISECOND("millisecond");
+    
+    private String value;
+    
+    private TimeUnit(String value) {
+        this.value = value;
+    }
+    
+    public static final String VALID_VALUES = Joiner.on(", ").join(TimeUnit.values());
+    
+    public static TimeUnit getTimeUnit(String timeUnit) {
+        if(timeUnit == null) {
+            throw new IllegalArgumentException("No time unit value specified. Only a time unit value that belongs to one of these : " + VALID_VALUES + " is allowed.");
+        }
+        for(TimeUnit tu : values()) {
+            if(timeUnit.equalsIgnoreCase(tu.value)) {
+                return tu;
+            }    
+        }
+        throw new IllegalArgumentException("Invalid value of time unit " + timeUnit + ". Only a time unit value that belongs to one of these : " + VALID_VALUES + " is allowed.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToCharFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToCharFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToCharFunction.java
new file mode 100644
index 0000000..d6cdc99
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToCharFunction.java
@@ -0,0 +1,135 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import java.io.*;
+import java.sql.SQLException;
+import java.text.Format;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.io.WritableUtils;
+
+import com.google.common.base.Preconditions;
+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.*;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+
+/**
+ * 
+ * Implementation of the TO_CHAR(&lt;date&gt;/&lt;number&gt;,[&lt;format-string&gt;] built-in function.
+ * The first argument must be of type DATE or TIME or TIMESTAMP or DECIMAL or INTEGER, and the second argument must be a constant string. 
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+@BuiltInFunction(name=ToCharFunction.NAME, nodeClass=ToCharParseNode.class, args={
+    @Argument(allowedTypes={PDataType.TIMESTAMP, PDataType.DECIMAL}),
+    @Argument(allowedTypes={PDataType.VARCHAR},isConstant=true,defaultValue="null") } )
+public class ToCharFunction extends ScalarFunction {
+    public static final String NAME = "TO_CHAR";
+    private String formatString;
+    private Format formatter;
+    private FunctionArgumentType type;
+    
+    public ToCharFunction() {
+    }
+
+    public ToCharFunction(List<Expression> children, FunctionArgumentType type, String formatString, Format formatter) throws SQLException {
+        super(children.subList(0, 1));
+        Preconditions.checkNotNull(formatString);
+        Preconditions.checkNotNull(formatter);
+        Preconditions.checkNotNull(type);
+        this.type = type;
+        this.formatString = formatString;
+        this.formatter = formatter;
+    }
+    
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + formatString.hashCode();
+        result = prime * result + getExpression().hashCode();
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) return true;
+        if (obj == null) return false;
+        if (getClass() != obj.getClass()) return false;
+        ToCharFunction other = (ToCharFunction)obj;
+        if (!getExpression().equals(other.getExpression())) return false;
+        if (!formatString.equals(other.formatString)) return false;
+        return true;
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        Expression expression = getExpression();
+        if (!expression.evaluate(tuple, ptr)) {
+            return false;
+        }
+        PDataType type = expression.getDataType();
+        Object value = formatter.format(type.toObject(ptr, expression.getColumnModifier()));
+        byte[] b = getDataType().toBytes(value);
+        ptr.set(b);
+        return true;
+     }
+
+    @Override
+    public PDataType getDataType() {
+        return PDataType.VARCHAR;
+    }
+
+    @Override
+    public boolean isNullable() {
+        return getExpression().isNullable();
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        super.readFields(input);
+        formatString = WritableUtils.readString(input);
+        type = WritableUtils.readEnum(input, FunctionArgumentType.class);
+        formatter = type.getFormatter(formatString);
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+        super.write(output);
+        WritableUtils.writeString(output, formatString);
+        WritableUtils.writeEnum(output, type);
+    }
+
+    private Expression getExpression() {
+        return children.get(0);
+    }
+
+    @Override
+    public String getName() {
+        return NAME;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToDateFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToDateFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToDateFunction.java
new file mode 100644
index 0000000..5ef0034
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToDateFunction.java
@@ -0,0 +1,133 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import java.io.*;
+import java.sql.SQLException;
+import java.text.Format;
+import java.text.ParseException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.io.WritableUtils;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.parse.*;
+import org.apache.phoenix.parse.FunctionParseNode.Argument;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.DateUtil;
+
+
+/**
+ * 
+ * Implementation of the TO_DATE(<string>,[<format-string>]) built-in function.
+ * The second argument is optional and defaults to the phoenix.query.dateFormat value
+ * from the HBase config. If present it must be a constant string.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+@BuiltInFunction(name=ToDateFunction.NAME, nodeClass=ToDateParseNode.class, args= {@Argument(allowedTypes={PDataType.VARCHAR}),@Argument(allowedTypes={PDataType.VARCHAR},isConstant=true,defaultValue="null")} )
+public class ToDateFunction extends ScalarFunction {
+    public static final String NAME = "TO_DATE";
+    private Format dateParser;
+    private String dateFormat;
+
+    public ToDateFunction() {
+    }
+
+    public ToDateFunction(List<Expression> children, String dateFormat, Format dateParser) throws SQLException {
+        super(children.subList(0, 1));
+        this.dateFormat = dateFormat;
+        this.dateParser = dateParser;
+    }
+    
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + dateFormat.hashCode();
+        result = prime * result + getExpression().hashCode();
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) return true;
+        if (obj == null) return false;
+        if (getClass() != obj.getClass()) return false;
+        ToDateFunction other = (ToDateFunction)obj;
+        if (!getExpression().equals(other.getExpression())) return false;
+        if (!dateFormat.equals(other.dateFormat)) return false;
+        return true;
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        Expression expression = getExpression();
+        if (!expression.evaluate(tuple, ptr) || ptr.getLength() == 0) {
+            return false;
+        }
+        PDataType type = expression.getDataType();
+        String dateStr = (String)type.toObject(ptr, expression.getColumnModifier());
+        try {
+            Object value = dateParser.parseObject(dateStr);
+            byte[] byteValue = getDataType().toBytes(value);
+            ptr.set(byteValue);
+            return true;
+        } catch (ParseException e) {
+            throw new IllegalStateException("to_date('" + dateStr + ")' did not match expected date format of '" + dateFormat + "'.");
+        }
+     }
+
+    @Override
+    public PDataType getDataType() {
+        return PDataType.DATE;
+    }
+
+    @Override
+    public boolean isNullable() {
+        return getExpression().isNullable();
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        super.readFields(input);
+        dateFormat = WritableUtils.readString(input);
+        dateParser = DateUtil.getDateParser(dateFormat);
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+        super.write(output);
+        WritableUtils.writeString(output, dateFormat);
+    }
+
+    private Expression getExpression() {
+        return children.get(0);
+    }
+
+    @Override
+    public String getName() {
+        return NAME;
+    }
+}


[26/51] [partial] Initial commit of master branch from github

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/filter/SkipScanFilter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/filter/SkipScanFilter.java b/phoenix-core/src/main/java/org/apache/phoenix/filter/SkipScanFilter.java
new file mode 100644
index 0000000..ac8f174
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/filter/SkipScanFilter.java
@@ -0,0 +1,523 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValue.Type;
+import org.apache.hadoop.hbase.filter.FilterBase;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import com.google.common.base.Objects;
+import com.google.common.collect.Lists;
+import com.google.common.hash.HashFunction;
+import com.google.common.hash.Hasher;
+import com.google.common.hash.Hashing;
+import org.apache.phoenix.query.KeyRange;
+import org.apache.phoenix.query.KeyRange.Bound;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.RowKeySchema;
+import org.apache.phoenix.schema.ValueSchema.Field;
+import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.ScanUtil;
+
+
+/**
+ * 
+ * Filter that seeks based on CNF containing anded and ored key ranges
+ * 
+ * TODO: figure out when to reset/not reset position array
+ *
+ * @author ryang, jtaylor
+ * @since 0.1
+ */
+public class SkipScanFilter extends FilterBase {
+    private enum Terminate {AT, AFTER};
+    // Conjunctive normal form of or-ed ranges or point lookups
+    private List<List<KeyRange>> slots;
+    // schema of the row key
+    private RowKeySchema schema;
+    // current position for each slot
+    private int[] position;
+    // buffer used for skip hint
+    private int maxKeyLength;
+    private byte[] startKey;
+    private int startKeyLength;
+    private byte[] endKey; 
+    private int endKeyLength;
+    private boolean isDone;
+
+    private final ImmutableBytesWritable ptr = new ImmutableBytesWritable();
+
+    /**
+     * We know that initially the first row will be positioned at or 
+     * after the first possible key.
+     */
+    public SkipScanFilter() {
+    }
+
+    public SkipScanFilter(List<List<KeyRange>> slots, RowKeySchema schema) {
+        int maxKeyLength = getTerminatorCount(schema);
+        for (List<KeyRange> slot : slots) {
+            int maxSlotLength = 0;
+            for (KeyRange range : slot) {
+                int maxRangeLength = Math.max(range.getLowerRange().length, range.getUpperRange().length);
+                if (maxSlotLength < maxRangeLength) {
+                    maxSlotLength = maxRangeLength;
+                }
+            }
+            maxKeyLength += maxSlotLength;
+        }
+        init(slots, schema, maxKeyLength);
+    }
+
+    private void init(List<List<KeyRange>> slots, RowKeySchema schema, int maxKeyLength) {
+        for (List<KeyRange> ranges : slots) {
+            if (ranges.isEmpty()) {
+                throw new IllegalStateException();
+            }
+        }
+        this.slots = slots;
+        this.schema = schema;
+        this.maxKeyLength = maxKeyLength;
+        this.position = new int[slots.size()];
+        startKey = new byte[maxKeyLength];
+        endKey = new byte[maxKeyLength];
+        endKeyLength = 0;
+    }
+
+    // Exposed for testing.
+    List<List<KeyRange>> getSlots() {
+        return slots;
+    }
+
+    @Override
+    public boolean filterAllRemaining() {
+        return isDone;
+    }
+
+    @Override
+    public ReturnCode filterKeyValue(KeyValue kv) {
+        return navigate(kv.getBuffer(), kv.getRowOffset(),kv.getRowLength(),Terminate.AFTER);
+    }
+
+    @Override
+    public KeyValue getNextKeyHint(KeyValue kv) {
+        // TODO: don't allocate new key value every time here if possible
+        return isDone ? null : new KeyValue(startKey, 0, startKeyLength,
+                null, 0, 0, null, 0, 0, HConstants.LATEST_TIMESTAMP, Type.Maximum, null, 0, 0);
+    }
+
+    public boolean hasIntersect(byte[] lowerInclusiveKey, byte[] upperExclusiveKey) {
+        return intersect(lowerInclusiveKey, upperExclusiveKey, null);
+    }
+    /**
+     * Intersect the ranges of this filter with the ranges form by lowerInclusive and upperInclusive
+     * key and filter out the ones that are not included in the region. Return the new intersected
+     * SkipScanFilter or null if there is no intersection.
+     */
+    public SkipScanFilter intersect(byte[] lowerInclusiveKey, byte[] upperExclusiveKey) {
+        List<List<KeyRange>> newSlots = Lists.newArrayListWithCapacity(slots.size());
+        if (intersect(lowerInclusiveKey, upperExclusiveKey, newSlots)) {
+            return new SkipScanFilter(newSlots, schema);
+        }
+        return null;
+    }
+    
+    private boolean intersect(byte[] lowerInclusiveKey, byte[] upperExclusiveKey, List<List<KeyRange>> newSlots) {
+        boolean lowerUnbound = (lowerInclusiveKey.length == 0);
+        Arrays.fill(position, 0);
+        isDone = false;
+        int startPos = 0;
+        int lastSlot = slots.size()-1;
+        if (!lowerUnbound) {
+            // Find the position of the first slot of the lower range
+            schema.next(ptr, 0, schema.iterator(lowerInclusiveKey,ptr));
+            startPos = ScanUtil.searchClosestKeyRangeWithUpperHigherThanPtr(slots.get(0), ptr, 0);
+            // Lower range is past last upper range of first slot, so cannot possibly be in range
+            if (startPos >= slots.get(0).size()) {
+                return false;
+            }
+        }
+        boolean upperUnbound = (upperExclusiveKey.length == 0);
+        int endPos = slots.get(0).size()-1;
+        if (!upperUnbound) {
+            // Find the position of the first slot of the upper range
+            schema.next(ptr, 0, schema.iterator(upperExclusiveKey,ptr));
+            endPos = ScanUtil.searchClosestKeyRangeWithUpperHigherThanPtr(slots.get(0), ptr, startPos);
+            // Upper range lower than first lower range of first slot, so cannot possibly be in range
+            if (endPos == 0 && Bytes.compareTo(upperExclusiveKey, slots.get(0).get(0).getLowerRange()) <= 0) {
+                return false;
+            }
+            // Past last position, so we can include everything from the start position
+            if (endPos >= slots.get(0).size()) {
+                upperUnbound = true;
+                endPos = slots.get(0).size()-1;
+            }
+        }
+        if (!lowerUnbound) {
+            position[0] = startPos;
+            navigate(lowerInclusiveKey, 0, lowerInclusiveKey.length, Terminate.AFTER);
+            if (filterAllRemaining()) {
+                return false;
+            }
+        }
+        if (upperUnbound) {
+            if (newSlots != null) {
+                newSlots.add(slots.get(0).subList(startPos, endPos+1));
+                newSlots.addAll(slots.subList(1, slots.size()));
+            }
+            return true;
+        }
+        int[] lowerPosition = Arrays.copyOf(position, position.length);
+        // Navigate to the upperExclusiveKey, but not past it
+        ReturnCode endCode = navigate(upperExclusiveKey, 0, upperExclusiveKey.length, Terminate.AT);
+        if (endCode == ReturnCode.INCLUDE) {
+            setStartKey();
+            // If the upperExclusiveKey is equal to the start key, we've gone one position too far, since
+            // our upper key is exclusive. In that case, go to the previous key
+            if (Bytes.compareTo(startKey, 0, startKeyLength, upperExclusiveKey, 0, upperExclusiveKey.length) == 0 &&
+                    (previousPosition(lastSlot) < 0 || position[0] < lowerPosition[0])) {
+                // If by backing up one position we have an empty range, then return
+                return false;
+            }
+        } else if (endCode == ReturnCode.SEEK_NEXT_USING_HINT) {
+            // The upperExclusive key is smaller than the slots stored in the position. Check if it's the same position
+            // as the slots for lowerInclusive. If so, there is no intersection.
+            if (Arrays.equals(lowerPosition, position)) {
+                return false;
+            }
+        }
+        // Copy inclusive all positions 
+        for (int i = 0; i <= lastSlot; i++) {
+            List<KeyRange> newRanges = slots.get(i).subList(lowerPosition[i], Math.min(position[i] + 1, slots.get(i).size()));
+            if (newRanges.isEmpty()) {
+                return false;
+            }
+            if (newSlots != null) {
+                newSlots.add(newRanges);
+            }
+            if (position[i] > lowerPosition[i]) {
+                if (newSlots != null) {
+                    newSlots.addAll(slots.subList(i+1, slots.size()));
+                }
+                break;
+            }
+        }
+        return true;
+    }
+
+    private int previousPosition(int i) {
+        while (i >= 0 && --position[i] < 0) {
+            position[i] = slots.get(i).size()-1;
+            i--;
+        }
+        return i;
+    }
+    
+    @edu.umd.cs.findbugs.annotations.SuppressWarnings(
+            value="QBA_QUESTIONABLE_BOOLEAN_ASSIGNMENT", 
+            justification="Assignment designed to work this way.")
+    private ReturnCode navigate(final byte[] currentKey, final int offset, final int length, Terminate terminate) {
+        int nSlots = slots.size();
+        // First check to see if we're in-range until we reach our end key
+        if (endKeyLength > 0) {
+            if (Bytes.compareTo(currentKey, offset, length, endKey, 0, endKeyLength) < 0) {
+                return ReturnCode.INCLUDE;
+            }
+
+            // If key range of last slot is a single key, we can increment our position
+            // since we know we'll be past the current row after including it.
+            if (slots.get(nSlots-1).get(position[nSlots-1]).isSingleKey()) {
+                if (nextPosition(nSlots-1) < 0) {
+                    // Current row will be included, but we have no more
+                    isDone = true;
+                    return ReturnCode.NEXT_ROW;
+                }
+            }
+            else {
+                // Reset the positions to zero from the next slot after the earliest ranged slot, since the
+                // next key could be bigger at this ranged slot, and smaller than the current position of
+                // less significant slots.
+                int earliestRangeIndex = nSlots-1;
+                for (int i = 0; i < nSlots; i++) {
+                    if (!slots.get(i).get(position[i]).isSingleKey()) {
+                        earliestRangeIndex = i;
+                        break;
+                    }
+                }
+                Arrays.fill(position, earliestRangeIndex+1, position.length, 0);
+            }
+        }
+        endKeyLength = 0;
+        
+        // We could have included the previous
+        if (isDone) {
+            return ReturnCode.NEXT_ROW;
+        }
+
+        int i = 0;
+        boolean seek = false;
+        int earliestRangeIndex = nSlots-1;
+        int minOffset = offset;
+        int maxOffset = schema.iterator(currentKey, minOffset, length, ptr);
+        schema.next(ptr, i, maxOffset);
+        while (true) {
+            // Increment to the next range while the upper bound of our current slot is less than our current key
+            while (position[i] < slots.get(i).size() && slots.get(i).get(position[i]).compareUpperToLowerBound(ptr) < 0) {
+                position[i]++;
+            }
+            Arrays.fill(position, i+1, position.length, 0);
+            if (position[i] >= slots.get(i).size()) {
+                // Our current key is bigger than the last range of the current slot.
+                // If navigating after current key, backtrack and increment the key of the previous slot values.
+                // If navigating to current key, just return
+                if (terminate == Terminate.AT) {
+                    return ReturnCode.SEEK_NEXT_USING_HINT;
+                }
+                if (i == 0) {
+                    isDone = true;
+                    return ReturnCode.NEXT_ROW;
+                }
+                // Increment key and backtrack until in range. We know at this point that we'll be
+                // issuing a seek next hint.
+                seek = true;
+                Arrays.fill(position, i, position.length, 0);
+                int j  = i - 1;
+                // If we're positioned at a single key, no need to copy the current key and get the next key .
+                // Instead, just increment to the next key and continue.
+                boolean incremented = false;
+                while (j >= 0 && slots.get(j).get(position[j]).isSingleKey() && (incremented=true) && (position[j] = (position[j] + 1) % slots.get(j).size()) == 0) {
+                    j--;
+                    incremented = false;
+                }
+                if (j < 0) {
+                    isDone = true;
+                    return ReturnCode.NEXT_ROW;
+                }
+                if (incremented) {
+                    // Continue the loop after setting the start key, because our start key maybe smaller than
+                    // the current key, so we'll end up incrementing the start key until it's bigger than the
+                    // current key.
+                    setStartKey();
+                    schema.reposition(ptr, i, j, minOffset, maxOffset);
+                } else {
+                    int currentLength = setStartKey(ptr, minOffset, j+1);
+                    // From here on, we use startKey as our buffer (resetting minOffset and maxOffset)
+                    // We've copied the part of the current key above that we need into startKey
+                    // Reinitialize the iterator to be positioned at previous slot position
+                    minOffset = 0;
+                    maxOffset = startKeyLength;
+                    schema.iterator(startKey, minOffset, maxOffset, ptr, j+1);
+                    // Do nextKey after setting the accessor b/c otherwise the null byte may have
+                    // been incremented causing us not to find it
+                    ByteUtil.nextKey(startKey, currentLength);
+                }
+                i = j;
+            } else if (slots.get(i).get(position[i]).compareLowerToUpperBound(ptr) > 0) {
+                // Our current key is less than the lower range of the current position in the current slot.
+                // Seek to the lower range, since it's bigger than the current key
+                setStartKey(ptr, minOffset, i);
+                return ReturnCode.SEEK_NEXT_USING_HINT;
+            } else { // We're in range, check the next slot
+                if (!slots.get(i).get(position[i]).isSingleKey() && i < earliestRangeIndex) {
+                    earliestRangeIndex = i;
+                }
+                // If we're past the last slot or we know we're seeking to the next (in
+                // which case the previously updated slot was verified to be within the
+                // range, so we don't need to check the rest of the slots. If we were
+                // to check the rest of the slots, we'd get into trouble because we may
+                // have a null byte that was incremented which screws up our schema.next call)
+                if (i == nSlots-1 || seek) {
+                    break;
+                }
+                i++;
+                // If we run out of slots in our key, it means we have a partial key.
+                if (schema.next(ptr, i, maxOffset) == null) {
+                    // If the rest of the slots are checking for IS NULL, then break because
+                    // that's the case (since we don't store trailing nulls).
+                    if (allTrailingNulls(i)) {
+                        break;
+                    }
+                    // Otherwise we seek to the next start key because we're before it now
+                    setStartKey(ptr, minOffset, i);
+                    return ReturnCode.SEEK_NEXT_USING_HINT;
+                }
+            }
+        }
+            
+        if (seek) {
+            return ReturnCode.SEEK_NEXT_USING_HINT;
+        }
+        // Else, we're in range for all slots and can include this row plus all rows 
+        // up to the upper range of our last slot. We do this for ranges and single keys
+        // since we potentially have multiple key values for the same row key.
+        setEndKey(ptr, minOffset, i);
+        return ReturnCode.INCLUDE;
+    }
+
+    private boolean allTrailingNulls(int i) {
+        for (; i < slots.size(); i++) {
+            List<KeyRange> keyRanges = slots.get(i);
+            if (keyRanges.size() != 1) {
+                return false;
+            }
+            KeyRange keyRange = keyRanges.get(0);
+            if (!keyRange.isSingleKey()) {
+                return false;
+            }
+            if (keyRange.getLowerRange().length != 0) {
+                return false;
+            }
+        }
+        return true;
+    }
+    
+    private int nextPosition(int i) {
+        while (i >= 0 && slots.get(i).get(position[i]).isSingleKey() && (position[i] = (position[i] + 1) % slots.get(i).size()) == 0) {
+            i--;
+        }
+        return i;
+    }
+
+    private void setStartKey() {
+        startKeyLength = setKey(Bound.LOWER, startKey, 0, 0);
+    }
+
+    private int setStartKey(ImmutableBytesWritable ptr, int offset, int i) {
+        int length = ptr.getOffset() - offset;
+        startKey = copyKey(startKey, length + this.maxKeyLength, ptr.get(), offset, length);
+        startKeyLength = length;
+        // Add separator byte if we're at the end of the buffer, since trailing separator bytes are stripped
+        if (ptr.getOffset() + ptr.getLength() == offset + length && i-1 > 0 && !schema.getField(i-1).getDataType().isFixedWidth()) {
+            startKey[startKeyLength++] = QueryConstants.SEPARATOR_BYTE;
+        }
+        startKeyLength += setKey(Bound.LOWER, startKey, startKeyLength, i);
+        return length;
+    }
+    
+    private int setEndKey(ImmutableBytesWritable ptr, int offset, int i) {
+        int length = ptr.getOffset() - offset;
+        endKey = copyKey(endKey, length + this.maxKeyLength, ptr.get(), offset, length);
+        endKeyLength = length;
+        endKeyLength += setKey(Bound.UPPER, endKey, length, i);
+        return length;
+    }
+    
+    private int setKey(Bound bound, byte[] key, int keyOffset, int slotStartIndex) {
+        return ScanUtil.setKey(schema, slots, position, bound, key, keyOffset, slotStartIndex, position.length);
+    }
+
+    private static byte[] copyKey(byte[] targetKey, int targetLength, byte[] sourceKey, int offset, int length) {
+        if (targetLength > targetKey.length) {
+            targetKey = new byte[targetLength];
+        }
+        System.arraycopy(sourceKey, offset, targetKey, 0, length);
+        return targetKey;
+    }
+
+    private int getTerminatorCount(RowKeySchema schema) {
+        int nTerminators = 0;
+        for (int i = 0; i < schema.getFieldCount(); i++) {
+            Field field = schema.getField(i);
+            // We won't have a terminator on the last PK column
+            // unless it is variable length and exclusive, but
+            // having the extra byte irregardless won't hurt anything
+            if (!field.getDataType().isFixedWidth()) {
+                nTerminators++;
+            }
+        }
+        return nTerminators;
+    }
+
+    @Override
+    public void readFields(DataInput in) throws IOException {
+        RowKeySchema schema = new RowKeySchema();
+        schema.readFields(in);
+        int maxLength = getTerminatorCount(schema);
+        int andLen = in.readInt();
+        List<List<KeyRange>> slots = Lists.newArrayListWithExpectedSize(andLen);
+        for (int i=0; i<andLen; i++) {
+            int orlen = in.readInt();
+            List<KeyRange> orclause = Lists.newArrayListWithExpectedSize(orlen);
+            slots.add(orclause);
+            int maxSlotLength = 0;
+            for (int j=0; j<orlen; j++) {
+                KeyRange range = new KeyRange();
+                range.readFields(in);
+                if (range.getLowerRange().length > maxSlotLength) {
+                    maxSlotLength = range.getLowerRange().length;
+                }
+                if (range.getUpperRange().length > maxSlotLength) {
+                    maxSlotLength = range.getUpperRange().length;
+                }
+                orclause.add(range);
+            }
+            maxLength += maxSlotLength;
+        }
+        this.init(slots, schema, maxLength);
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+        schema.write(out);
+        out.writeInt(slots.size());
+        for (List<KeyRange> orclause : slots) {
+            out.writeInt(orclause.size());
+            for (KeyRange range : orclause) {
+                range.write(out);
+            }
+        }
+    }
+
+    @Override
+    public int hashCode() {
+        HashFunction hf = Hashing.goodFastHash(32);
+        Hasher h = hf.newHasher();
+        h.putInt(slots.size());
+        for (int i=0; i<slots.size(); i++) {
+            h.putInt(slots.get(i).size());
+            for (int j=0; j<slots.size(); j++) {
+                h.putBytes(slots.get(i).get(j).getLowerRange());
+                h.putBytes(slots.get(i).get(j).getUpperRange());
+            }
+        }
+        return h.hash().asInt();
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (!(obj instanceof SkipScanFilter)) return false;
+        SkipScanFilter other = (SkipScanFilter)obj;
+        return Objects.equal(slots, other.slots) && Objects.equal(schema, other.schema);
+    }
+
+    @Override
+    public String toString() {
+        return "SkipScanFilter "+ slots.toString() ;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/index/BaseIndexCodec.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/BaseIndexCodec.java b/phoenix-core/src/main/java/org/apache/phoenix/index/BaseIndexCodec.java
new file mode 100644
index 0000000..c06becc
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/BaseIndexCodec.java
@@ -0,0 +1,61 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.index;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+
+import org.apache.hadoop.hbase.index.covered.IndexCodec;
+
+/**
+ *
+ */
+public abstract class BaseIndexCodec implements IndexCodec {
+
+  @Override
+  public void initialize(RegionCoprocessorEnvironment env) throws IOException {
+    // noop
+  }
+
+  /**
+   * {@inheritDoc}
+   * <p>
+   * By default, the codec is always enabled. Subclasses should override this method if they want do
+   * decide to index on a per-mutation basis.
+ * @throws IOException 
+   */
+  @Override
+  public boolean isEnabled(Mutation m) throws IOException {
+    return true;
+  }
+
+  /**
+   * {@inheritDoc}
+   * <p>
+   * Assumes each mutation is not in a batch. Subclasses that have different batching behavior
+   * should override this.
+   */
+  @Override
+  public byte[] getBatchId(Mutation m) {
+    return null;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..99d0b20
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
@@ -0,0 +1,810 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.index;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.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.Set;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableUtils;
+
+import com.google.common.base.Predicate;
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+import org.apache.hadoop.hbase.index.ValueGetter;
+import org.apache.hadoop.hbase.index.covered.update.ColumnReference;
+import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.client.KeyValueBuilder;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PColumnFamily;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.PIndexState;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.schema.RowKeySchema;
+import org.apache.phoenix.schema.SaltingUtil;
+import org.apache.phoenix.schema.ValueSchema;
+import org.apache.phoenix.schema.ValueSchema.Field;
+import org.apache.phoenix.util.BitSet;
+import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.IndexUtil;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.TrustedByteArrayOutputStream;
+
+/**
+ * 
+ * Class that builds index row key from data row key and current state of
+ * 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}
+ * Mutation attribute or as a separate RPC call using 
+ * {@link org.apache.phoenix.cache.ServerCacheClient})
+ *
+ * @author jtaylor
+ * @since 2.1.0
+ */
+public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
+    
+    public static IndexMaintainer create(PTable dataTable, PTable index) {
+        if (dataTable.getType() == PTableType.INDEX || index.getType() != PTableType.INDEX || !dataTable.getIndexes().contains(index)) {
+            throw new IllegalArgumentException();
+        }
+        int indexPosOffset = index.getBucketNum() == null ? 0 : 1;
+        int nIndexColumns = index.getColumns().size() - indexPosOffset;
+        int nIndexPKColumns = index.getPKColumns().size() - indexPosOffset;
+        IndexMaintainer maintainer = new IndexMaintainer(
+                dataTable.getRowKeySchema(),
+                dataTable.getBucketNum() != null,
+                index.getPhysicalName().getBytes(), 
+                nIndexColumns,
+                nIndexPKColumns,
+                index.getBucketNum(),
+                dataTable.getColumnFamilies(),
+                index.isWALDisabled());
+        RowKeyMetaData rowKeyMetaData = maintainer.getRowKeyMetaData();
+        for (int i = indexPosOffset; i < index.getPKColumns().size(); i++) {
+            PColumn indexColumn = index.getPKColumns().get(i);
+            int indexPos = i - indexPosOffset;
+            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);
+                rowKeyMetaData.setIndexPkPosition(dataPkPos, indexPos);
+            } else {
+                maintainer.getIndexedColumnTypes().add(column.getDataType());
+                maintainer.getIndexedColumnSizes().add(column.getByteSize());
+                maintainer.getIndexedColumns().add(new ColumnReference(column.getFamilyName().getBytes(), column.getName().getBytes()));
+            }
+            if (indexColumn.getColumnModifier() != null) {
+                rowKeyMetaData.getDescIndexColumnBitSet().set(indexPos);
+            }
+        }
+        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());
+                maintainer.getCoverededColumns().add(new ColumnReference(column.getFamilyName().getBytes(), column.getName().getBytes()));
+            }
+        }
+        maintainer.initCachedState();
+        return maintainer;
+    }
+    
+    public static Iterator<PTable> nonDisabledIndexIterator(Iterator<PTable> indexes) {
+        return Iterators.filter(indexes, new Predicate<PTable>() {
+            @Override
+            public boolean apply(PTable index) {
+                return !PIndexState.DISABLE.equals(index.getIndexState());
+            }
+        });
+    }
+    
+    /**
+     * For client-side to serialize all IndexMaintainers for a given table
+     * @param dataTable data table
+     * @param ptr bytes pointer to hold returned serialized value
+     */
+    public static void serialize(PTable dataTable, ImmutableBytesWritable ptr) {
+        Iterator<PTable> indexes = nonDisabledIndexIterator(dataTable.getIndexes().iterator());
+        if (dataTable.isImmutableRows() || !indexes.hasNext()) {
+            ptr.set(ByteUtil.EMPTY_BYTE_ARRAY);
+            return;
+        }
+        int nIndexes = 0;
+        int estimatedSize = dataTable.getRowKeySchema().getEstimatedByteSize() + 2;
+        while (indexes.hasNext()) {
+            nIndexes++;
+            PTable index = indexes.next();
+            estimatedSize += index.getIndexMaintainer(dataTable).getEstimatedByteSize();
+        }
+        TrustedByteArrayOutputStream stream = new TrustedByteArrayOutputStream(estimatedSize + 1);
+        DataOutput output = new DataOutputStream(stream);
+        try {
+            // Encode data table salting in sign of number of indexes
+            WritableUtils.writeVInt(output, nIndexes * (dataTable.getBucketNum() == null ? 1 : -1));
+            // Write out data row key schema once, since it's the same for all index maintainers
+            dataTable.getRowKeySchema().write(output);
+            indexes = nonDisabledIndexIterator(dataTable.getIndexes().iterator());
+            while (indexes.hasNext()) {
+                    indexes.next().getIndexMaintainer(dataTable).write(output);
+            }
+        } catch (IOException e) {
+            throw new RuntimeException(e); // Impossible
+        }
+        ptr.set(stream.getBuffer(), 0, stream.size());
+    }
+    
+    public static List<IndexMaintainer> deserialize(ImmutableBytesWritable metaDataPtr,
+            KeyValueBuilder builder) {
+        return deserialize(metaDataPtr.get(), metaDataPtr.getOffset(), metaDataPtr.getLength(),
+            builder);
+    }
+    
+    public static List<IndexMaintainer> deserialize(byte[] buf, KeyValueBuilder builder) {
+        return deserialize(buf, 0, buf.length, builder);
+    }
+
+    public static List<IndexMaintainer> deserialize(byte[] buf, int offset, int length,
+            KeyValueBuilder builder) {
+        ByteArrayInputStream stream = new ByteArrayInputStream(buf, offset, length);
+        DataInput input = new DataInputStream(stream);
+        List<IndexMaintainer> maintainers = Collections.emptyList();
+        try {
+            int size = WritableUtils.readVInt(input);
+            boolean isDataTableSalted = size < 0;
+            size = Math.abs(size);
+            RowKeySchema rowKeySchema = new RowKeySchema();
+            rowKeySchema.readFields(input);
+            maintainers = Lists.newArrayListWithExpectedSize(size);
+            for (int i = 0; i < size; i++) {
+                IndexMaintainer maintainer = new IndexMaintainer(rowKeySchema, isDataTableSalted);
+                maintainer.readFields(input);
+                maintainer.setKvBuilder(builder);
+                maintainers.add(maintainer);
+            }
+        } catch (IOException e) {
+            throw new RuntimeException(e); // Impossible
+        }
+        return maintainers;
+    }
+
+    private Set<ColumnReference> indexedColumns;
+    private Set<ColumnReference> coveredColumns;
+    private Set<ColumnReference> allColumns;
+    private List<PDataType> indexedColumnTypes;
+    private List<Integer> indexedColumnByteSizes;
+    private RowKeyMetaData rowKeyMetaData;
+    private byte[] indexTableName;
+    private int nIndexSaltBuckets;
+    private byte[] dataEmptyKeyValueCF;
+    private int nDataCFs;
+    private boolean indexWALDisabled;
+
+    // Transient state
+    private final boolean isDataTableSalted;
+    private final RowKeySchema dataRowKeySchema;
+    
+    private ImmutableBytesPtr emptyKeyValueCF;
+    private List<ImmutableBytesPtr> indexQualifiers;
+    private int estimatedIndexRowKeyBytes;
+    private int[] dataPkPosition;
+    private int maxTrailingNulls;
+    private ColumnReference dataEmptyKeyValueRef;
+    private KeyValueBuilder kvBuilder;
+    
+    private IndexMaintainer(RowKeySchema dataRowKeySchema, boolean isDataTableSalted) {
+        this.dataRowKeySchema = dataRowKeySchema;
+        this.isDataTableSalted = isDataTableSalted;
+    }
+
+    private IndexMaintainer(RowKeySchema dataRowKeySchema, boolean isDataTableSalted, byte[] indexTableName,
+            int nIndexColumns, int nIndexPKColumns, Integer nIndexSaltBuckets, List<PColumnFamily> cfs, boolean indexWALDisabled) {
+        this(dataRowKeySchema, isDataTableSalted);
+        int nDataPKColumns = dataRowKeySchema.getFieldCount() - (isDataTableSalted ? 1 : 0);
+        this.indexTableName = indexTableName;
+        this.indexedColumns = Sets.newLinkedHashSetWithExpectedSize(nIndexPKColumns-nDataPKColumns);
+        this.indexedColumnTypes = Lists.<PDataType>newArrayListWithExpectedSize(nIndexPKColumns-nDataPKColumns);
+        this.indexedColumnByteSizes = Lists.<Integer>newArrayListWithExpectedSize(nIndexPKColumns-nDataPKColumns);
+        this.coveredColumns = Sets.newLinkedHashSetWithExpectedSize(nIndexColumns-nIndexPKColumns);
+        this.allColumns = Sets.newLinkedHashSetWithExpectedSize(nDataPKColumns + nIndexColumns);
+        this.allColumns.addAll(indexedColumns);
+        this.allColumns.addAll(coveredColumns);
+        this.rowKeyMetaData = newRowKeyMetaData(nIndexPKColumns);
+        this.nIndexSaltBuckets  = nIndexSaltBuckets == null ? 0 : nIndexSaltBuckets;
+        this.dataEmptyKeyValueCF = SchemaUtil.getEmptyColumnFamily(cfs);
+        this.nDataCFs = cfs.size();
+        this.indexWALDisabled = indexWALDisabled;
+    }
+
+    public byte[] buildRowKey(ValueGetter valueGetter, ImmutableBytesWritable rowKeyPtr)  {
+        ImmutableBytesWritable ptr = new ImmutableBytesWritable();
+        TrustedByteArrayOutputStream stream = new TrustedByteArrayOutputStream(estimatedIndexRowKeyBytes);
+        DataOutput output = new DataOutputStream(stream);
+        try {
+            if (nIndexSaltBuckets > 0) {
+                output.write(0); // will be set at end to index salt byte
+            }
+            
+            // The dataRowKeySchema includes the salt byte field,
+            // so we must adjust for that here.
+            int dataPosOffset = isDataTableSalted ? 1 : 0 ;
+            int nIndexedColumns = getIndexPkColumnCount();
+            int[][] dataRowKeyLocator = new int[2][nIndexedColumns];
+            // Skip data table salt byte
+            int maxRowKeyOffset = rowKeyPtr.getOffset() + rowKeyPtr.getLength();
+            dataRowKeySchema.iterator(rowKeyPtr, ptr, dataPosOffset);
+            // Write index row key
+            for (int i = dataPosOffset; i < dataRowKeySchema.getFieldCount(); i++) {
+                Boolean hasValue=dataRowKeySchema.next(ptr, i, maxRowKeyOffset);
+                int pos = rowKeyMetaData.getIndexPkPosition(i-dataPosOffset);
+                if (Boolean.TRUE.equals(hasValue)) {
+                    dataRowKeyLocator[0][pos] = ptr.getOffset();
+                    dataRowKeyLocator[1][pos] = ptr.getLength();
+                } else {
+                    dataRowKeyLocator[0][pos] = 0;
+                    dataRowKeyLocator[1][pos] = 0;
+                }
+            }
+            BitSet descIndexColumnBitSet = rowKeyMetaData.getDescIndexColumnBitSet();
+            int j = 0;
+            Iterator<ColumnReference> iterator = indexedColumns.iterator();
+            for (int i = 0; i < nIndexedColumns; i++) {
+                PDataType dataColumnType;
+                boolean isNullable = true;
+                boolean isDataColumnInverted = false;
+                ColumnModifier dataColumnModifier = null;
+                if (dataPkPosition[i] == -1) {
+                    dataColumnType = indexedColumnTypes.get(j);
+                    ImmutableBytesPtr value = valueGetter.getLatestValue(iterator.next());
+                    if (value == null) {
+                        ptr.set(ByteUtil.EMPTY_BYTE_ARRAY);
+                    } else {
+                        ptr.set(value.copyBytesIfNecessary());
+                    }
+                    j++;
+               } else {
+                   Field field = dataRowKeySchema.getField(dataPkPosition[i]);
+                    dataColumnType = field.getDataType();
+                    ptr.set(rowKeyPtr.get(), dataRowKeyLocator[0][i], dataRowKeyLocator[1][i]);
+                    dataColumnModifier = field.getColumnModifier();
+                    isDataColumnInverted = dataColumnModifier != null;
+                    isNullable = field.isNullable();
+                }
+                PDataType indexColumnType = IndexUtil.getIndexColumnDataType(isNullable, dataColumnType);
+                boolean isBytesComparable = dataColumnType.isBytesComparableWith(indexColumnType) ;
+                if (isBytesComparable && isDataColumnInverted == descIndexColumnBitSet.get(i)) {
+                    output.write(ptr.get(), ptr.getOffset(), ptr.getLength());
+                } else {
+                    if (!isBytesComparable)  {
+                        indexColumnType.coerceBytes(ptr, dataColumnType, dataColumnModifier, null);
+                    }
+                    if (descIndexColumnBitSet.get(i) != isDataColumnInverted) {
+                        writeInverted(ptr.get(), ptr.getOffset(), ptr.getLength(), output);
+                    } else {
+                        output.write(ptr.get(), ptr.getOffset(), ptr.getLength());
+                    }
+                }
+                if (!indexColumnType.isFixedWidth()) {
+                    output.writeByte(QueryConstants.SEPARATOR_BYTE);
+                }
+            }
+            int length = stream.size();
+            int minLength = length - maxTrailingNulls;
+            byte[] indexRowKey = stream.getBuffer();
+            // Remove trailing nulls
+            while (length > minLength && indexRowKey[length-1] == QueryConstants.SEPARATOR_BYTE) {
+                length--;
+            }
+            if (nIndexSaltBuckets > 0) {
+                // Set salt byte
+                byte saltByte = SaltingUtil.getSaltingByte(indexRowKey, SaltingUtil.NUM_SALTING_BYTES, length-SaltingUtil.NUM_SALTING_BYTES, nIndexSaltBuckets);
+                indexRowKey[0] = saltByte;
+            }
+            return indexRowKey.length == length ? indexRowKey : Arrays.copyOf(indexRowKey, length);
+        } catch (IOException e) {
+            throw new RuntimeException(e); // Impossible
+        } finally {
+            try {
+                stream.close();
+            } catch (IOException e) {
+                throw new RuntimeException(e); // Impossible
+            }
+        }
+    }
+
+    @SuppressWarnings("deprecation")
+    public Put buildUpdateMutation(ValueGetter valueGetter, ImmutableBytesWritable dataRowKeyPtr, long ts) throws IOException {
+        Put put = null;
+        // New row being inserted: add the empty key value
+        if (valueGetter.getLatestValue(dataEmptyKeyValueRef) == null) {
+            byte[] indexRowKey = this.buildRowKey(valueGetter, dataRowKeyPtr);
+            put = new Put(indexRowKey);
+            // add the keyvalue for the empty row
+            put.add(this.kvBuilder.buildPut(new ImmutableBytesPtr(indexRowKey),
+                this.getEmptyKeyValueFamily(), QueryConstants.EMPTY_COLUMN_BYTES_PTR, ts,
+                ByteUtil.EMPTY_BYTE_ARRAY_PTR));
+            put.setWriteToWAL(!indexWALDisabled);
+        }
+        int i = 0;
+        for (ColumnReference ref : this.getCoverededColumns()) {
+            ImmutableBytesPtr cq = this.indexQualifiers.get(i++);
+            ImmutableBytesPtr value = valueGetter.getLatestValue(ref);
+            byte[] indexRowKey = this.buildRowKey(valueGetter, dataRowKeyPtr);
+            ImmutableBytesPtr rowKey = new ImmutableBytesPtr(indexRowKey);
+            if (value != null) {
+                if (put == null) {
+                    put = new Put(indexRowKey);
+                    put.setWriteToWAL(!indexWALDisabled);
+                }
+                //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
+                put.add(this.kvBuilder.buildPut(rowKey, ref.getFamilyWritable(), cq, ts, value));
+            }
+        }
+        return put;
+    }
+
+    public Put buildUpdateMutation(ValueGetter valueGetter, ImmutableBytesWritable dataRowKeyPtr) throws IOException {
+        return buildUpdateMutation(valueGetter, dataRowKeyPtr, HConstants.LATEST_TIMESTAMP);
+    }
+    
+    public Delete buildDeleteMutation(ValueGetter valueGetter, ImmutableBytesWritable dataRowKeyPtr, Collection<KeyValue> pendingUpdates) throws IOException {
+        return buildDeleteMutation(valueGetter, dataRowKeyPtr, pendingUpdates, HConstants.LATEST_TIMESTAMP);
+    }
+    
+    public boolean isRowDeleted(Collection<KeyValue> pendingUpdates) {
+        int nDeleteCF = 0;
+        for (KeyValue kv : pendingUpdates) {
+            if (kv.getType() == KeyValue.Type.DeleteFamily.getCode()) {
+                nDeleteCF++;
+                boolean isEmptyCF = Bytes.compareTo(kv.getFamily(), dataEmptyKeyValueCF) == 0;
+                // This is what a delete looks like on the client side for immutable indexing...
+                if (isEmptyCF) {
+                    return true;
+                }
+            }
+        }
+        // This is what a delete looks like on the server side for mutable indexing...
+        return nDeleteCF == this.nDataCFs;
+    }
+    
+    private boolean hasIndexedColumnChanged(ValueGetter oldState, Collection<KeyValue> pendingUpdates) throws IOException {
+        if (pendingUpdates.isEmpty()) {
+            return false;
+        }
+        Map<ColumnReference,KeyValue> newState = Maps.newHashMapWithExpectedSize(pendingUpdates.size()); 
+        for (KeyValue kv : pendingUpdates) {
+            newState.put(new ColumnReference(kv.getFamily(), kv.getQualifier()), kv);
+        }
+        for (ColumnReference ref : indexedColumns) {
+            KeyValue newValue = newState.get(ref);
+            if (newValue != null) { // Indexed column was potentially changed
+                ImmutableBytesPtr oldValue = oldState.getLatestValue(ref);
+                // If there was no old value or the old value is different than the new value, the index row needs to be deleted
+                if (oldValue == null || 
+                        Bytes.compareTo(oldValue.get(), oldValue.getOffset(), oldValue.getLength(), 
+                                                   newValue.getBuffer(), newValue.getValueOffset(), newValue.getValueLength()) != 0){
+                    return true;
+                }
+            }
+        }
+        return false;
+    }
+
+    /**
+     * Used for immutable indexes that only index PK column values. In that case, we can handle a data row deletion,
+     * since we can build the corresponding index row key.
+     */
+    public Delete buildDeleteMutation(ImmutableBytesWritable dataRowKeyPtr, long ts) throws IOException {
+        return buildDeleteMutation(null, dataRowKeyPtr, Collections.<KeyValue>emptyList(), ts);
+    }
+    
+    @SuppressWarnings("deprecation")
+    public Delete buildDeleteMutation(ValueGetter oldState, ImmutableBytesWritable dataRowKeyPtr, Collection<KeyValue> pendingUpdates, long ts) throws IOException {
+        byte[] indexRowKey = this.buildRowKey(oldState, dataRowKeyPtr);
+        // Delete the entire row if any of the indexed columns changed
+        if (oldState == null || isRowDeleted(pendingUpdates) || hasIndexedColumnChanged(oldState, pendingUpdates)) { // Deleting the entire row
+            Delete delete = new Delete(indexRowKey, ts, null);
+            delete.setWriteToWAL(!indexWALDisabled);
+            return delete;
+        }
+        Delete delete = null;
+        // Delete columns for missing key values
+        for (KeyValue kv : pendingUpdates) {
+            if (kv.getType() != KeyValue.Type.Put.getCode()) {
+                ColumnReference ref = new ColumnReference(kv.getFamily(), kv.getQualifier());
+                if (coveredColumns.contains(ref)) {
+                    if (delete == null) {
+                        delete = new Delete(indexRowKey);                    
+                        delete.setWriteToWAL(!indexWALDisabled);
+                    }
+                    delete.deleteColumns(ref.getFamily(), IndexUtil.getIndexColumnName(ref.getFamily(), ref.getQualifier()), ts);
+                }
+            }
+        }
+        return delete;
+  }
+
+    public byte[] getIndexTableName() {
+        return indexTableName;
+    }
+    
+    public Set<ColumnReference> getCoverededColumns() {
+        return coveredColumns;
+    }
+
+    public Set<ColumnReference> getIndexedColumns() {
+        return indexedColumns;
+    }
+
+    public Set<ColumnReference> getAllColumns() {
+        return allColumns;
+    }
+    
+    private ImmutableBytesPtr getEmptyKeyValueFamily() {
+        // Since the metadata of an index table will never change,
+        // we can infer this based on the family of the first covered column
+        // If if there are no covered columns, we know it's our default name
+        return emptyKeyValueCF;
+    }
+
+    private RowKeyMetaData getRowKeyMetaData() {
+        return rowKeyMetaData;
+    }
+    
+    private List<Integer> getIndexedColumnSizes() {
+        return indexedColumnByteSizes;
+    }
+
+    private List<PDataType> getIndexedColumnTypes() {
+        return indexedColumnTypes;
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        nIndexSaltBuckets = WritableUtils.readVInt(input);
+        int nIndexedColumns = WritableUtils.readVInt(input);
+        indexedColumns = Sets.newLinkedHashSetWithExpectedSize(nIndexedColumns);
+        for (int i = 0; i < nIndexedColumns; i++) {
+            byte[] cf = Bytes.readByteArray(input);
+            byte[] cq = Bytes.readByteArray(input);
+            indexedColumns.add(new ColumnReference(cf,cq));
+        }
+        indexedColumnTypes = Lists.newArrayListWithExpectedSize(nIndexedColumns);
+        for (int i = 0; i < nIndexedColumns; i++) {
+            PDataType type = PDataType.values()[WritableUtils.readVInt(input)];
+            indexedColumnTypes.add(type);
+        }
+        indexedColumnByteSizes = Lists.newArrayListWithExpectedSize(nIndexedColumns);
+        for (int i = 0; i < nIndexedColumns; i++) {
+            int byteSize = WritableUtils.readVInt(input);
+            indexedColumnByteSizes.add(byteSize == 0 ? null : Integer.valueOf(byteSize));
+        }
+        int nCoveredColumns = WritableUtils.readVInt(input);
+        coveredColumns = Sets.newLinkedHashSetWithExpectedSize(nCoveredColumns);
+        for (int i = 0; i < nCoveredColumns; i++) {
+            byte[] cf = Bytes.readByteArray(input);
+            byte[] cq = Bytes.readByteArray(input);
+            coveredColumns.add(new ColumnReference(cf,cq));
+        }
+        indexTableName = Bytes.readByteArray(input);
+        dataEmptyKeyValueCF = Bytes.readByteArray(input);
+        rowKeyMetaData = newRowKeyMetaData();
+        rowKeyMetaData.readFields(input);
+        int nDataCFs = WritableUtils.readVInt(input);
+        // Encode indexWALDisabled in nDataCFs
+        indexWALDisabled = nDataCFs < 0;
+        this.nDataCFs = Math.abs(nDataCFs) - 1;
+        
+        initCachedState();
+    }
+    
+    @Override
+    public void write(DataOutput output) throws IOException {
+        WritableUtils.writeVInt(output, nIndexSaltBuckets);
+        WritableUtils.writeVInt(output, indexedColumns.size());
+        for (ColumnReference ref : indexedColumns) {
+            Bytes.writeByteArray(output, ref.getFamily());
+            Bytes.writeByteArray(output, ref.getQualifier());
+        }
+        for (int i = 0; i < indexedColumnTypes.size(); i++) {
+            PDataType type = indexedColumnTypes.get(i);
+            WritableUtils.writeVInt(output, type.ordinal());
+        }
+        for (int i = 0; i < indexedColumnByteSizes.size(); i++) {
+            Integer byteSize = indexedColumnByteSizes.get(i);
+            WritableUtils.writeVInt(output, byteSize == null ? 0 : byteSize);
+        }
+        WritableUtils.writeVInt(output, coveredColumns.size());
+        for (ColumnReference ref : coveredColumns) {
+            Bytes.writeByteArray(output, ref.getFamily());
+            Bytes.writeByteArray(output, ref.getQualifier());
+        }
+        Bytes.writeByteArray(output, indexTableName);
+        Bytes.writeByteArray(output, dataEmptyKeyValueCF);
+        rowKeyMetaData.write(output);
+        // Encode indexWALDisabled in nDataCFs
+        WritableUtils.writeVInt(output, (nDataCFs + 1) * (indexWALDisabled ? -1 : 1));
+    }
+
+    private int estimateIndexRowKeyByteSize() {
+        int estimatedIndexRowKeyBytes = dataRowKeySchema.getEstimatedValueLength() + (nIndexSaltBuckets == 0 ?  0 : SaltingUtil.NUM_SALTING_BYTES);
+        for (Integer byteSize : indexedColumnByteSizes) {
+            estimatedIndexRowKeyBytes += (byteSize == null ? ValueSchema.ESTIMATED_VARIABLE_LENGTH_SIZE : byteSize);
+        }
+        return estimatedIndexRowKeyBytes;
+   }
+    
+    /**
+     * Init calculated state reading/creating
+     */
+    private void initCachedState() {
+        if (coveredColumns.isEmpty()) {
+            emptyKeyValueCF = QueryConstants.EMPTY_COLUMN_BYTES_PTR;
+        } else {
+            emptyKeyValueCF = new ImmutableBytesPtr(coveredColumns.iterator().next().getFamily());
+        }
+        dataEmptyKeyValueRef =
+                new ColumnReference(emptyKeyValueCF.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())));
+        }
+        estimatedIndexRowKeyBytes = estimateIndexRowKeyByteSize();
+
+        this.allColumns = Sets.newLinkedHashSetWithExpectedSize(indexedColumns.size() + coveredColumns.size());
+        allColumns.addAll(indexedColumns);
+        allColumns.addAll(coveredColumns);
+        
+        int dataPkOffset = isDataTableSalted ? 1 : 0;
+        int nIndexPkColumns = getIndexPkColumnCount();
+        dataPkPosition = new int[nIndexPkColumns];
+        Arrays.fill(dataPkPosition, -1);
+        for (int i = dataPkOffset; i < dataRowKeySchema.getFieldCount(); i++) {
+            int dataPkPosition = rowKeyMetaData.getIndexPkPosition(i-dataPkOffset);
+            this.dataPkPosition[dataPkPosition] = i;
+        }
+        
+        // Calculate the max number of trailing nulls that we should get rid of after building the index row key.
+        // We only get rid of nulls for variable length types, so we have to be careful to consider the type of the
+        // index table, not the data type of the data table
+        int indexedColumnTypesPos = indexedColumnTypes.size()-1;
+        int indexPkPos = nIndexPkColumns-1;
+        while (indexPkPos >= 0) {
+            int dataPkPos = dataPkPosition[indexPkPos];
+            boolean isDataNullable;
+            PDataType dataType;
+            if (dataPkPos == -1) {
+                isDataNullable = true;
+                dataType = indexedColumnTypes.get(indexedColumnTypesPos--);
+            } else {
+                Field dataField = dataRowKeySchema.getField(dataPkPos);
+                dataType = dataField.getDataType();
+                isDataNullable = dataField.isNullable();
+            }
+            PDataType indexDataType = IndexUtil.getIndexColumnDataType(isDataNullable, dataType);
+            if (indexDataType.isFixedWidth()) {
+                break;
+            }
+            indexPkPos--;
+        }
+        maxTrailingNulls = nIndexPkColumns-indexPkPos-1;
+    }
+
+    public void setKvBuilder(KeyValueBuilder builder) {
+        this.kvBuilder = builder;
+    }
+
+    private int getIndexPkColumnCount() {
+        return dataRowKeySchema.getFieldCount() + indexedColumns.size() - (isDataTableSalted ? 1 : 0);
+    }
+    
+    private RowKeyMetaData newRowKeyMetaData() {
+        return getIndexPkColumnCount() < 0xFF ? new ByteSizeRowKeyMetaData() : new IntSizedRowKeyMetaData();
+    }
+
+    private RowKeyMetaData newRowKeyMetaData(int capacity) {
+        return capacity < 0xFF ? new ByteSizeRowKeyMetaData(capacity) : new IntSizedRowKeyMetaData(capacity);
+    }
+
+    public int getEstimatedByteSize() {
+        int size = WritableUtils.getVIntSize(nIndexSaltBuckets);
+        size += WritableUtils.getVIntSize(indexedColumns.size());
+        for (ColumnReference ref : indexedColumns) {
+            size += WritableUtils.getVIntSize(ref.getFamily().length);
+            size += ref.getFamily().length;
+            size += WritableUtils.getVIntSize(ref.getQualifier().length);
+            size += ref.getQualifier().length;
+        }
+        size += indexedColumnTypes.size();
+        size += indexedColumnByteSizes.size();
+        size += WritableUtils.getVIntSize(coveredColumns.size());
+        for (ColumnReference ref : coveredColumns) {
+            size += WritableUtils.getVIntSize(ref.getFamily().length);
+            size += ref.getFamily().length;
+            size += WritableUtils.getVIntSize(ref.getQualifier().length);
+            size += ref.getQualifier().length;
+        }
+        size += indexTableName.length + WritableUtils.getVIntSize(indexTableName.length);
+        size += rowKeyMetaData.getByteSize();
+        size += dataEmptyKeyValueCF.length + + WritableUtils.getVIntSize(dataEmptyKeyValueCF.length);
+        size += WritableUtils.getVIntSize(nDataCFs+1);
+        return size;
+    }
+    
+    private static void writeInverted(byte[] buf, int offset, int length, DataOutput output) throws IOException {
+        for (int i = offset; i < offset + length; i++) {
+            byte b = ColumnModifier.SORT_DESC.apply(buf[i]);
+            output.write(b);
+        }
+    }
+    
+    private abstract class RowKeyMetaData implements Writable {
+        private BitSet descIndexColumnBitSet;
+        
+        private RowKeyMetaData() {
+        }
+        
+        private RowKeyMetaData(int nIndexedColumns) {
+            descIndexColumnBitSet = BitSet.withCapacity(nIndexedColumns);
+      }
+        
+        protected int getByteSize() {
+            return BitSet.getByteSize(getIndexPkColumnCount()) * 3;
+        }
+        
+        protected abstract int getIndexPkPosition(int dataPkPosition);
+        protected abstract int setIndexPkPosition(int dataPkPosition, int indexPkPosition);
+        
+        @Override
+        public void readFields(DataInput input) throws IOException {
+            int length = getIndexPkColumnCount();
+            descIndexColumnBitSet = BitSet.read(input, length);
+        }
+        
+        @Override
+        public void write(DataOutput output) throws IOException {
+            int length = getIndexPkColumnCount();
+            BitSet.write(output, descIndexColumnBitSet, length);
+        }
+
+        private BitSet getDescIndexColumnBitSet() {
+            return descIndexColumnBitSet;
+        }
+    }
+    
+    private static int BYTE_OFFSET = 127;
+    
+    private class ByteSizeRowKeyMetaData extends RowKeyMetaData {
+        private byte[] indexPkPosition;
+        
+        private ByteSizeRowKeyMetaData() {
+        }
+
+        private ByteSizeRowKeyMetaData(int nIndexedColumns) {
+            super(nIndexedColumns);
+            this.indexPkPosition = new byte[nIndexedColumns];
+        }
+        
+        @Override
+        protected int getIndexPkPosition(int dataPkPosition) {
+            // Use offset for byte so that we can get full range of 0 - 255
+            // We use -128 as marker for a non row key index column,
+            // that's why our offset if 127 instead of 128
+            return this.indexPkPosition[dataPkPosition] + BYTE_OFFSET;
+        }
+
+        @Override
+        protected int setIndexPkPosition(int dataPkPosition, int indexPkPosition) {
+            return this.indexPkPosition[dataPkPosition] = (byte)(indexPkPosition - BYTE_OFFSET);
+        }
+
+        @Override
+        public void write(DataOutput output) throws IOException {
+            super.write(output);
+            output.write(indexPkPosition);
+        }
+
+        @Override
+        protected int getByteSize() {
+            return super.getByteSize() + indexPkPosition.length;
+        }
+
+        @Override
+        public void readFields(DataInput input) throws IOException {
+            super.readFields(input);
+            this.indexPkPosition = new byte[getIndexPkColumnCount()];
+            input.readFully(indexPkPosition);
+        }
+    }
+    
+    private class IntSizedRowKeyMetaData extends RowKeyMetaData {
+        private int[] indexPkPosition;
+        
+        private IntSizedRowKeyMetaData() {
+        }
+
+        private IntSizedRowKeyMetaData(int nIndexedColumns) {
+            super(nIndexedColumns);
+            this.indexPkPosition = new int[nIndexedColumns];
+        }
+        
+        @Override
+        protected int getIndexPkPosition(int dataPkPosition) {
+            return this.indexPkPosition[dataPkPosition];
+        }
+
+        @Override
+        protected int setIndexPkPosition(int dataPkPosition, int indexPkPosition) {
+            return this.indexPkPosition[dataPkPosition] = indexPkPosition;
+        }
+        
+        @Override
+        public void write(DataOutput output) throws IOException {
+            super.write(output);
+            for (int i = 0; i < indexPkPosition.length; i++) {
+                output.writeInt(indexPkPosition[i]);
+            }
+        }
+
+        @Override
+        protected int getByteSize() {
+            return super.getByteSize() + indexPkPosition.length * Bytes.SIZEOF_INT;
+        }
+
+        @Override
+        public void readFields(DataInput input) throws IOException {
+            super.readFields(input);
+            this.indexPkPosition = new int[getIndexPkColumnCount()];
+            for (int i = 0; i < indexPkPosition.length; i++) {
+                indexPkPosition[i] = input.readInt();
+            }
+        }
+    }
+
+    @Override
+    public Iterator<ColumnReference> iterator() {
+        return allColumns.iterator();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..97ba55b
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMetaDataCacheClient.java
@@ -0,0 +1,98 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.index;
+
+import static org.apache.phoenix.query.QueryServices.INDEX_MUTATE_BATCH_SIZE_THRESHOLD_ATTRIB;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.cache.ServerCacheClient;
+import org.apache.phoenix.cache.ServerCacheClient.ServerCache;
+import org.apache.phoenix.compile.ScanRanges;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.join.MaxServerCacheSizeExceededException;
+import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.ScanUtil;
+
+public class IndexMetaDataCacheClient {
+
+    private final ServerCacheClient serverCache;
+    private TableRef cacheUsingTableRef;
+    
+    /**
+     * Construct client used to send index metadata to each region server
+     * for caching during batched put for secondary index maintenance.
+     * @param connection the client connection
+     * @param cacheUsingTableRef table ref to table that will use the cache during its scan
+     */
+    public IndexMetaDataCacheClient(PhoenixConnection connection, TableRef cacheUsingTableRef) {
+        serverCache = new ServerCacheClient(connection);
+        this.cacheUsingTableRef = cacheUsingTableRef;
+    }
+
+    /**
+     * Determines whether or not to use the IndexMetaDataCache to send the index metadata
+     * to the region servers. The alternative is to just set the index metadata as an attribute on
+     * the mutations.
+     * @param connection 
+     * @param mutations the list of mutations that will be sent in a batch to server
+     * @param indexMetaDataByteLength length in bytes of the index metadata cache
+     */
+    public static boolean useIndexMetadataCache(PhoenixConnection connection, List<Mutation> mutations, int indexMetaDataByteLength) {
+        ReadOnlyProps props = connection.getQueryServices().getProps();
+        int threshold = props.getInt(INDEX_MUTATE_BATCH_SIZE_THRESHOLD_ATTRIB, QueryServicesOptions.DEFAULT_INDEX_MUTATE_BATCH_SIZE_THRESHOLD);
+        return (indexMetaDataByteLength > ServerCacheClient.UUID_LENGTH && mutations.size() > threshold);
+    }
+    
+    /**
+     * Send the index metadata cahce to all region servers for regions that will handle the mutations.
+     * @return client-side {@link ServerCache} representing the added index metadata cache
+     * @throws SQLException 
+     * @throws MaxServerCacheSizeExceededException if size of hash cache exceeds max allowed
+     * size
+     */
+    public ServerCache addIndexMetadataCache(List<Mutation> mutations, ImmutableBytesWritable ptr) throws SQLException {
+        /**
+         * Serialize and compress hashCacheTable
+         */
+        return serverCache.addServerCache(ScanUtil.newScanRanges(mutations), ptr, new IndexMetaDataCacheFactory(), cacheUsingTableRef);
+    }
+    
+    
+    /**
+     * Send the index metadata cahce to all region servers for regions that will handle the mutations.
+     * @return client-side {@link ServerCache} representing the added index metadata cache
+     * @throws SQLException 
+     * @throws MaxServerCacheSizeExceededException if size of hash cache exceeds max allowed
+     * size
+     */
+    public ServerCache addIndexMetadataCache(ScanRanges ranges, ImmutableBytesWritable ptr) throws SQLException {
+        /**
+         * Serialize and compress hashCacheTable
+         */
+        return serverCache.addServerCache(ranges, ptr, new IndexMetaDataCacheFactory(), cacheUsingTableRef);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..3be8833
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMetaDataCacheFactory.java
@@ -0,0 +1,66 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.index;
+
+import java.io.Closeable;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.cache.IndexMetaDataCache;
+import org.apache.phoenix.client.GenericKeyValueBuilder;
+import org.apache.phoenix.coprocessor.ServerCachingProtocol.ServerCacheFactory;
+import org.apache.phoenix.memory.MemoryManager.MemoryChunk;
+
+public class IndexMetaDataCacheFactory implements ServerCacheFactory {
+    public IndexMetaDataCacheFactory() {
+    }
+
+    @Override
+    public void readFields(DataInput arg0) throws IOException {
+    }
+
+    @Override
+    public void write(DataOutput arg0) throws IOException {
+    }
+
+    @Override
+    public Closeable newCache (ImmutableBytesWritable cachePtr, final MemoryChunk chunk) 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);
+        return new IndexMetaDataCache() {
+
+            @Override
+            public void close() throws IOException {
+                chunk.close();
+            }
+
+            @Override
+            public List<IndexMaintainer> getIndexMaintainers() {
+                return maintainers;
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..e278b24
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexBuilder.java
@@ -0,0 +1,96 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.index;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
+import org.apache.hadoop.hbase.regionserver.MultiVersionConsistencyControl;
+import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.util.Pair;
+
+import com.google.common.collect.Lists;
+import org.apache.hadoop.hbase.index.covered.CoveredColumnsIndexBuilder;
+import org.apache.hadoop.hbase.index.util.IndexManagementUtil;
+import org.apache.phoenix.compile.ScanRanges;
+import org.apache.phoenix.query.KeyRange;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.util.SchemaUtil;
+
+/**
+ * Index builder for covered-columns index that ties into phoenix for faster use.
+ */
+public class PhoenixIndexBuilder extends CoveredColumnsIndexBuilder {
+
+    @Override
+    public void batchStarted(MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp) throws IOException {
+        // The entire purpose of this method impl is to get the existing rows for the
+        // table rows being indexed into the block cache, as the index maintenance code
+        // does a point scan per row
+        List<KeyRange> keys = Lists.newArrayListWithExpectedSize(miniBatchOp.size());
+        List<IndexMaintainer> maintainers = new ArrayList<IndexMaintainer>();
+        for (int i = 0; i < miniBatchOp.size(); i++) {
+            Mutation m = miniBatchOp.getOperation(i).getFirst();
+            keys.add(PDataType.VARBINARY.getKeyRange(m.getRow()));
+            maintainers.addAll(getCodec().getIndexMaintainers(m.getAttributesMap()));
+        }
+        Scan scan = IndexManagementUtil.newLocalStateScan(maintainers);
+        ScanRanges scanRanges = ScanRanges.create(Collections.singletonList(keys), SchemaUtil.VAR_BINARY_SCHEMA);
+        scanRanges.setScanStartStopRow(scan);
+        scan.setFilter(scanRanges.getSkipScanFilter());
+        HRegion region = this.env.getRegion();
+        RegionScanner scanner = region.getScanner(scan);
+        // Run through the scanner using internal nextRaw method
+        MultiVersionConsistencyControl.setThreadReadPoint(scanner.getMvccReadPoint());
+        region.startRegionOperation();
+        try {
+            boolean hasMore;
+            do {
+                List<KeyValue> results = Lists.newArrayList();
+                // 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
+                hasMore = scanner.nextRaw(results, null);
+            } while (hasMore);
+        } finally {
+            try {
+                scanner.close();
+            } finally {
+                region.closeRegionOperation();
+            }
+        }
+    }
+
+    private PhoenixIndexCodec getCodec() {
+        return (PhoenixIndexCodec)this.codec;
+    }
+    
+    @Override
+    public byte[] getBatchId(Mutation m){
+        return this.codec.getBatchId(m);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..4f98a32
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexCodec.java
@@ -0,0 +1,182 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.index;
+
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Pair;
+
+import com.google.common.collect.Lists;
+import org.apache.hadoop.hbase.index.ValueGetter;
+import org.apache.hadoop.hbase.index.covered.IndexCodec;
+import org.apache.hadoop.hbase.index.covered.IndexUpdate;
+import org.apache.hadoop.hbase.index.covered.TableState;
+import org.apache.hadoop.hbase.index.scanner.Scanner;
+import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+import org.apache.hadoop.hbase.index.util.IndexManagementUtil;
+import org.apache.hadoop.hbase.index.write.IndexWriter;
+import org.apache.phoenix.cache.GlobalCache;
+import org.apache.phoenix.cache.IndexMetaDataCache;
+import org.apache.phoenix.cache.ServerCacheClient;
+import org.apache.phoenix.cache.TenantCache;
+import org.apache.phoenix.client.KeyValueBuilder;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.ServerUtil;
+
+/**
+ * Phoenix-based {@link IndexCodec}. Manages all the logic of how to cleanup an index (
+ * {@link #getIndexDeletes(TableState)}) as well as what the new index state should be (
+ * {@link #getIndexUpserts(TableState)}).
+ */
+public class PhoenixIndexCodec extends BaseIndexCodec {
+    public static final String INDEX_MD = "IdxMD";
+    public static final String INDEX_UUID = "IdxUUID";
+
+    private RegionCoprocessorEnvironment env;
+    private KeyValueBuilder builder;
+
+    @Override
+    public void initialize(RegionCoprocessorEnvironment env) {
+        this.env = env;
+        Configuration conf = env.getConfiguration();
+        // Install handler that will attempt to disable the index first before killing the region
+        // server
+        conf.setIfUnset(IndexWriter.INDEX_FAILURE_POLICY_CONF_KEY,
+            PhoenixIndexFailurePolicy.class.getName());
+        this.builder = KeyValueBuilder.get(env.getHBaseVersion());
+    }
+
+    List<IndexMaintainer> getIndexMaintainers(Map<String, byte[]> attributes) throws IOException{
+        if (attributes == null) {
+            return Collections.emptyList();
+        }
+        byte[] uuid = attributes.get(INDEX_UUID);
+        if (uuid == null) {
+            return Collections.emptyList();
+        }
+        byte[] md = attributes.get(INDEX_MD);
+        List<IndexMaintainer> indexMaintainers;
+        if (md != null) {
+            indexMaintainers = IndexMaintainer.deserialize(md, builder);
+        } else {
+            byte[] tenantIdBytes = attributes.get(PhoenixRuntime.TENANT_ID_ATTRIB);
+            ImmutableBytesWritable tenantId =
+                tenantIdBytes == null ? null : new ImmutableBytesWritable(tenantIdBytes);
+            TenantCache cache = GlobalCache.getTenantCache(env, tenantId);
+            IndexMetaDataCache indexCache =
+                (IndexMetaDataCache) cache.getServerCache(new ImmutableBytesPtr(uuid));
+            if (indexCache == null) {
+                String msg = "key="+ServerCacheClient.idToString(uuid) + " region=" + env.getRegion();
+                SQLException e = new SQLExceptionInfo.Builder(SQLExceptionCode.INDEX_METADATA_NOT_FOUND)
+                    .setMessage(msg).build().buildException();
+                ServerUtil.throwIOException("Index update failed", e); // will not return
+            }
+            indexMaintainers = indexCache.getIndexMaintainers();
+        }
+    
+        return indexMaintainers;
+    }
+    
+    @Override
+    public Iterable<IndexUpdate> getIndexUpserts(TableState state) throws IOException {
+        List<IndexMaintainer> indexMaintainers = getIndexMaintainers(state.getUpdateAttributes());
+        if (indexMaintainers.isEmpty()) {
+            return Collections.emptyList();
+        }
+        ImmutableBytesWritable ptr = new ImmutableBytesWritable();
+        List<IndexUpdate> indexUpdates = Lists.newArrayList();
+        // TODO: state.getCurrentRowKey() should take an ImmutableBytesWritable arg to prevent byte copy
+        byte[] dataRowKey = state.getCurrentRowKey();
+        for (IndexMaintainer maintainer : indexMaintainers) {
+            // Short-circuit building state when we know it's a row deletion
+            if (maintainer.isRowDeleted(state.getPendingUpdate())) {
+                continue;
+            }
+
+            // Get a scanner over the columns this maintainer would like to look at
+            // Any updates that we would make for those columns are then added to the index update
+            Pair<Scanner,IndexUpdate> statePair = state.getIndexedColumnsTableState(maintainer.getAllColumns());
+            IndexUpdate indexUpdate = statePair.getSecond();
+            Scanner scanner = statePair.getFirst();
+
+            // get the values from the scanner so we can actually use them
+            ValueGetter valueGetter = IndexManagementUtil.createGetterFromScanner(scanner, dataRowKey);
+            ptr.set(dataRowKey);
+            Put put = maintainer.buildUpdateMutation(valueGetter, ptr, state.getCurrentTimestamp());
+            indexUpdate.setTable(maintainer.getIndexTableName());
+            indexUpdate.setUpdate(put);
+            //make sure we close the scanner when we are done
+            scanner.close();
+            indexUpdates.add(indexUpdate);
+        }
+        return indexUpdates;
+    }
+
+    @Override
+    public Iterable<IndexUpdate> getIndexDeletes(TableState state) throws IOException {
+        List<IndexMaintainer> indexMaintainers = getIndexMaintainers(state.getUpdateAttributes());
+        if (indexMaintainers.isEmpty()) {
+            return Collections.emptyList();
+        }
+        List<IndexUpdate> indexUpdates = Lists.newArrayList();
+        ImmutableBytesWritable ptr = new ImmutableBytesWritable();
+        // TODO: state.getCurrentRowKey() should take an ImmutableBytesWritable arg to prevent byte copy
+        byte[] dataRowKey = state.getCurrentRowKey();
+        for (IndexMaintainer maintainer : indexMaintainers) {
+            // TODO: if more efficient, I could do this just once with all columns in all indexes
+            Pair<Scanner,IndexUpdate> statePair = state.getIndexedColumnsTableState(maintainer.getAllColumns());
+            Scanner scanner = statePair.getFirst();
+            IndexUpdate indexUpdate = statePair.getSecond();
+            indexUpdate.setTable(maintainer.getIndexTableName());
+            ValueGetter valueGetter = IndexManagementUtil.createGetterFromScanner(scanner, dataRowKey);
+            ptr.set(dataRowKey);
+            Delete delete =
+                maintainer.buildDeleteMutation(valueGetter, ptr, state.getPendingUpdate(),
+                  state.getCurrentTimestamp());
+            scanner.close();
+            indexUpdate.setUpdate(delete);
+            indexUpdates.add(indexUpdate);
+        }
+        return indexUpdates;
+    }
+    
+  @Override
+  public boolean isEnabled(Mutation m) throws IOException {
+      return !getIndexMaintainers(m.getAttributesMap()).isEmpty();
+  }
+  
+  @Override
+  public byte[] getBatchId(Mutation m) {
+    Map<String, byte[]> attributes = m.getAttributesMap();
+    return attributes.get(INDEX_UUID);
+  }
+}
\ No newline at end of file


[18/51] [partial] Initial commit of master branch from github

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/ExistsParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ExistsParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ExistsParseNode.java
new file mode 100644
index 0000000..aaa1a73
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ExistsParseNode.java
@@ -0,0 +1,55 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.List;
+
+
+
+/**
+ * 
+ * Node representing EXISTS and NOT EXISTS expressions in SQL
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class ExistsParseNode extends BinaryParseNode {
+    private final boolean negate;
+
+    ExistsParseNode(ParseNode l, ParseNode r, boolean negate) {
+        super(l, r);
+        this.negate = negate;
+    }
+    
+    public boolean isNegate() {
+        return negate;
+    }
+
+    @Override
+    public <T> T accept(ParseNodeVisitor<T> visitor) throws SQLException {
+        List<T> l = Collections.emptyList();
+        if (visitor.visitEnter(this)) {
+            l = acceptChildren(visitor);
+        }
+        return visitor.visitLeave(this, l);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/ExplainStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ExplainStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ExplainStatement.java
new file mode 100644
index 0000000..fc437c5
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ExplainStatement.java
@@ -0,0 +1,37 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+public class ExplainStatement implements BindableStatement {
+    private final BindableStatement statement;
+    
+    public ExplainStatement(BindableStatement statement) {
+        this.statement = statement;
+    }
+
+    public BindableStatement getStatement() {
+        return statement;
+    }
+
+    @Override
+    public int getBindCount() {
+        return statement.getBindCount();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/FamilyWildcardParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/FamilyWildcardParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/FamilyWildcardParseNode.java
new file mode 100644
index 0000000..438146c
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/FamilyWildcardParseNode.java
@@ -0,0 +1,55 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+
+/**
+ * 
+ * Node representing the selection of all columns of a family (cf.*) in the SELECT clause of SQL
+ *
+ * @author nmaillard
+ * @since 1.2
+ */
+
+public class FamilyWildcardParseNode extends NamedParseNode {
+    private final boolean isRewrite;
+    
+    public FamilyWildcardParseNode(String familyName, boolean isRewrite){
+        super(familyName);
+        this.isRewrite = isRewrite;
+    }
+    
+    public FamilyWildcardParseNode(FamilyWildcardParseNode familyName, boolean isRewrite){
+        super(familyName);
+        this.isRewrite = isRewrite;
+    }
+    
+	@Override
+	public <T> T accept(ParseNodeVisitor<T> visitor) throws SQLException {
+		return visitor.visit(this);
+	}
+
+    public boolean isRewrite() {
+        return isRewrite;
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/FilterableStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/FilterableStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/FilterableStatement.java
new file mode 100644
index 0000000..47969d9
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/FilterableStatement.java
@@ -0,0 +1,31 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.util.List;
+
+public interface FilterableStatement extends BindableStatement {
+    public HintNode getHint();
+    public ParseNode getWhere();
+    public boolean isDistinct();
+    public boolean isAggregate();
+    public List<OrderByNode> getOrderBy();
+    public LimitNode getLimit();
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/FloorParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/FloorParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/FloorParseNode.java
new file mode 100644
index 0000000..2f88bf2
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/FloorParseNode.java
@@ -0,0 +1,81 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.function.FloorDateExpression;
+import org.apache.phoenix.expression.function.FloorDecimalExpression;
+import org.apache.phoenix.expression.function.FloorFunction;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.TypeMismatchException;
+
+/**
+ * Parse node corresponding to {@link FloorFunction}. 
+ * It also acts as a factory for creating the right kind of
+ * floor expression according to the data type of the 
+ * first child.
+ *
+ * @author samarth.jain
+ * @since 3.0.0
+ */
+public class FloorParseNode extends FunctionParseNode {
+
+    FloorParseNode(String name, List<ParseNode> children, BuiltInFunctionInfo info) {
+        super(name, children, info);
+    }
+
+    @Override
+    public Expression create(List<Expression> children, StatementContext context) throws SQLException {
+        return getFloorExpression(children);
+    }
+
+    public static Expression getFloorExpression(List<Expression> children) throws SQLException {
+        final Expression firstChild = children.get(0);
+        final PDataType firstChildDataType = firstChild.getDataType();
+        
+        //FLOOR on timestamp doesn't really care about the nanos part i.e. it just sets it to zero. 
+        //Which is exactly what FloorDateExpression does too. 
+        if(firstChildDataType.isCoercibleTo(PDataType.TIMESTAMP)) {
+            return FloorDateExpression.create(children);
+        } else if(firstChildDataType.isCoercibleTo(PDataType.DECIMAL)) {
+            return new FloorDecimalExpression(children);
+        } else {
+            throw TypeMismatchException.newException(firstChildDataType, "1");
+        }
+    }
+    
+    /**
+     * When rounding off decimals, user need not specify the scale. In such cases, 
+     * we need to prevent the function from getting evaluated as null. This is really
+     * a hack. A better way would have been if {@link org.apache.phoenix.parse.FunctionParseNode.BuiltInFunctionInfo} provided a 
+     * way of associating default values for each permissible data type.
+     * Something like: @ Argument(allowedTypes={PDataType.VARCHAR, PDataType.INTEGER}, defaultValues = {"null", "1"} isConstant=true)
+     * Till then, this will have to do.
+     */
+    @Override
+    public boolean evalToNullIfParamIsNull(StatementContext context, int index) throws SQLException {
+        return index == 0;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/FunctionParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/FunctionParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/FunctionParseNode.java
new file mode 100644
index 0000000..e31a19e
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/FunctionParseNode.java
@@ -0,0 +1,422 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.http.annotation.Immutable;
+
+import com.google.common.collect.ImmutableSet;
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.LiteralExpression;
+import org.apache.phoenix.expression.function.AggregateFunction;
+import org.apache.phoenix.expression.function.FunctionExpression;
+import org.apache.phoenix.schema.ArgumentTypeMismatchException;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.ValueRangeExcpetion;
+import org.apache.phoenix.util.SchemaUtil;
+
+
+
+/**
+ * 
+ * Node representing a function expression in SQL
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class FunctionParseNode extends CompoundParseNode {
+    private final String name;
+    private final BuiltInFunctionInfo info;
+    
+    FunctionParseNode(String name, List<ParseNode> children, BuiltInFunctionInfo info) {
+        super(children);
+        this.name = SchemaUtil.normalizeIdentifier(name);
+        this.info = info;
+    }
+
+    public BuiltInFunctionInfo getInfo() {
+        return info;
+    }
+    
+    public String getName() {
+        return name;
+    }
+
+    @Override
+    public <T> T accept(ParseNodeVisitor<T> visitor) throws SQLException {
+        List<T> l = Collections.emptyList();
+        if (visitor.visitEnter(this)) {
+            l = acceptChildren(visitor);
+        }
+        return visitor.visitLeave(this, l);
+    }
+    
+    @Override
+    public String toString() {
+        StringBuilder buf = new StringBuilder(name + "(");
+        for (ParseNode child : getChildren()) {
+            buf.append(child.toString());
+            buf.append(',');
+        }
+        buf.setLength(buf.length()-1);
+        buf.append(')');
+        return buf.toString();
+    }
+
+    public boolean isAggregate() {
+        return getInfo().isAggregate();
+    }
+    
+    /**
+     * Determines whether or not we can collapse a function expression to null if a required
+     * parameter is null.
+     * @param context
+     * @param index index of parameter
+     * @return true if when the parameter at index is null, the function always evaluates to null
+     *  and false otherwise.
+     * @throws SQLException
+     */
+    public boolean evalToNullIfParamIsNull(StatementContext context, int index) throws SQLException {
+        return true;
+    }
+    
+
+    private static Constructor<? extends FunctionParseNode> getParseNodeCtor(Class<? extends FunctionParseNode> clazz) {
+        Constructor<? extends FunctionParseNode> ctor;
+        try {
+            ctor = clazz.getDeclaredConstructor(String.class, List.class, BuiltInFunctionInfo.class);
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+        ctor.setAccessible(true);
+        return ctor;
+    }
+    
+    private static Constructor<? extends FunctionExpression> getExpressionCtor(Class<? extends FunctionExpression> clazz) {
+        Constructor<? extends FunctionExpression> ctor;
+        try {
+            ctor = clazz.getDeclaredConstructor(List.class);
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+        ctor.setAccessible(true);
+        return ctor;
+    }
+    
+    public List<Expression> validate(List<Expression> children, StatementContext context) throws SQLException {
+        BuiltInFunctionInfo info = this.getInfo();
+        BuiltInFunctionArgInfo[] args = info.getArgs();
+        if (args.length > children.size()) {
+            List<Expression> moreChildren = new ArrayList<Expression>(children);
+            for (int i = children.size(); i < info.getArgs().length; i++) {
+                moreChildren.add(LiteralExpression.newConstant(null, args[i].allowedTypes.length == 0 ? null :  args[i].allowedTypes[0], true));
+            }
+            children = moreChildren;
+        }
+        List<ParseNode> nodeChildren = this.getChildren();
+        for (int i = 0; i < children.size(); i++) {
+            BindParseNode bindNode = null;
+            PDataType[] allowedTypes = args[i].getAllowedTypes();
+            // Check if the node is a bind parameter, and set the parameter
+            // metadata based on the function argument annotation. Check to
+            // make sure we're not looking past the end of the list of
+            // child expression nodes, which can happen if the function
+            // invocation hasn't specified all arguments and is using default
+            // values.
+            if (i < nodeChildren.size() && nodeChildren.get(i) instanceof BindParseNode) {
+                bindNode = (BindParseNode)nodeChildren.get(i);
+            }
+            // If the child type is null, then the expression is unbound.
+            // Skip any validation, since we either 1) have a default value
+            // which has already been validated, 2) attempting to get the
+            // parameter metadata, or 3) have an unbound parameter which
+            // will be detected futher downstream.
+            Expression child = children.get(i);
+            if (child.getDataType() == null /* null used explicitly in query */ || i >= nodeChildren.size() /* argument left off */) {
+                // Replace the unbound expression with the default value expression if specified
+                if (args[i].getDefaultValue() != null) {
+                    Expression defaultValue = args[i].getDefaultValue();
+                    children.set(i, defaultValue);
+                    // Set the parameter metadata if this is a bind parameter
+                    if (bindNode != null) {
+                        context.getBindManager().addParamMetaData(bindNode, defaultValue);
+                    }
+                } else if (bindNode != null) {
+                    // Otherwise if the node is a bind parameter and we have type information
+                    // based on the function argument annonation set the parameter meta data.
+                    if (child.getDataType() == null) {
+                        if (allowedTypes.length > 0) {
+                            context.getBindManager().addParamMetaData(bindNode, LiteralExpression.newConstant(null, allowedTypes[0], true));
+                        }
+                    } else { // Use expression as is, since we already have the data type set
+                        context.getBindManager().addParamMetaData(bindNode, child);
+                    }
+                }
+            } else {
+                if (allowedTypes.length > 0) {
+                    boolean isCoercible = false;
+                    for (PDataType type : allowedTypes) {
+                        if (child.getDataType().isCoercibleTo(type)) {
+                            isCoercible = true;
+                            break;
+                        }
+                    }
+                    if (!isCoercible) {
+                        throw new ArgumentTypeMismatchException(Arrays.toString(args[i].getAllowedTypes()),
+                                child.getDataType().toString(), info.getName() + " argument " + (i + 1));
+                    }
+                    if (child instanceof LiteralExpression) {
+                        LiteralExpression valueExp = (LiteralExpression) child;
+                        LiteralExpression minValue = args[i].getMinValue();
+                        LiteralExpression maxValue = args[i].getMaxValue();
+                        if (minValue != null && minValue.getDataType().compareTo(minValue.getValue(), valueExp.getValue(), valueExp.getDataType()) > 0) {
+                            throw new ValueRangeExcpetion(minValue, maxValue == null ? "" : maxValue, valueExp.getValue(), info.getName() + " argument " + (i + 1));
+                        }
+                        if (maxValue != null && maxValue.getDataType().compareTo(maxValue.getValue(), valueExp.getValue(), valueExp.getDataType()) < 0) {
+                            throw new ValueRangeExcpetion(minValue == null ? "" : minValue, maxValue, valueExp.getValue(), info.getName() + " argument " + (i + 1));
+                        }
+                    }
+                }
+                if (args[i].isConstant() && ! (child instanceof LiteralExpression) ) {
+                    throw new ArgumentTypeMismatchException("constant", child.toString(), info.getName() + " argument " + (i + 1));
+                }
+                if (!args[i].getAllowedValues().isEmpty()) {
+                    Object value = ((LiteralExpression)child).getValue();
+                    if (!args[i].getAllowedValues().contains(value.toString().toUpperCase())) {
+                        throw new ArgumentTypeMismatchException(Arrays.toString(args[i].getAllowedValues().toArray(new String[0])),
+                                value.toString(), info.getName() + " argument " + (i + 1));
+                    }
+                }
+            }
+        }
+        return children;
+    }
+    
+    /**
+     * Entry point for parser to instantiate compiled representation of built-in function
+     * @param children Compiled expressions for child nodes
+     * @param context Query context for accessing state shared across the processing of multiple clauses
+     * @return compiled representation of built-in function
+     * @throws SQLException
+     */
+    public Expression create(List<Expression> children, StatementContext context) throws SQLException {
+        try {
+            return info.getFuncCtor().newInstance(children);
+        } catch (InstantiationException e) {
+            throw new SQLException(e);
+        } catch (IllegalAccessException e) {
+            throw new SQLException(e);
+        } catch (IllegalArgumentException e) {
+            throw new SQLException(e);
+        } catch (InvocationTargetException e) {
+            if (e.getTargetException() instanceof SQLException) {
+                throw (SQLException) e.getTargetException();
+            }
+            throw new SQLException(e);
+        }
+    }
+    
+    /**
+     * Marker used to indicate that a class should be used by DirectFunctionExpressionExec below
+     */
+    @Retention(RetentionPolicy.RUNTIME)
+    @Target(ElementType.TYPE)
+    public
+    @interface BuiltInFunction {
+        String name();
+        Argument[] args() default {};
+        Class<? extends FunctionParseNode> nodeClass() default FunctionParseNode.class;
+    }
+
+    @Retention(RetentionPolicy.RUNTIME)
+    @Target(ElementType.TYPE)
+    public
+    @interface Argument {
+        PDataType[] allowedTypes() default {};
+        boolean isConstant() default false;
+        String defaultValue() default "";
+        String enumeration() default "";
+        String minValue() default "";
+        String maxValue() default "";
+    }
+    
+    /**
+     * Structure used to hold parse-time information about Function implementation classes
+     */
+    @Immutable
+    public static final class BuiltInFunctionInfo {
+        private final String name;
+        private final Constructor<? extends FunctionExpression> funcCtor;
+        private final Constructor<? extends FunctionParseNode> nodeCtor;
+        private final BuiltInFunctionArgInfo[] args;
+        private final boolean isAggregate;
+        private final int requiredArgCount;
+
+        BuiltInFunctionInfo(Class<? extends FunctionExpression> f, BuiltInFunction d) {
+            this.name = SchemaUtil.normalizeIdentifier(d.name());
+            this.funcCtor = d.nodeClass() == FunctionParseNode.class ? getExpressionCtor(f) : null;
+            this.nodeCtor = d.nodeClass() == FunctionParseNode.class ? null : getParseNodeCtor(d.nodeClass());
+            this.args = new BuiltInFunctionArgInfo[d.args().length];
+            int requiredArgCount = 0;
+            for (int i = 0; i < args.length; i++) {
+                this.args[i] = new BuiltInFunctionArgInfo(d.args()[i]);
+                if (requiredArgCount < i && this.args[i].getDefaultValue() != null) {
+                    requiredArgCount = i;
+                }
+            }
+            this.requiredArgCount = requiredArgCount;
+            this.isAggregate = AggregateFunction.class.isAssignableFrom(f);
+        }
+
+        public int getRequiredArgCount() {
+            return requiredArgCount;
+        }
+        
+        public String getName() {
+            return name;
+        }
+
+        public Constructor<? extends FunctionExpression> getFuncCtor() {
+            return funcCtor;
+        }
+
+        public Constructor<? extends FunctionParseNode> getNodeCtor() {
+            return nodeCtor;
+        }
+
+        public boolean isAggregate() {
+            return isAggregate;
+        }
+        
+        public BuiltInFunctionArgInfo[] getArgs() {
+            return args;
+        }
+    }
+    
+    @Immutable
+    public static class BuiltInFunctionArgInfo {
+        private static final PDataType[] ENUMERATION_TYPES = new PDataType[] {PDataType.VARCHAR};
+        private final PDataType[] allowedTypes;
+        private final boolean isConstant;
+        private final Set<String> allowedValues; // Enumeration of possible values
+        private final LiteralExpression defaultValue;
+        private final LiteralExpression minValue;
+        private final LiteralExpression maxValue;
+        
+        @SuppressWarnings({ "unchecked", "rawtypes" })
+        BuiltInFunctionArgInfo(Argument argument) {
+            
+            if (argument.enumeration().length() > 0) {
+                this.isConstant = true;
+                this.defaultValue = null;
+                this.minValue = null;
+                this.maxValue = null;
+                this.allowedTypes = ENUMERATION_TYPES;
+                Class<?> clazz = null;
+                String packageName = FunctionExpression.class.getPackage().getName();
+                try {
+                    clazz = Class.forName(packageName + "." + argument.enumeration());
+                } catch (ClassNotFoundException e) {
+                    try {
+                        clazz = Class.forName(argument.enumeration());
+                    } catch (ClassNotFoundException e1) {
+                    }
+                }
+                if (clazz == null || !clazz.isEnum()) {
+                    throw new IllegalStateException("The enumeration annotation '" + argument.enumeration() + "' does not resolve to a enumeration class");
+                }
+                Class<? extends Enum> enumClass = (Class<? extends Enum>)clazz;
+				Enum[] enums = enumClass.getEnumConstants();
+                ImmutableSet.Builder<String> builder = ImmutableSet.builder();
+                for (Enum en : enums) {
+                    builder.add(en.name());
+                }
+                allowedValues = builder.build();
+            } else {
+                this.allowedValues = Collections.emptySet();
+                this.isConstant = argument.isConstant();
+                this.allowedTypes = argument.allowedTypes();
+                this.defaultValue = getExpFromConstant(argument.defaultValue());
+                this.minValue = getExpFromConstant(argument.minValue());
+                this.maxValue = getExpFromConstant(argument.maxValue());
+            }
+        }
+
+        private LiteralExpression getExpFromConstant(String strValue) {
+            LiteralExpression exp = null;
+            if (strValue.length() > 0) {
+                SQLParser parser = new SQLParser(strValue);
+                try {
+                    LiteralParseNode node = parser.parseLiteral();
+                    LiteralExpression defaultValue = LiteralExpression.newConstant(node.getValue(), this.allowedTypes[0], true);
+                    if (this.getAllowedTypes().length > 0) {
+                        for (PDataType type : this.getAllowedTypes()) {
+                            if (defaultValue.getDataType() == null || defaultValue.getDataType().isCoercibleTo(type, node.getValue())) {
+                                return LiteralExpression.newConstant(node.getValue(), type, true);
+                            }
+                        }
+                        throw new IllegalStateException("Unable to coerce default value " + strValue + " to any of the allowed types of " + Arrays.toString(this.getAllowedTypes()));
+                    }
+                    exp = defaultValue;
+                } catch (SQLException e) {
+                    throw new RuntimeException(e);
+                }
+            }
+            return exp;
+        }
+        
+        public boolean isConstant() {
+            return isConstant;
+        }
+
+        public LiteralExpression getDefaultValue() {
+            return defaultValue;
+        }
+
+        public LiteralExpression getMinValue() {
+            return minValue;
+        }
+        
+        public LiteralExpression getMaxValue() {
+            return maxValue;
+        }
+        
+        public PDataType[] getAllowedTypes() {
+            return allowedTypes;
+        }
+        
+        public Set<String> getAllowedValues() {
+            return allowedValues;
+        }
+    }    
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/GreaterThanOrEqualParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/GreaterThanOrEqualParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/GreaterThanOrEqualParseNode.java
new file mode 100644
index 0000000..4780d56
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/GreaterThanOrEqualParseNode.java
@@ -0,0 +1,49 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import org.apache.hadoop.hbase.filter.CompareFilter;
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+
+
+/**
+ * 
+ * Node representing the greater than or equal to operator (>=) in SQL
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class GreaterThanOrEqualParseNode extends ComparisonParseNode {
+
+    GreaterThanOrEqualParseNode(ParseNode lhs, ParseNode rhs) {
+        super(lhs, rhs);
+    }
+
+    @Override
+    public CompareOp getFilterOp() {
+        return CompareFilter.CompareOp.GREATER_OR_EQUAL;
+    }
+
+
+    @Override
+    public CompareOp getInvertFilterOp() {
+        return CompareOp.LESS_OR_EQUAL;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/GreaterThanParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/GreaterThanParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/GreaterThanParseNode.java
new file mode 100644
index 0000000..ff0442b
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/GreaterThanParseNode.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import org.apache.hadoop.hbase.filter.CompareFilter;
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+
+
+/**
+ * 
+ * Node representing the greater than operator (>) in SQL
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class GreaterThanParseNode extends ComparisonParseNode {
+
+    GreaterThanParseNode(ParseNode lhs, ParseNode rhs) {
+        super(lhs, rhs);
+    }
+
+    @Override
+    public CompareOp getFilterOp() {
+        return CompareFilter.CompareOp.GREATER;
+    }
+
+    @Override
+    public CompareOp getInvertFilterOp() {
+        return CompareOp.LESS;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/HintNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/HintNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/HintNode.java
new file mode 100644
index 0000000..fa82382
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/HintNode.java
@@ -0,0 +1,160 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.phoenix.util.SchemaUtil;
+
+
+/**
+ * Node representing optimizer hints in SQL
+ */
+public class HintNode {
+    public static final HintNode EMPTY_HINT_NODE = new HintNode();
+    
+    public static final char SEPARATOR = ' ';
+    public static final String PREFIX = "(";
+    public static final String SUFFIX = ")";
+    // Split on whitespace and parenthesis, keeping the parenthesis in the token array
+    private static final String SPLIT_REGEXP = "\\s+|((?<=\\" + PREFIX + ")|(?=\\" + PREFIX + "))|((?<=\\" + SUFFIX + ")|(?=\\" + SUFFIX + "))";
+    
+    public enum Hint {
+        /**
+         * Forces a range scan to be used to process the query.
+         */
+        RANGE_SCAN,
+        /**
+         * Forces a skip scan to be used to process the query.
+         */
+        SKIP_SCAN,
+        /**
+         * Prevents the spawning of multiple threads during
+         * query processing.
+         */
+        NO_INTRA_REGION_PARALLELIZATION,
+        /**
+        * Prevents the usage of indexes, forcing usage
+        * of the data table for a query.
+        */
+       NO_INDEX,
+       /**
+       * Hint of the form INDEX(<table_name> <index_name>...)
+       * to suggest usage of the index if possible. The first
+       * usable index in the list of indexes will be choosen.
+       * Table and index names may be surrounded by double quotes
+       * if they are case sensitive.
+       */
+       INDEX,
+       /**
+        * All things being equal, use the data table instead of
+        * the index table when optimizing.
+        */
+       USE_DATA_OVER_INDEX_TABLE,
+       /**
+        * All things being equal, use the index table instead of
+        * the data table when optimizing.
+        */
+       USE_INDEX_OVER_DATA_TABLE,
+    };
+
+    private final Map<Hint,String> hints;
+
+    public static HintNode create(HintNode hintNode, Hint hint) {
+        return create(hintNode, hint, "");
+    }
+    
+    public static HintNode create(HintNode hintNode, Hint hint, String value) {
+        Map<Hint,String> hints = new HashMap<Hint,String>(hintNode.hints);
+        hints.put(hint, value);
+        return new HintNode(hints);
+    }
+    
+    private HintNode() {
+        hints = new HashMap<Hint,String>();
+    }
+
+    private HintNode(Map<Hint,String> hints) {
+        this.hints = ImmutableMap.copyOf(hints);
+    }
+
+    public HintNode(String hint) {
+        Map<Hint,String> hints = new HashMap<Hint,String>();
+        // Split on whitespace or parenthesis. We do not need to handle escaped or
+        // embedded whitespace/parenthesis, since we are parsing what will be HBase
+        // table names which are not allowed to contain whitespace or parenthesis.
+        String[] hintWords = hint.split(SPLIT_REGEXP);
+        for (int i = 0; i < hintWords.length; i++) {
+            String hintWord = hintWords[i];
+            if (hintWord.isEmpty()) {
+                continue;
+            }
+            try {
+                Hint key = Hint.valueOf(hintWord.toUpperCase());
+                String hintValue = "";
+                if (i+1 < hintWords.length && PREFIX.equals(hintWords[i+1])) {
+                    StringBuffer hintValueBuf = new StringBuffer(hint.length());
+                    hintValueBuf.append(PREFIX);
+                    i+=2;
+                    while (i < hintWords.length && !SUFFIX.equals(hintWords[i])) {
+                        hintValueBuf.append(SchemaUtil.normalizeIdentifier(hintWords[i++]));
+                        hintValueBuf.append(SEPARATOR);
+                    }
+                    // Replace trailing separator with suffix
+                    hintValueBuf.replace(hintValueBuf.length()-1, hintValueBuf.length(), SUFFIX);
+                    hintValue = hintValueBuf.toString();
+                }
+                String oldValue = hints.put(key, hintValue);
+                // Concatenate together any old value with the new value
+                if (oldValue != null) {
+                    hints.put(key, oldValue + hintValue);
+                }
+            } catch (IllegalArgumentException e) { // Ignore unknown/invalid hints
+            }
+        }
+        this.hints = ImmutableMap.copyOf(hints);
+    }
+    
+    public boolean isEmpty() {
+        return hints.isEmpty();
+    }
+
+    /**
+     * Gets the value of the hint or null if the hint is not present.
+     * @param hint the hint
+     * @return the value specified in parenthesis following the hint or null
+     * if the hint is not present.
+     * 
+     */
+    public String getHint(Hint hint) {
+        return hints.get(hint);
+    }
+
+    /**
+     * Tests for the presence of a hint in a query
+     * @param hint the hint
+     * @return true if the hint is present and false otherwise
+     */
+    public boolean hasHint(Hint hint) {
+        return hints.containsKey(hint);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/InListParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/InListParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/InListParseNode.java
new file mode 100644
index 0000000..7632de7
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/InListParseNode.java
@@ -0,0 +1,66 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+
+
+
+/**
+ * 
+ * Node representing the IN literal list expression in SQL
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class InListParseNode extends CompoundParseNode {
+    private final boolean negate;
+
+    InListParseNode(List<ParseNode> children, boolean negate) {
+        super(children);
+        // All values in the IN must be constant. First child is the LHS
+        for (int i = 1; i < children.size(); i++) {
+            ParseNode child = children.get(i);
+            if (!child.isStateless()) {
+                throw new ParseException(new SQLExceptionInfo.Builder(SQLExceptionCode.VALUE_IN_LIST_NOT_CONSTANT)
+                .build().buildException());
+            }
+        }
+        this.negate = negate;
+    }
+    
+    public boolean isNegate() {
+        return negate;
+    }
+
+    @Override
+    public <T> T accept(ParseNodeVisitor<T> visitor) throws SQLException {
+        List<T> l = Collections.emptyList();
+        if (visitor.visitEnter(this)) {
+            l = acceptChildren(visitor);
+        }
+        return visitor.visitLeave(this, l);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/InParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/InParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/InParseNode.java
new file mode 100644
index 0000000..d7708fc
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/InParseNode.java
@@ -0,0 +1,55 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.List;
+
+
+
+/**
+ * 
+ * Node representing IN subquery expression in SQL
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class InParseNode extends BinaryParseNode {
+    private final boolean negate;
+
+    InParseNode(ParseNode l, ParseNode r, boolean negate) {
+        super(l, r);
+        this.negate = negate;
+    }
+    
+    public boolean isNegate() {
+        return negate;
+    }
+
+    @Override
+    public <T> T accept(ParseNodeVisitor<T> visitor) throws SQLException {
+        List<T> l = Collections.emptyList();
+        if (visitor.visitEnter(this)) {
+            l = acceptChildren(visitor);
+        }
+        return visitor.visitLeave(this, l);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java
new file mode 100644
index 0000000..6dfc744
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java
@@ -0,0 +1,39 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.util.List;
+
+import org.apache.hadoop.hbase.util.Pair;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.phoenix.schema.ColumnModifier;
+
+public class IndexKeyConstraint {
+    private final List<Pair<ColumnParseNode, ColumnModifier>> columnNameToModifier;
+    
+    IndexKeyConstraint(List<Pair<ColumnParseNode, ColumnModifier>> columnNameAndModifier) {
+        this.columnNameToModifier = ImmutableList.copyOf(columnNameAndModifier);
+    }
+
+    public List<Pair<ColumnParseNode, ColumnModifier>> getColumns() {
+        return columnNameToModifier;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/IsNullParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/IsNullParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/IsNullParseNode.java
new file mode 100644
index 0000000..a1afc2e
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/IsNullParseNode.java
@@ -0,0 +1,55 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.List;
+
+
+
+/**
+ * 
+ * Node representing the IS NULL and IS NOT NULL expressions in SQL
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class IsNullParseNode extends UnaryParseNode {
+    private final boolean negate;
+    
+    IsNullParseNode(ParseNode expr, boolean negate) {
+        super(expr);
+        this.negate = negate;
+    }
+    
+    public boolean isNegate() {
+        return negate;
+    }
+
+    @Override
+    public <T> T accept(ParseNodeVisitor<T> visitor) throws SQLException {
+        List<T> l = Collections.emptyList();
+        if (visitor.visitEnter(this)) {
+            l = acceptChildren(visitor);
+        }
+        return visitor.visitLeave(this, l);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/JoinTableNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/JoinTableNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/JoinTableNode.java
new file mode 100644
index 0000000..d6f7f82
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/JoinTableNode.java
@@ -0,0 +1,64 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+
+
+
+/**
+ * 
+ * Node representing the join specified in the FROM clause of SQL
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class JoinTableNode extends TableNode {
+    public enum JoinType {Inner, Left, Right, Full};
+    
+    private final JoinType type;
+    private final ParseNode on;
+    private final TableNode table;
+    
+    JoinTableNode(JoinType type, ParseNode on, TableNode table) {
+        super(table.getAlias());
+        this.type = type;
+        this.on = on;
+        this.table = table;
+    }
+
+    public JoinType getType() {
+        return type;
+    }
+
+    public ParseNode getOnNode() {
+        return on;
+    }
+    
+    public TableNode getTable() {
+        return table;
+    }
+
+    @Override
+    public void accept(TableNodeVisitor visitor) throws SQLException {
+        visitor.visit(this);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/LessThanOrEqualParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/LessThanOrEqualParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/LessThanOrEqualParseNode.java
new file mode 100644
index 0000000..4fb91dd
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/LessThanOrEqualParseNode.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import org.apache.hadoop.hbase.filter.CompareFilter;
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+
+
+/**
+ * 
+ * Node representing the less than or equal to operator (<=) in SQL
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class LessThanOrEqualParseNode extends ComparisonParseNode {
+
+    LessThanOrEqualParseNode(ParseNode lhs, ParseNode rhs) {
+        super(lhs, rhs);
+    }
+
+    @Override
+    public CompareOp getFilterOp() {
+        return CompareFilter.CompareOp.LESS_OR_EQUAL;
+    }
+
+    @Override
+    public CompareOp getInvertFilterOp() {
+        return CompareOp.GREATER_OR_EQUAL;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/LessThanParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/LessThanParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/LessThanParseNode.java
new file mode 100644
index 0000000..c9f69dc
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/LessThanParseNode.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import org.apache.hadoop.hbase.filter.CompareFilter;
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+
+
+/**
+ * 
+ * Node representing the less than operator (<) in SQL
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class LessThanParseNode extends ComparisonParseNode {
+
+    LessThanParseNode(ParseNode lhs, ParseNode rhs) {
+        super(lhs, rhs);
+    }
+
+    @Override
+    public CompareOp getFilterOp() {
+        return CompareFilter.CompareOp.LESS;
+    }
+
+    @Override
+    public CompareOp getInvertFilterOp() {
+        return CompareOp.GREATER;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/LikeParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/LikeParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/LikeParseNode.java
new file mode 100644
index 0000000..b941d56
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/LikeParseNode.java
@@ -0,0 +1,55 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.List;
+
+
+
+/**
+ * 
+ * Node representing LIKE and NOT LIKE in SQL
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class LikeParseNode extends BinaryParseNode {
+    private final boolean negate;
+
+    LikeParseNode(ParseNode lhs, ParseNode rhs, boolean negate) {
+        super(lhs, rhs);
+        this.negate = negate;
+    }
+    
+    public boolean isNegate() {
+        return negate;
+    }
+
+    @Override
+    public <T> T accept(ParseNodeVisitor<T> visitor) throws SQLException {
+        List<T> l = Collections.emptyList();
+        if (visitor.visitEnter(this)) {
+            l = acceptChildren(visitor);
+        }
+        return visitor.visitLeave(this, l);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/LimitNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/LimitNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/LimitNode.java
new file mode 100644
index 0000000..cd4d6a9
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/LimitNode.java
@@ -0,0 +1,40 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+
+public class LimitNode {
+    private final BindParseNode bindNode;
+    private final LiteralParseNode limitNode;
+    
+    LimitNode(BindParseNode bindNode) {
+        this.bindNode = bindNode;
+        limitNode = null;
+    }
+    
+    LimitNode(LiteralParseNode limitNode) {
+        this.limitNode = limitNode;
+        this.bindNode = null;
+    }
+    
+    public ParseNode getLimitParseNode() {
+        return bindNode == null ? limitNode : bindNode;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/LiteralParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/LiteralParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/LiteralParseNode.java
new file mode 100644
index 0000000..25247ee
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/LiteralParseNode.java
@@ -0,0 +1,77 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.phoenix.schema.PDataType;
+
+
+
+/**
+ * 
+ * Node representing literal expressions such as 1,2.5,'foo', and NULL in SQL
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class LiteralParseNode extends TerminalParseNode {
+    public static final List<ParseNode> STAR = Collections.<ParseNode>singletonList(new LiteralParseNode(1));
+    public static final ParseNode NULL = new LiteralParseNode(null);
+    public static final ParseNode ZERO = new LiteralParseNode(0);
+    public static final ParseNode ONE = new LiteralParseNode(1);
+    
+    private final Object value;
+    private final PDataType type;
+    
+    public LiteralParseNode(Object value) {
+        this.value = value;
+        this.type = PDataType.fromLiteral(value);
+    }
+
+    public PDataType getType() {
+        return type;
+    }
+    
+    public Object getValue() {
+        return value;
+    }
+
+    @Override
+    public boolean isStateless() {
+        return true;
+    }
+    
+    @Override
+    public <T> T accept(ParseNodeVisitor<T> visitor) throws SQLException {
+        return visitor.visit(this);
+    }
+
+    public byte[] getBytes() {
+        return type == null ? null : type.toBytes(value);
+    }
+    
+    @Override
+    public String toString() {
+        return type == PDataType.VARCHAR ? ("'" + value.toString() + "'") : value == null ? "null" : value.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/MaxAggregateParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/MaxAggregateParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/MaxAggregateParseNode.java
new file mode 100644
index 0000000..6f59a72
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/MaxAggregateParseNode.java
@@ -0,0 +1,41 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.function.FunctionExpression;
+import org.apache.phoenix.expression.function.MaxAggregateFunction;
+
+
+public class MaxAggregateParseNode extends DelegateConstantToCountParseNode {
+
+    public MaxAggregateParseNode(String name, List<ParseNode> children, BuiltInFunctionInfo info) {
+        super(name, children, info);
+    }
+    
+    @Override
+    public FunctionExpression create(List<Expression> children, StatementContext context) throws SQLException {
+        return new MaxAggregateFunction(children, getDelegateFunction(children,context));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/MinAggregateParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/MinAggregateParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/MinAggregateParseNode.java
new file mode 100644
index 0000000..f6450dd
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/MinAggregateParseNode.java
@@ -0,0 +1,41 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.function.FunctionExpression;
+import org.apache.phoenix.expression.function.MinAggregateFunction;
+
+
+public class MinAggregateParseNode extends DelegateConstantToCountParseNode {
+
+    public MinAggregateParseNode(String name, List<ParseNode> children, BuiltInFunctionInfo info) {
+        super(name, children, info);
+    }
+    
+    @Override
+    public FunctionExpression create(List<Expression> children, StatementContext context) throws SQLException {
+        return new MinAggregateFunction(children, getDelegateFunction(children,context));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/MultiplyParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/MultiplyParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/MultiplyParseNode.java
new file mode 100644
index 0000000..1b100e9
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/MultiplyParseNode.java
@@ -0,0 +1,49 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.List;
+
+
+
+/**
+ * 
+ * Node representing multiplication in a SQL expression
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class MultiplyParseNode extends ArithmeticParseNode {
+
+    MultiplyParseNode(List<ParseNode> children) {
+        super(children);
+    }
+
+    @Override
+    public <T> T accept(ParseNodeVisitor<T> visitor) throws SQLException {
+        List<T> l = Collections.emptyList();
+        if (visitor.visitEnter(this)) {
+            l = acceptChildren(visitor);
+        }
+        return visitor.visitLeave(this, l);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/NamedNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/NamedNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/NamedNode.java
new file mode 100644
index 0000000..9c687aa
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/NamedNode.java
@@ -0,0 +1,45 @@
+package org.apache.phoenix.parse;
+
+import org.apache.phoenix.util.SchemaUtil;
+
+public class NamedNode {
+    private final String name;
+    private final boolean isCaseSensitive;
+    
+    public static NamedNode caseSensitiveNamedNode(String name) {
+        return new NamedNode(name,true);
+    }
+    
+    private NamedNode(String name, boolean isCaseSensitive) {
+        this.name = name;
+        this.isCaseSensitive = isCaseSensitive;
+    }
+
+    NamedNode(String name) {
+        this.name = SchemaUtil.normalizeIdentifier(name);
+        this.isCaseSensitive = name == null ? false : SchemaUtil.isCaseSensitive(name);
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public boolean isCaseSensitive() {
+        return isCaseSensitive;
+    }
+    
+    @Override
+    public int hashCode() {
+        return name.hashCode();
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) return true;
+        if (obj == null) return false;
+        if (getClass() != obj.getClass()) return false;
+        NamedNode other = (NamedNode)obj;
+        return name.equals(other.name);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/NamedParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/NamedParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/NamedParseNode.java
new file mode 100644
index 0000000..bd1c6cf
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/NamedParseNode.java
@@ -0,0 +1,53 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+
+/**
+ * 
+ * Abstract node representing named nodes such as binds and column expressions in SQL
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public abstract class NamedParseNode extends TerminalParseNode{
+    private final NamedNode namedNode;
+    
+    NamedParseNode(NamedParseNode node) {
+        this.namedNode = node.namedNode;
+    }
+
+    NamedParseNode(String name) {
+        this.namedNode = new NamedNode(name);
+    }
+
+    public String getName() {
+        return namedNode.getName();
+    }
+
+    public boolean isCaseSensitive() {
+        return namedNode.isCaseSensitive();
+    }
+    
+    @Override
+    public String toString() {
+        return getName();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/NamedTableNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/NamedTableNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/NamedTableNode.java
new file mode 100644
index 0000000..4be546d
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/NamedTableNode.java
@@ -0,0 +1,64 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.List;
+
+import com.google.common.collect.ImmutableList;
+
+/**
+ * Node representing an explicit table reference in the FROM clause of SQL
+ * 
+ * @author jtaylor
+ * @since 0.1
+ */
+public class NamedTableNode extends ConcreteTableNode {
+
+    private final List<ColumnDef> dynColumns;
+
+    public static NamedTableNode create (String alias, TableName name, List<ColumnDef> dynColumns) {
+        return new NamedTableNode(alias, name, dynColumns);
+    }
+    
+    NamedTableNode(String alias, TableName name) {
+        super(alias, name);
+        dynColumns = Collections.<ColumnDef> emptyList();
+    }
+
+    NamedTableNode(String alias, TableName name, List<ColumnDef> dynColumns) {
+        super(alias, name);
+        if (dynColumns != null) {
+            this.dynColumns = ImmutableList.copyOf(dynColumns);
+        } else {
+            this.dynColumns = Collections.<ColumnDef> emptyList();
+        }
+    }
+
+    @Override
+    public void accept(TableNodeVisitor visitor) throws SQLException {
+        visitor.visit(this);
+    }
+
+    public List<ColumnDef> getDynamicColumns() {
+        return dynColumns;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/NotEqualParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/NotEqualParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/NotEqualParseNode.java
new file mode 100644
index 0000000..7872275
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/NotEqualParseNode.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+
+
+
+/**
+ * 
+ * Node representing a not equal expression (!=,<>) in SQL
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class NotEqualParseNode extends ComparisonParseNode {
+
+    NotEqualParseNode(ParseNode lhs, ParseNode rhs) {
+        super(lhs, rhs);
+    }
+
+    @Override
+    public CompareOp getFilterOp() {
+        return CompareOp.NOT_EQUAL;
+    }
+
+    @Override
+    public CompareOp getInvertFilterOp() {
+        return CompareOp.NOT_EQUAL;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/NotParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/NotParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/NotParseNode.java
new file mode 100644
index 0000000..580e7ca
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/NotParseNode.java
@@ -0,0 +1,49 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.List;
+
+
+
+/**
+ * 
+ * Node representing the NOT in SQL
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class NotParseNode extends UnaryParseNode {
+
+    NotParseNode(ParseNode expr) {
+        super(expr);
+    }
+
+    @Override
+    public <T> T accept(ParseNodeVisitor<T> visitor) throws SQLException {
+        List<T> l = Collections.emptyList();
+        if (visitor.visitEnter(this)) {
+            l = acceptChildren(visitor);
+        }
+        return visitor.visitLeave(this, l);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/OrParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/OrParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/OrParseNode.java
new file mode 100644
index 0000000..0fe869f
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/OrParseNode.java
@@ -0,0 +1,49 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.List;
+
+
+
+/**
+ * 
+ * Node representing an OR in SQL
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class OrParseNode extends CompoundParseNode {
+
+    OrParseNode(List<ParseNode> children) {
+        super(children);
+    }
+
+    @Override
+    public <T> T accept(ParseNodeVisitor<T> visitor) throws SQLException {
+        List<T> l = Collections.emptyList();
+        if (visitor.visitEnter(this)) {
+            l = acceptChildren(visitor);
+        }
+        return visitor.visitLeave(this, l);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/OrderByNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/OrderByNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/OrderByNode.java
new file mode 100644
index 0000000..6a48d10
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/OrderByNode.java
@@ -0,0 +1,57 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+
+/**
+ * 
+ * Node representing an ORDER BY clause (including asc/desc and nulls first/last) in SQL
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public final class OrderByNode {
+    private final ParseNode child;
+    private final boolean nullsLast;
+    private final boolean orderAscending;
+    
+    OrderByNode(ParseNode child, boolean nullsLast, boolean orderAscending) {
+        this.child = child;
+        this.nullsLast = nullsLast;
+        this.orderAscending = orderAscending;
+    }
+    
+    public boolean isNullsLast() {
+        return nullsLast;
+    }
+    
+    public boolean isAscending() {
+        return orderAscending;
+    }
+    
+    public ParseNode getNode() {
+        return child;
+    }
+    
+    @Override
+    public String toString() {
+        return child.toString() + (orderAscending ? " asc" : " desc") + " nulls " + (nullsLast ? "last" : "first");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/OuterJoinParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/OuterJoinParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/OuterJoinParseNode.java
new file mode 100644
index 0000000..12ed92b
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/OuterJoinParseNode.java
@@ -0,0 +1,49 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.List;
+
+
+
+/**
+ * 
+ * Node representing an outer join qualifier (+) in SQL
+ * TODO: remove Oracle specific syntax
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class OuterJoinParseNode extends UnaryParseNode{
+    OuterJoinParseNode(ParseNode node) {
+        super(node);
+    }
+
+    @Override
+    public <T> T accept(ParseNodeVisitor<T> visitor) throws SQLException {
+        List<T> l = Collections.emptyList();
+        if (visitor.visitEnter(this)) {
+            l = acceptChildren(visitor);
+        }
+        return visitor.visitLeave(this, l);
+    }
+}


[48/51] [partial] Initial commit of master branch from github

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/IndexCodec.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/IndexCodec.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/IndexCodec.java
new file mode 100644
index 0000000..ba05307
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/IndexCodec.java
@@ -0,0 +1,112 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index.covered;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+
+import org.apache.phoenix.index.BaseIndexCodec;
+
+
+/**
+ * Codec for creating index updates from the current state of a table.
+ * <p>
+ * Generally, you should extend {@link BaseIndexCodec} instead, so help maintain compatibility as
+ * features need to be added to the codec, as well as potentially not haivng to implement some
+ * methods.
+ */
+public interface IndexCodec {
+
+  /**
+   * Do any code initialization necessary
+   * @param env environment in which the codec is operating
+   * @throws IOException if the codec cannot be initalized correctly
+   */
+  public void initialize(RegionCoprocessorEnvironment env) throws IOException;
+
+  /**
+   * Get the index cleanup entries. Currently, this must return just single row deletes (where just
+   * the row-key is specified and no columns are returned) mapped to the table name. For instance,
+   * to you have an index 'myIndex' with row :
+   * 
+   * <pre>
+   * v1,v2,v3 | CF:CQ0  | rowkey
+   *          | CF:CQ1  | rowkey
+   * </pre>
+   * 
+   * To then cleanup this entry, you would just return 'v1,v2,v3', 'myIndex'.
+   * @param state the current state of the table that needs to be cleaned up. Generally, you only
+   *          care about the latest column values, for each column you are indexing for each index
+   *          table.
+   * @return the pairs of (deletes, index table name) that should be applied.
+ * @throws IOException 
+   */
+  public Iterable<IndexUpdate> getIndexDeletes(TableState state) throws IOException;
+
+  // table state has the pending update already applied, before calling
+  // get the new index entries
+  /**
+   * Get the index updates for the primary table state, for each index table. The returned
+   * {@link Put}s need to be fully specified (including timestamp) to minimize passes over the same
+   * key-values multiple times.
+   * <p>
+   * You must specify the same timestamps on the Put as {@link TableState#getCurrentTimestamp()} so
+   * the index entries match the primary table row. This could be managed at a higher level, but
+   * would require iterating all the kvs in the Put again - very inefficient when compared to the
+   * current interface where you must provide a timestamp anyways (so you might as well provide the
+   * right one).
+   * @param state the current state of the table that needs to an index update Generally, you only
+   *          care about the latest column values, for each column you are indexing for each index
+   *          table.
+   * @return the pairs of (updates,index table name) that should be applied.
+ * @throws IOException 
+   */
+  public Iterable<IndexUpdate> getIndexUpserts(TableState state) throws IOException;
+
+  /**
+   * This allows the codec to dynamically change whether or not indexing should take place for a
+   * table. If it doesn't take place, we can save a lot of time on the regular Put patch. By making
+   * it dynamic, we can save offlining and then onlining a table just to turn indexing on.
+   * <p>
+   * We can also be smart about even indexing a given update here too - if the update doesn't
+   * contain any columns that we care about indexing, we can save the effort of analyzing the put
+   * and further.
+   * @param m mutation that should be indexed.
+   * @return <tt>true</tt> if indexing is enabled for the given table. This should be on a per-table
+   *         basis, as each codec is instantiated per-region.
+ * @throws IOException 
+   */
+  public boolean isEnabled(Mutation m) throws IOException;
+
+  /**
+   * Get the batch identifier of the given mutation. Generally, updates to the table will take place
+   * in a batch of updates; if we know that the mutation is part of a batch, we can build the state
+   * much more intelligently.
+   * <p>
+   * <b>If you have batches that have multiple updates to the same row state, you must specify a
+   * batch id for each batch. Otherwise, we cannot guarantee index correctness</b>
+   * @param m mutation that may or may not be part of the batch
+   * @return <tt>null</tt> if the mutation is not part of a batch or an id for the batch.
+   */
+  public byte[] getBatchId(Mutation m);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/IndexUpdate.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/IndexUpdate.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/IndexUpdate.java
new file mode 100644
index 0000000..3a1866e
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/IndexUpdate.java
@@ -0,0 +1,79 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index.covered;
+
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import org.apache.hadoop.hbase.index.covered.update.ColumnTracker;
+
+/**
+ * Update to make to the index table.
+ */
+public class IndexUpdate {
+  Mutation update;
+  byte[] tableName;
+  ColumnTracker columns;
+
+  IndexUpdate(ColumnTracker tracker) {
+    this.columns = tracker;
+  }
+
+  public void setUpdate(Mutation p) {
+    this.update = p;
+  }
+
+  public void setTable(byte[] tableName) {
+    this.tableName = tableName;
+  }
+
+  public Mutation getUpdate() {
+    return update;
+  }
+
+  public byte[] getTableName() {
+    return tableName;
+  }
+
+  public ColumnTracker getIndexedColumns() {
+    return columns;
+  }
+
+  @Override
+  public String toString() {
+    return "IndexUpdate: \n\ttable - " + Bytes.toString(tableName) + "\n\tupdate: " + update
+        + "\n\tcolumns: " + columns;
+  }
+
+  public static IndexUpdate createIndexUpdateForTesting(ColumnTracker tracker, byte[] table, Put p) {
+    IndexUpdate update = new IndexUpdate(tracker);
+    update.setTable(table);
+    update.setUpdate(p);
+    return update;
+  }
+
+  /**
+   * @return <tt>true</tt> if the necessary state for a valid index update has been set.
+   */
+  public boolean isValid() {
+    return this.tableName != null && this.update != null;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/KeyValueStore.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/KeyValueStore.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/KeyValueStore.java
new file mode 100644
index 0000000..1048831
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/KeyValueStore.java
@@ -0,0 +1,35 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index.covered;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
+
+/**
+ * Store a collection of KeyValues in memory.
+ */
+public interface KeyValueStore {
+
+  public void add(KeyValue kv, boolean overwrite);
+
+  public KeyValueScanner getScanner();
+
+  public void rollback(KeyValue kv);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/LocalTableState.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/LocalTableState.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/LocalTableState.java
new file mode 100644
index 0000000..1950864
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/LocalTableState.java
@@ -0,0 +1,244 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index.covered;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
+import org.apache.hadoop.hbase.util.Pair;
+
+import org.apache.hadoop.hbase.index.covered.data.IndexMemStore;
+import org.apache.hadoop.hbase.index.covered.data.LocalHBaseState;
+import org.apache.hadoop.hbase.index.covered.update.ColumnReference;
+import org.apache.hadoop.hbase.index.covered.update.ColumnTracker;
+import org.apache.hadoop.hbase.index.covered.update.IndexedColumnGroup;
+import org.apache.hadoop.hbase.index.scanner.Scanner;
+import org.apache.hadoop.hbase.index.scanner.ScannerBuilder;
+
+/**
+ * Manage the state of the HRegion's view of the table, for the single row.
+ * <p>
+ * Currently, this is a single-use object - you need to create a new one for each row that you need
+ * to manage. In the future, we could make this object reusable, but for the moment its easier to
+ * manage as a throw-away object.
+ * <p>
+ * This class is <b>not</b> thread-safe - it requires external synchronization is access
+ * concurrently.
+ */
+public class LocalTableState implements TableState {
+
+  private long ts;
+  private RegionCoprocessorEnvironment env;
+  private KeyValueStore memstore;
+  private LocalHBaseState table;
+  private Mutation update;
+  private Set<ColumnTracker> trackedColumns = new HashSet<ColumnTracker>();
+  private ScannerBuilder scannerBuilder;
+  private List<KeyValue> kvs = new ArrayList<KeyValue>();
+  private List<? extends IndexedColumnGroup> hints;
+  private CoveredColumns columnSet;
+
+  public LocalTableState(RegionCoprocessorEnvironment environment, LocalHBaseState table, Mutation update) {
+    this.env = environment;
+    this.table = table;
+    this.update = update;
+    this.memstore = new IndexMemStore();
+    this.scannerBuilder = new ScannerBuilder(memstore, update);
+    this.columnSet = new CoveredColumns();
+  }
+
+  public void addPendingUpdates(KeyValue... kvs) {
+    if (kvs == null) return;
+    addPendingUpdates(Arrays.asList(kvs));
+  }
+
+  public void addPendingUpdates(List<KeyValue> kvs) {
+    if(kvs == null) return;
+    setPendingUpdates(kvs);
+    addUpdate(kvs);
+  }
+
+  private void addUpdate(List<KeyValue> list) {
+    addUpdate(list, true);
+  }
+
+  private void addUpdate(List<KeyValue> list, boolean overwrite) {
+    if (list == null) return;
+    for (KeyValue kv : list) {
+      this.memstore.add(kv, overwrite);
+    }
+  }
+
+  @Override
+  public RegionCoprocessorEnvironment getEnvironment() {
+    return this.env;
+  }
+
+  @Override
+  public long getCurrentTimestamp() {
+    return this.ts;
+  }
+
+  @Override
+  public void setCurrentTimestamp(long timestamp) {
+    this.ts = timestamp;
+  }
+
+  public void resetTrackedColumns() {
+    this.trackedColumns.clear();
+  }
+
+  public Set<ColumnTracker> getTrackedColumns() {
+    return this.trackedColumns;
+  }
+
+  @Override
+  public Pair<Scanner, IndexUpdate> getIndexedColumnsTableState(
+      Collection<? extends ColumnReference> indexedColumns) throws IOException {
+    ensureLocalStateInitialized(indexedColumns);
+    // filter out things with a newer timestamp and track the column references to which it applies
+    ColumnTracker tracker = new ColumnTracker(indexedColumns);
+    synchronized (this.trackedColumns) {
+      // we haven't seen this set of columns before, so we need to create a new tracker
+      if (!this.trackedColumns.contains(tracker)) {
+        this.trackedColumns.add(tracker);
+      }
+    }
+
+    Scanner scanner =
+        this.scannerBuilder.buildIndexedColumnScanner(indexedColumns, tracker, ts);
+
+    return new Pair<Scanner, IndexUpdate>(scanner, new IndexUpdate(tracker));
+  }
+
+  /**
+   * Initialize the managed local state. Generally, this will only be called by
+   * {@link #getNonIndexedColumnsTableState(List)}, which is unlikely to be called concurrently from the outside.
+   * Even then, there is still fairly low contention as each new Put/Delete will have its own table
+   * state.
+   */
+  private synchronized void ensureLocalStateInitialized(
+      Collection<? extends ColumnReference> columns) throws IOException {
+    // check to see if we haven't initialized any columns yet
+    Collection<? extends ColumnReference> toCover = this.columnSet.findNonCoveredColumns(columns);
+    // we have all the columns loaded, so we are good to go.
+    if (toCover.isEmpty()) {
+      return;
+    }
+
+    // add the current state of the row
+    this.addUpdate(this.table.getCurrentRowState(update, toCover).list(), false);
+
+    // add the covered columns to the set
+    for (ColumnReference ref : toCover) {
+      this.columnSet.addColumn(ref);
+    }
+  }
+
+  @Override
+  public Map<String, byte[]> getUpdateAttributes() {
+    return this.update.getAttributesMap();
+  }
+
+  @Override
+  public byte[] getCurrentRowKey() {
+    return this.update.getRow();
+  }
+
+  public Result getCurrentRowState() {
+    KeyValueScanner scanner = this.memstore.getScanner();
+    List<KeyValue> kvs = new ArrayList<KeyValue>();
+    while (scanner.peek() != null) {
+      try {
+        kvs.add(scanner.next());
+      } catch (IOException e) {
+        // this should never happen - something has gone terribly arwy if it has
+        throw new RuntimeException("Local MemStore threw IOException!");
+      }
+    }
+    return new Result(kvs);
+  }
+
+  /**
+   * Helper to add a {@link Mutation} to the values stored for the current row
+   * @param pendingUpdate update to apply
+   */
+  public void addUpdateForTesting(Mutation pendingUpdate) {
+    for (Map.Entry<byte[], List<KeyValue>> e : pendingUpdate.getFamilyMap().entrySet()) {
+      List<KeyValue> edits = e.getValue();
+      addUpdate(edits);
+    }
+  }
+
+  /**
+   * @param hints
+   */
+  public void setHints(List<? extends IndexedColumnGroup> hints) {
+    this.hints = hints;
+  }
+
+  @Override
+  public List<? extends IndexedColumnGroup> getIndexColumnHints() {
+    return this.hints;
+  }
+
+  @Override
+  public Collection<KeyValue> getPendingUpdate() {
+    return this.kvs;
+  }
+
+  /**
+   * Set the {@link KeyValue}s in the update for which we are currently building an index update,
+   * but don't actually apply them.
+   * @param update pending {@link KeyValue}s
+   */
+  public void setPendingUpdates(Collection<KeyValue> update) {
+    this.kvs.clear();
+    this.kvs.addAll(update);
+  }
+
+  /**
+   * Apply the {@link KeyValue}s set in {@link #setPendingUpdates(Collection)}.
+   */
+  public void applyPendingUpdates() {
+    this.addUpdate(kvs);
+  }
+
+  /**
+   * Rollback all the given values from the underlying state.
+   * @param values
+   */
+  public void rollback(Collection<KeyValue> values) {
+    for (KeyValue kv : values) {
+      this.memstore.rollback(kv);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/TableState.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/TableState.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/TableState.java
new file mode 100644
index 0000000..52d4e85
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/TableState.java
@@ -0,0 +1,98 @@
+package org.apache.hadoop.hbase.index.covered;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.util.Pair;
+
+import org.apache.hadoop.hbase.index.covered.update.ColumnReference;
+import org.apache.hadoop.hbase.index.covered.update.IndexedColumnGroup;
+import org.apache.hadoop.hbase.index.scanner.Scanner;
+
+/**
+ * Interface for the current state of the table. This is generally going to be as of a timestamp - a
+ * view on the current state of the HBase table - so you don't have to worry about exposing too much
+ * information.
+ */
+public interface TableState {
+
+  // use this to get batch ids/ptable stuff
+  /**
+   * WARNING: messing with this can affect the indexing plumbing. Use with caution :)
+   * @return get the current environment in which this table lives.
+   */
+  public RegionCoprocessorEnvironment getEnvironment();
+
+  /**
+   * @return the current timestamp up-to-which we are releasing table state.
+   */
+  public long getCurrentTimestamp();
+
+  /**
+   * Set the current timestamp up to which the table should allow access to the underlying table.
+   * This overrides the timestamp view provided by the indexer - use with care!
+   * @param timestamp timestamp up to which the table should allow access.
+   */
+  public void setCurrentTimestamp(long timestamp);
+
+  /**
+   * @return the attributes attached to the current update (e.g. {@link Mutation}).
+   */
+  public Map<String, byte[]> getUpdateAttributes();
+
+  /**
+   * Get a scanner on the columns that are needed by the index.
+   * <p>
+   * The returned scanner is already pre-seeked to the first {@link KeyValue} that matches the given
+   * columns with a timestamp earlier than the timestamp to which the table is currently set (the
+   * current state of the table for which we need to build an update).
+   * <p>
+   * If none of the passed columns matches any of the columns in the pending update (as determined
+   * by {@link ColumnReference#matchesFamily(byte[])} and
+   * {@link ColumnReference#matchesQualifier(byte[])}, then an empty scanner will be returned. This
+   * is because it doesn't make sense to build index updates when there is no change in the table
+   * state for any of the columns you are indexing.
+   * <p>
+   * <i>NOTE:</i> This method should <b>not</b> be used during
+   * {@link IndexCodec#getIndexDeletes(TableState)} as the pending update will not yet have been
+   * applied - you are merely attempting to cleanup the current state and therefore do <i>not</i>
+   * need to track the indexed columns.
+   * <p>
+   * As a side-effect, we update a timestamp for the next-most-recent timestamp for the columns you
+   * request - you will never see a column with the timestamp we are tracking, but the next oldest
+   * timestamp for that column.
+   * @param indexedColumns the columns to that will be indexed
+   * @return an iterator over the columns and the {@link IndexUpdate} that should be passed back to
+   *         the builder. Even if no update is necessary for the requested columns, you still need
+   *         to return the {@link IndexUpdate}, just don't set the update for the
+   *         {@link IndexUpdate}.
+   * @throws IOException
+   */
+  Pair<Scanner, IndexUpdate> getIndexedColumnsTableState(
+      Collection<? extends ColumnReference> indexedColumns) throws IOException;
+
+  /**
+   * @return the row key for the current row for which we are building an index update.
+   */
+  byte[] getCurrentRowKey();
+
+  /**
+   * Get the 'hint' for the columns that were indexed last time for the same set of keyvalues.
+   * Generally, this will only be used when fixing up a 'back in time' put or delete as we need to
+   * fix up all the indexes that reference the changed columns.
+   * @return the hint the index columns that were queried on the last iteration for the changed
+   *         column
+   */
+  List<? extends IndexedColumnGroup> getIndexColumnHints();
+
+  /**
+   * Can be used to help the codec to determine which columns it should attempt to index.
+   * @return the keyvalues in the pending update to the table.
+   */
+  Collection<KeyValue> getPendingUpdate();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/data/IndexMemStore.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/data/IndexMemStore.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/data/IndexMemStore.java
new file mode 100644
index 0000000..ac29dd1
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/data/IndexMemStore.java
@@ -0,0 +1,333 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index.covered.data;
+
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.SortedSet;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValue.KeyComparator;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.regionserver.IndexKeyValueSkipListSet;
+import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
+import org.apache.hadoop.hbase.regionserver.MemStore;
+import org.apache.hadoop.hbase.regionserver.NonLazyKeyValueScanner;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import org.apache.hadoop.hbase.index.covered.KeyValueStore;
+import org.apache.hadoop.hbase.index.covered.LocalTableState;
+
+/**
+ * Like the HBase {@link MemStore}, but without all that extra work around maintaining snapshots and
+ * sizing (for right now). We still support the concurrent access (in case indexes are built in
+ * parallel).
+ * <p>
+ * 
+ We basically wrap a KeyValueSkipListSet, just like a regular MemStore, except we are:
+ * <ol>
+ *  <li>not dealing with
+ *    <ul>
+ *      <li>space considerations</li>
+ *      <li>a snapshot set</li>
+ *    </ul>
+ *  </li>
+ *  <li>ignoring memstore timestamps in favor of deciding when we want to overwrite keys based on how
+ *    we obtain them</li>
+ *   <li>ignoring time range updates (so 
+ *    {@link KeyValueScanner#shouldUseScanner(Scan, SortedSet, long)} isn't supported from 
+ *    {@link #getScanner()}).</li>
+ * </ol>
+ * <p>
+ * We can ignore the memstore timestamps because we know that anything we get from the local region
+ * is going to be MVCC visible - so it should just go in. However, we also want overwrite any
+ * existing state with our pending write that we are indexing, so that needs to clobber the KVs we
+ * get from the HRegion. This got really messy with a regular memstore as each KV from the MemStore
+ * frequently has a higher MemStoreTS, but we can't just up the pending KVs' MemStoreTs because a
+ * memstore relies on the MVCC readpoint, which generally is less than {@link Long#MAX_VALUE}.
+ * <p>
+ * By realizing that we don't need the snapshot or space requirements, we can go much faster than
+ * the previous implementation. Further, by being smart about how we manage the KVs, we can drop the
+ * extra object creation we were doing to wrap the pending KVs (which we did previously to ensure
+ * they sorted before the ones we got from the HRegion). We overwrite {@link KeyValue}s when we add
+ * them from external sources {@link #add(KeyValue, boolean)}, but then don't overwrite existing
+ * keyvalues when read them from the underlying table (because pending keyvalues should always
+ * overwrite current ones) - this logic is all contained in LocalTableState.
+ * @see LocalTableState
+ */
+public class IndexMemStore implements KeyValueStore {
+
+  private static final Log LOG = LogFactory.getLog(IndexMemStore.class);
+  private IndexKeyValueSkipListSet kvset;
+  private Comparator<KeyValue> comparator;
+
+  /**
+   * Compare two {@link KeyValue}s based only on their row keys. Similar to the standard
+   * {@link KeyValue#COMPARATOR}, but doesn't take into consideration the memstore timestamps. We
+   * instead manage which KeyValue to retain based on how its loaded here
+   */
+  public static final Comparator<KeyValue> COMPARATOR = new Comparator<KeyValue>() {
+
+    private final KeyComparator rawcomparator = new KeyComparator();
+
+    @Override
+    public int compare(final KeyValue left, final KeyValue right) {
+      return rawcomparator.compare(left.getBuffer(), left.getOffset() + KeyValue.ROW_OFFSET,
+        left.getKeyLength(), right.getBuffer(), right.getOffset() + KeyValue.ROW_OFFSET,
+        right.getKeyLength());
+    }
+  };
+
+  public IndexMemStore() {
+    this(COMPARATOR);
+  }
+
+  /**
+   * Create a store with the given comparator. This comparator is used to determine both sort order
+   * <b>as well as equality of {@link KeyValue}s</b>.
+   * <p>
+   * Exposed for subclassing/testing.
+   * @param comparator to use
+   */
+  IndexMemStore(Comparator<KeyValue> comparator) {
+    this.comparator = comparator;
+    this.kvset = IndexKeyValueSkipListSet.create(comparator);
+  }
+
+  @Override
+  public void add(KeyValue kv, boolean overwrite) {
+    if (LOG.isDebugEnabled()) {
+      LOG.info("Inserting: " + toString(kv));
+    }
+    // if overwriting, we will always update
+    if (!overwrite) {
+      // null if there was no previous value, so we added the kv
+      kvset.putIfAbsent(kv);
+    } else {
+      kvset.add(kv);
+    }
+
+    if (LOG.isTraceEnabled()) {
+      dump();
+    }
+  }
+
+  private void dump() {
+    LOG.trace("Current kv state:\n");
+    for (KeyValue kv : this.kvset) {
+      LOG.trace("KV: " + toString(kv));
+    }
+    LOG.trace("========== END MemStore Dump ==================\n");
+  }
+
+  private String toString(KeyValue kv) {
+    return kv.toString() + "/value=" + Bytes.toString(kv.getValue());
+  }
+
+  @Override
+  public void rollback(KeyValue kv) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Rolling back: " + toString(kv));
+    }
+    // If the key is in the store, delete it
+    this.kvset.remove(kv);
+    if (LOG.isTraceEnabled()) {
+      dump();
+    }
+  }
+
+  @Override
+  public KeyValueScanner getScanner() {
+    return new MemStoreScanner();
+  }
+
+  /*
+   * MemStoreScanner implements the KeyValueScanner. It lets the caller scan the contents of a
+   * memstore -- both current map and snapshot. This behaves as if it were a real scanner but does
+   * not maintain position.
+   */
+  // This class is adapted from org.apache.hadoop.hbase.MemStore.MemStoreScanner, HBase 0.94.12
+  // It does basically the same thing as the MemStoreScanner, but it only keeps track of a single
+  // set, rather than a primary and a secondary set of KeyValues.
+  protected class MemStoreScanner extends NonLazyKeyValueScanner {
+    // Next row information for the set
+    private KeyValue nextRow = null;
+
+    // last iterated KVs for kvset and snapshot (to restore iterator state after reseek)
+    private KeyValue kvsetItRow = null;
+
+    // iterator based scanning.
+    private Iterator<KeyValue> kvsetIt;
+
+    // The kvset at the time of creating this scanner
+    volatile IndexKeyValueSkipListSet kvsetAtCreation;
+
+    MemStoreScanner() {
+      super();
+      kvsetAtCreation = kvset;
+    }
+
+    private KeyValue getNext(Iterator<KeyValue> it) {
+      // in the original implementation we cared about the current thread's readpoint from MVCC.
+      // However, we don't need to worry here because everything the index can see, is also visible
+      // to the client (or is the pending primary table update, so it will be once the index is
+      // written, so it might as well be).
+      KeyValue v = null;
+      try {
+        while (it.hasNext()) {
+          v = it.next();
+          return v;
+        }
+
+        return null;
+      } finally {
+        if (v != null) {
+          kvsetItRow = v;
+        }
+      }
+    }
+
+    /**
+     * Set the scanner at the seek key. Must be called only once: there is no thread safety between
+     * the scanner and the memStore.
+     * @param key seek value
+     * @return false if the key is null or if there is no data
+     */
+    @Override
+    public synchronized boolean seek(KeyValue key) {
+      if (key == null) {
+        close();
+        return false;
+      }
+
+      // kvset and snapshot will never be null.
+      // if tailSet can't find anything, SortedSet is empty (not null).
+      kvsetIt = kvsetAtCreation.tailSet(key).iterator();
+      kvsetItRow = null;
+
+      return seekInSubLists(key);
+    }
+
+    /**
+     * (Re)initialize the iterators after a seek or a reseek.
+     */
+    private synchronized boolean seekInSubLists(KeyValue key) {
+      nextRow = getNext(kvsetIt);
+      return nextRow != null;
+    }
+
+    /**
+     * Move forward on the sub-lists set previously by seek.
+     * @param key seek value (should be non-null)
+     * @return true if there is at least one KV to read, false otherwise
+     */
+    @Override
+    public synchronized boolean reseek(KeyValue key) {
+      /*
+       * See HBASE-4195 & HBASE-3855 & HBASE-6591 for the background on this implementation. This
+       * code is executed concurrently with flush and puts, without locks. Two points must be known
+       * when working on this code: 1) It's not possible to use the 'kvTail' and 'snapshot'
+       * variables, as they are modified during a flush. 2) The ideal implementation for performance
+       * would use the sub skip list implicitly pointed by the iterators 'kvsetIt' and 'snapshotIt'.
+       * Unfortunately the Java API does not offer a method to get it. So we remember the last keys
+       * we iterated to and restore the reseeked set to at least that point.
+       */
+
+      kvsetIt = kvsetAtCreation.tailSet(getHighest(key, kvsetItRow)).iterator();
+      return seekInSubLists(key);
+    }
+
+    /*
+     * Returns the higher of the two key values, or null if they are both null. This uses
+     * comparator.compare() to compare the KeyValue using the memstore comparator.
+     */
+    private KeyValue getHighest(KeyValue first, KeyValue second) {
+      if (first == null && second == null) {
+        return null;
+      }
+      if (first != null && second != null) {
+        int compare = comparator.compare(first, second);
+        return (compare > 0 ? first : second);
+      }
+      return (first != null ? first : second);
+    }
+
+    @Override
+    public synchronized KeyValue peek() {
+      // DebugPrint.println(" MS@" + hashCode() + " peek = " + getLowest());
+      return nextRow;
+    }
+
+    @Override
+    public synchronized KeyValue next() {
+      if (nextRow == null) {
+        return null;
+      }
+
+      final KeyValue ret = nextRow;
+
+      // Advance the iterators
+      nextRow = getNext(kvsetIt);
+
+      return ret;
+    }
+
+    @Override
+    public synchronized void close() {
+      this.nextRow = null;
+      this.kvsetIt = null;
+      this.kvsetItRow = null;
+    }
+
+    /**
+     * MemStoreScanner returns max value as sequence id because it will always have the latest data
+     * among all files.
+     */
+    @Override
+    public long getSequenceID() {
+      return Long.MAX_VALUE;
+    }
+
+    @Override
+    public boolean shouldUseScanner(Scan scan, SortedSet<byte[]> columns, long oldestUnexpiredTS) {
+      throw new UnsupportedOperationException(this.getClass().getName()
+          + " doesn't support checking to see if it should use a scanner!");
+    }
+
+    /*
+    @Override
+    public boolean backwardSeek(KeyValue arg0) throws IOException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public boolean seekToLastRow() throws IOException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public boolean seekToPreviousRow(KeyValue arg0) throws IOException {
+        throw new UnsupportedOperationException();
+    }
+    */
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/data/LazyValueGetter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/data/LazyValueGetter.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/data/LazyValueGetter.java
new file mode 100644
index 0000000..619a4cf
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/data/LazyValueGetter.java
@@ -0,0 +1,89 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index.covered.data;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.KeyValue;
+
+import org.apache.hadoop.hbase.index.ValueGetter;
+import org.apache.hadoop.hbase.index.covered.update.ColumnReference;
+import org.apache.hadoop.hbase.index.scanner.Scanner;
+import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+
+/**
+ * {@link ValueGetter} that uses lazy initialization to get the value for the given
+ * {@link ColumnReference}. Once stored, the mapping for that reference is retained.
+ */
+public class LazyValueGetter implements ValueGetter {
+
+  private Scanner scan;
+  private volatile Map<ColumnReference, ImmutableBytesPtr> values;
+  private byte[] row;
+  
+  /**
+   * Back the getter with a {@link Scanner} to actually access the local data.
+   * @param scan backing scanner
+   * @param currentRow row key for the row to seek in the scanner
+   */
+  public LazyValueGetter(Scanner scan, byte[] currentRow) {
+    this.scan = scan;
+    this.row = currentRow;
+  }
+
+  @Override
+  public ImmutableBytesPtr getLatestValue(ColumnReference ref) throws IOException {
+    // ensure we have a backing map
+    if (values == null) {
+      synchronized (this) {
+        values = Collections.synchronizedMap(new HashMap<ColumnReference, ImmutableBytesPtr>());
+      }
+    }
+
+    // check the value in the map
+    ImmutableBytesPtr value = values.get(ref);
+    if (value == null) {
+      value = get(ref);
+      values.put(ref, value);
+    }
+
+    return value;
+  }
+
+  /**
+   * @param ref
+   * @return the first value on the scanner for the given column
+   */
+  private ImmutableBytesPtr get(ColumnReference ref) throws IOException {
+    KeyValue first = ref.getFirstKeyValueForRow(row);
+    if (!scan.seek(first)) {
+      return null;
+    }
+    // there is a next value - we only care about the current value, so we can just snag that
+    KeyValue next = scan.next();
+    if (ref.matches(next)) {
+      return new ImmutableBytesPtr(next.getBuffer(), next.getValueOffset(), next.getValueLength());
+    }
+    return null;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/data/LocalHBaseState.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/data/LocalHBaseState.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/data/LocalHBaseState.java
new file mode 100644
index 0000000..0058efa
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/data/LocalHBaseState.java
@@ -0,0 +1,49 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index.covered.data;
+
+import java.io.IOException;
+import java.util.Collection;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Result;
+
+import org.apache.hadoop.hbase.index.covered.update.ColumnReference;
+
+/**
+ * Access the current state of the row in the local HBase table, given a mutation
+ */
+public interface LocalHBaseState {
+
+  /**
+   * @param m mutation for which we should get the current table state
+   * @param toCover all the columns the current row state needs to cover; hint the underlying lookup
+   *          to save getting all the columns for the row
+   * @return the full state of the given row. Includes all current versions (even if they are not
+   *         usually visible to the client (unless they are also doing a raw scan)). Never returns a
+   *         <tt>null</tt> {@link Result} - instead, when there is not data for the row, returns a
+   *         {@link Result} with no stored {@link KeyValue}s.
+   * @throws IOException if there is an issue reading the row
+   */
+  public Result getCurrentRowState(Mutation m, Collection<? extends ColumnReference> toCover)
+      throws IOException;
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/data/LocalTable.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/data/LocalTable.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/data/LocalTable.java
new file mode 100644
index 0000000..2790523
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/data/LocalTable.java
@@ -0,0 +1,74 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index.covered.data;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.RegionScanner;
+
+import org.apache.hadoop.hbase.index.covered.update.ColumnReference;
+import org.apache.hadoop.hbase.index.util.IndexManagementUtil;
+
+/**
+ * Wrapper around a lazily instantiated, local HTable.
+ * <p>
+ * Previously, we had used various row and batch caches. However, this ends up being very
+ * complicated when attempting manage updating and invalidating the cache with no real gain as any
+ * row accessed multiple times will likely be in HBase's block cache, invalidating any extra caching
+ * we are doing here. In the end, its simpler and about as efficient to just get the current state
+ * of the row from HBase and let HBase manage caching the row from disk on its own.
+ */
+public class LocalTable implements LocalHBaseState {
+
+  private RegionCoprocessorEnvironment env;
+
+  public LocalTable(RegionCoprocessorEnvironment env) {
+    this.env = env;
+  }
+
+  @Override
+  public Result getCurrentRowState(Mutation m, Collection<? extends ColumnReference> columns)
+      throws IOException {
+    byte[] row = m.getRow();
+    // need to use a scan here so we can get raw state, which Get doesn't provide.
+    Scan s = IndexManagementUtil.newLocalStateScan(Collections.singletonList(columns));
+    s.setStartRow(row);
+    s.setStopRow(row);
+    HRegion region = this.env.getRegion();
+    RegionScanner scanner = region.getScanner(s);
+    List<KeyValue> kvs = new ArrayList<KeyValue>(1);
+    boolean more = scanner.next(kvs);
+    assert !more : "Got more than one result when scanning" + " a single row in the primary table!";
+
+    Result r = new Result(kvs);
+    scanner.close();
+    return r;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/example/ColumnGroup.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/example/ColumnGroup.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/example/ColumnGroup.java
new file mode 100644
index 0000000..9f923b1
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/example/ColumnGroup.java
@@ -0,0 +1,95 @@
+
+package org.apache.hadoop.hbase.index.covered.example;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.hadoop.hbase.util.Bytes;
+
+
+/**
+ * A collection of {@link CoveredColumn}s that should be included in a covered index.
+ */
+public class ColumnGroup implements Iterable<CoveredColumn> {
+
+  private List<CoveredColumn> columns = new ArrayList<CoveredColumn>();
+  private String table;
+
+  public ColumnGroup(String tableName) {
+    this.table = tableName;
+  }
+
+  public void add(CoveredColumn column) {
+    this.columns.add(column);
+  }
+
+  public String getTable() {
+    return table;
+  }
+
+  /**
+   * Check to see if any {@link CoveredColumn} in <tt>this</tt> matches the given family
+   * @param family to check
+   * @return <tt>true</tt> if any column covers the family
+   */
+  public boolean matches(String family) {
+    for (CoveredColumn column : columns) {
+      if (column.matchesFamily(family)) {
+        return true;
+      }
+    }
+
+    return false;
+  }
+
+  /**
+   * Check to see if any column matches the family/qualifier pair
+   * @param family family to match against
+   * @param qualifier qualifier to match, can be <tt>null</tt>, in which case we match all
+   *          qualifiers
+   * @return <tt>true</tt> if any column matches, <tt>false</tt> otherwise
+   */
+  public boolean matches(byte[] family, byte[] qualifier) {
+    // families are always printable characters
+    String fam = Bytes.toString(family);
+    for (CoveredColumn column : columns) {
+      if (column.matchesFamily(fam)) {
+        // check the qualifier
+          if (column.matchesQualifier(qualifier)) {
+            return true;
+        }
+      }
+    }
+    return false;
+  }
+
+  /**
+   * @return the number of columns in the group
+   */
+  public int size() {
+    return this.columns.size();
+  }
+
+  @Override
+  public Iterator<CoveredColumn> iterator() {
+    return columns.iterator();
+  }
+
+  /**
+   * @param index index of the column to get
+   * @return the specified column
+   */
+  public CoveredColumn getColumnForTesting(int index) {
+    return this.columns.get(index);
+  }
+
+  @Override
+  public String toString() {
+    return "ColumnGroup - table: " + table + ", columns: " + columns;
+  }
+
+  public List<CoveredColumn> getColumns() {
+    return this.columns;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/example/CoveredColumn.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/example/CoveredColumn.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/example/CoveredColumn.java
new file mode 100644
index 0000000..ef4b2ba
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/example/CoveredColumn.java
@@ -0,0 +1,89 @@
+package org.apache.hadoop.hbase.index.covered.example;
+
+import org.apache.hadoop.hbase.util.Bytes;
+
+import org.apache.hadoop.hbase.index.covered.update.ColumnReference;
+
+/**
+ * A single Column (either a Column Family or a full Family:Qualifier pair) in a {@link ColumnGroup}
+ * . If no column qualifier is specified (null), matches all known qualifiers of the family.
+ */
+public class CoveredColumn extends ColumnReference {
+
+  public static final String SEPARATOR = ":";
+  String familyString;
+  private final int hashCode;
+
+  public CoveredColumn(byte[] family, byte[] qualifier){
+    this(Bytes.toString(family), qualifier);
+  }
+
+  public CoveredColumn(String family, byte[] qualifier) {
+    super(Bytes.toBytes(family), qualifier == null ? ColumnReference.ALL_QUALIFIERS : qualifier);
+    this.familyString = family;
+    this.hashCode = calcHashCode(family, qualifier);
+  }
+
+  public static CoveredColumn parse(String spec) {
+    int sep = spec.indexOf(SEPARATOR);
+    if (sep < 0) {
+      throw new IllegalArgumentException(spec + " is not a valid specifier!");
+    }
+    String family = spec.substring(0, sep);
+    String qual = spec.substring(sep + 1);
+    byte[] column = qual.length() == 0 ? null : Bytes.toBytes(qual);
+    return new CoveredColumn(family, column);
+  }
+
+  public String serialize() {
+    return CoveredColumn.serialize(familyString, qualifier);
+  }
+
+  public static String serialize(String first, byte[] second) {
+    String nextValue = first + CoveredColumn.SEPARATOR;
+    if (second != null) {
+      nextValue += Bytes.toString(second);
+    }
+    return nextValue;
+  }
+
+  /**
+   * @param family2 to check
+   * @return <tt>true</tt> if the passed family matches the family this column covers
+   */
+  public boolean matchesFamily(String family2) {
+    return this.familyString.equals(family2);
+  }
+
+  @Override
+  public int hashCode() {
+    return hashCode;
+  }
+
+  private static int calcHashCode(String familyString, byte[] qualifier) {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + familyString.hashCode();
+    if (qualifier != null) {
+      result = prime * result + Bytes.hashCode(qualifier);
+    }
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) return true;
+    if (!super.equals(obj)) return false;
+    if (getClass() != obj.getClass()) return false;
+    CoveredColumn other = (CoveredColumn) obj;
+    if (hashCode != other.hashCode) return false;
+    if (!familyString.equals(other.familyString)) return false;
+    return Bytes.equals(qualifier, other.qualifier);
+  }
+
+  @Override
+  public String toString() {
+    String qualString = qualifier == null ? "null" : Bytes.toString(qualifier);
+    return "CoveredColumn:[" + familyString + ":" + qualString + "]";
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/example/CoveredColumnIndexCodec.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/example/CoveredColumnIndexCodec.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/example/CoveredColumnIndexCodec.java
new file mode 100644
index 0000000..8f0ee99
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/example/CoveredColumnIndexCodec.java
@@ -0,0 +1,367 @@
+/**
+ * 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.hadoop.hbase.index.covered.example;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map.Entry;
+
+import org.apache.commons.lang.ArrayUtils;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+
+import com.google.common.collect.Lists;
+import org.apache.hadoop.hbase.index.covered.IndexUpdate;
+import org.apache.hadoop.hbase.index.covered.TableState;
+import org.apache.hadoop.hbase.index.scanner.Scanner;
+import org.apache.phoenix.index.BaseIndexCodec;
+
+/**
+ *
+ */
+public class CoveredColumnIndexCodec extends BaseIndexCodec {
+
+  private static final byte[] EMPTY_BYTES = new byte[0];
+  public static final byte[] INDEX_ROW_COLUMN_FAMILY = Bytes.toBytes("INDEXED_COLUMNS");
+
+  private List<ColumnGroup> groups;
+
+  /**
+   * @param groups to initialize the codec with
+   * @return an instance that is initialized with the given {@link ColumnGroup}s, for testing
+   *         purposes
+   */
+  public static CoveredColumnIndexCodec getCodecForTesting(List<ColumnGroup> groups) {
+    CoveredColumnIndexCodec codec = new CoveredColumnIndexCodec();
+    codec.groups = Lists.newArrayList(groups);
+    return codec;
+  }
+
+  @Override
+  public void initialize(RegionCoprocessorEnvironment env) {
+    groups = CoveredColumnIndexSpecifierBuilder.getColumns(env.getConfiguration());
+  }
+
+  @Override
+  public Iterable<IndexUpdate> getIndexUpserts(TableState state) {
+    List<IndexUpdate> updates = new ArrayList<IndexUpdate>();
+    for (ColumnGroup group : groups) {
+      IndexUpdate update = getIndexUpdateForGroup(group, state);
+      updates.add(update);
+    }
+    return updates;
+  }
+
+  /**
+   * @param group
+   * @param state
+   * @return the update that should be made to the table
+   */
+  private IndexUpdate getIndexUpdateForGroup(ColumnGroup group, TableState state) {
+    List<CoveredColumn> refs = group.getColumns();
+    try {
+      Pair<Scanner, IndexUpdate> stateInfo = state.getIndexedColumnsTableState(refs);
+      Scanner kvs = stateInfo.getFirst();
+      Pair<Integer, List<ColumnEntry>> columns =
+          getNextEntries(refs, kvs, state.getCurrentRowKey());
+      // make sure we close the scanner
+      kvs.close();
+      if (columns.getFirst().intValue() == 0) {
+        return stateInfo.getSecond();
+      }
+      // have all the column entries, so just turn it into a Delete for the row
+      // convert the entries to the needed values
+      byte[] rowKey =
+          composeRowKey(state.getCurrentRowKey(), columns.getFirst(), columns.getSecond());
+      Put p = new Put(rowKey, state.getCurrentTimestamp());
+      // add the columns to the put
+      addColumnsToPut(p, columns.getSecond());
+
+      // update the index info
+      IndexUpdate update = stateInfo.getSecond();
+      update.setTable(Bytes.toBytes(group.getTable()));
+      update.setUpdate(p);
+      return update;
+    } catch (IOException e) {
+      throw new RuntimeException("Unexpected exception when getting state for columns: " + refs);
+    }
+  }
+
+  private static void addColumnsToPut(Put indexInsert, List<ColumnEntry> columns) {
+    // add each of the corresponding families to the put
+    int count = 0;
+    for (ColumnEntry column : columns) {
+      indexInsert.add(INDEX_ROW_COLUMN_FAMILY,
+        ArrayUtils.addAll(Bytes.toBytes(count++), toIndexQualifier(column.ref)), null);
+    }
+  }
+
+  private static byte[] toIndexQualifier(CoveredColumn column) {
+    return ArrayUtils.addAll(Bytes.toBytes(column.familyString + CoveredColumn.SEPARATOR),
+      column.getQualifier());
+  }
+
+  @Override
+  public Iterable<IndexUpdate> getIndexDeletes(TableState state) {
+    List<IndexUpdate> deletes = new ArrayList<IndexUpdate>();
+    for (ColumnGroup group : groups) {
+      deletes.add(getDeleteForGroup(group, state));
+    }
+    return deletes;
+  }
+
+
+  /**
+   * Get all the deletes necessary for a group of columns - logically, the cleanup the index table
+   * for a given index.
+   * @param group index information
+   * @return the cleanup for the given index, or <tt>null</tt> if no cleanup is necessary
+   */
+  private IndexUpdate getDeleteForGroup(ColumnGroup group, TableState state) {
+    List<CoveredColumn> refs = group.getColumns();
+    try {
+      Pair<Scanner, IndexUpdate> kvs = state.getIndexedColumnsTableState(refs);
+      Pair<Integer, List<ColumnEntry>> columns =
+          getNextEntries(refs, kvs.getFirst(), state.getCurrentRowKey());
+      // make sure we close the scanner reference
+      kvs.getFirst().close();
+      // no change, just return the passed update
+      if (columns.getFirst() == 0) {
+        return kvs.getSecond();
+      }
+      // have all the column entries, so just turn it into a Delete for the row
+      // convert the entries to the needed values
+      byte[] rowKey =
+          composeRowKey(state.getCurrentRowKey(), columns.getFirst(), columns.getSecond());
+      Delete d = new Delete(rowKey);
+      d.setTimestamp(state.getCurrentTimestamp());
+      IndexUpdate update = kvs.getSecond();
+      update.setUpdate(d);
+      update.setTable(Bytes.toBytes(group.getTable()));
+      return update;
+    } catch (IOException e) {
+      throw new RuntimeException("Unexpected exception when getting state for columns: " + refs);
+    }
+  }
+
+  /**
+   * Get the next batch of primary table values for the given columns
+   * @param refs columns to match against
+   * @param state
+   * @return the total length of all values found and the entries to add for the index
+   */
+  private Pair<Integer, List<ColumnEntry>> getNextEntries(List<CoveredColumn> refs, Scanner kvs,
+      byte[] currentRow) throws IOException {
+    int totalValueLength = 0;
+    List<ColumnEntry> entries = new ArrayList<ColumnEntry>(refs.size());
+
+    // pull out the latest state for each column reference, in order
+    for (CoveredColumn ref : refs) {
+      KeyValue first = ref.getFirstKeyValueForRow(currentRow);
+      if (!kvs.seek(first)) {
+        // no more keys, so add a null value
+        entries.add(new ColumnEntry(null, ref));
+        continue;
+      }
+      // there is a next value - we only care about the current value, so we can just snag that
+      KeyValue next = kvs.next();
+      if (ref.matchesFamily(next.getFamily()) && ref.matchesQualifier(next.getQualifier())) {
+        byte[] v = next.getValue();
+        totalValueLength += v.length;
+        entries.add(new ColumnEntry(v, ref));
+      } else {
+        // this first one didn't match at all, so we have to put in a null entry
+        entries.add(new ColumnEntry(null, ref));
+        continue;
+      }
+      // here's where is gets a little tricky - we either need to decide if we should continue
+      // adding entries (matches all qualifiers) or if we are done (matches a single qualifier)
+      if (!ref.allColumns()) {
+        continue;
+      }
+      // matches all columns, so we need to iterate until we hit the next column with the same
+      // family as the current key
+      byte[] lastQual = next.getQualifier();
+      byte[] nextQual = null;
+      while ((next = kvs.next()) != null) {
+        // different family, done with this column
+        if (!ref.matchesFamily(next.getFamily())) {
+          break;
+        }
+        nextQual = next.getQualifier();
+        // we are still on the same qualifier - skip it, since we already added a column for it
+        if (Arrays.equals(lastQual, nextQual)) {
+          continue;
+        }
+        // this must match the qualifier since its an all-qualifiers specifier, so we add it
+        byte[] v = next.getValue();
+        totalValueLength += v.length;
+        entries.add(new ColumnEntry(v, ref));
+        // update the last qualifier to check against
+        lastQual = nextQual;
+      }
+    }
+    return new Pair<Integer, List<ColumnEntry>>(totalValueLength, entries);
+  }
+
+  static class ColumnEntry {
+    byte[] value = EMPTY_BYTES;
+    CoveredColumn ref;
+
+    public ColumnEntry(byte[] value, CoveredColumn ref) {
+      this.value = value == null ? EMPTY_BYTES : value;
+      this.ref = ref;
+    }
+  }
+
+  /**
+   * Compose the final index row key.
+   * <p>
+   * This is faster than adding each value independently as we can just build a single a array and
+   * copy everything over once.
+   * @param pk primary key of the original row
+   * @param length total number of bytes of all the values that should be added
+   * @param values to use when building the key
+   * @return
+   */
+  static byte[] composeRowKey(byte[] pk, int length, List<ColumnEntry> values) {
+    // now build up expected row key, each of the values, in order, followed by the PK and then some
+    // info about lengths so we can deserialize each value
+    byte[] output = new byte[length + pk.length];
+    int pos = 0;
+    int[] lengths = new int[values.size()];
+    int i = 0;
+    for (ColumnEntry entry : values) {
+      byte[] v = entry.value;
+      // skip doing the copy attempt, if we don't need to
+      if (v.length != 0) {
+        System.arraycopy(v, 0, output, pos, v.length);
+        pos += v.length;
+      }
+      lengths[i++] = v.length;
+    }
+
+    // add the primary key to the end of the row key
+    System.arraycopy(pk, 0, output, pos, pk.length);
+
+    // add the lengths as suffixes so we can deserialize the elements again
+    for (int l : lengths) {
+      output = ArrayUtils.addAll(output, Bytes.toBytes(l));
+    }
+
+    // and the last integer is the number of values
+    return ArrayUtils.addAll(output, Bytes.toBytes(values.size()));
+  }
+
+  /**
+   * Essentially a short-cut from building a {@link Put}.
+   * @param pk row key
+   * @param timestamp timestamp of all the keyvalues
+   * @param values expected value--column pair
+   * @return a keyvalues that the index contains for a given row at a timestamp with the given value
+   *         -- column pairs.
+   */
+  public static List<KeyValue> getIndexKeyValueForTesting(byte[] pk, long timestamp,
+      List<Pair<byte[], CoveredColumn>> values) {
+  
+    int length = 0;
+    List<ColumnEntry> expected = new ArrayList<ColumnEntry>(values.size());
+    for (Pair<byte[], CoveredColumn> value : values) {
+      ColumnEntry entry = new ColumnEntry(value.getFirst(), value.getSecond());
+      length += value.getFirst().length;
+      expected.add(entry);
+    }
+  
+    byte[] rowKey = CoveredColumnIndexCodec.composeRowKey(pk, length, expected);
+    Put p = new Put(rowKey, timestamp);
+    CoveredColumnIndexCodec.addColumnsToPut(p, expected);
+    List<KeyValue> kvs = new ArrayList<KeyValue>();
+    for (Entry<byte[], List<KeyValue>> entry : p.getFamilyMap().entrySet()) {
+      kvs.addAll(entry.getValue());
+    }
+  
+    return kvs;
+  }
+
+  public static List<byte[]> getValues(byte[] bytes) {
+    // get the total number of keys in the bytes
+    int keyCount = CoveredColumnIndexCodec.getPreviousInteger(bytes, bytes.length);
+    List<byte[]> keys = new ArrayList<byte[]>(keyCount);
+    int[] lengths = new int[keyCount];
+    int lengthPos = keyCount - 1;
+    int pos = bytes.length - Bytes.SIZEOF_INT;
+    // figure out the length of each key
+    for (int i = 0; i < keyCount; i++) {
+      lengths[lengthPos--] = CoveredColumnIndexCodec.getPreviousInteger(bytes, pos);
+      pos -= Bytes.SIZEOF_INT;
+    }
+
+    int current = 0;
+    for (int length : lengths) {
+      byte[] key = Arrays.copyOfRange(bytes, current, current + length);
+      keys.add(key);
+      current += length;
+    }
+
+    return keys;
+  }
+
+  /**
+   * Read an integer from the preceding {@value Bytes#SIZEOF_INT} bytes
+   * @param bytes array to read from
+   * @param start start point, backwards from which to read. For example, if specifying "25", we
+   *          would try to read an integer from 21 -> 25
+   * @return an integer from the proceeding {@value Bytes#SIZEOF_INT} bytes, if it exists.
+   */
+  private static int getPreviousInteger(byte[] bytes, int start) {
+    return Bytes.toInt(bytes, start - Bytes.SIZEOF_INT);
+  }
+
+  /**
+   * Check to see if an row key just contains a list of null values.
+   * @param bytes row key to examine
+   * @return <tt>true</tt> if all the values are zero-length, <tt>false</tt> otherwise
+   */
+  public static boolean checkRowKeyForAllNulls(byte[] bytes) {
+    int keyCount = CoveredColumnIndexCodec.getPreviousInteger(bytes, bytes.length);
+    int pos = bytes.length - Bytes.SIZEOF_INT;
+    for (int i = 0; i < keyCount; i++) {
+      int next = CoveredColumnIndexCodec.getPreviousInteger(bytes, pos);
+      if (next > 0) {
+        return false;
+      }
+      pos -= Bytes.SIZEOF_INT;
+    }
+
+    return true;
+  }
+
+  @Override
+  public boolean isEnabled(Mutation m) {
+    // this could be a bit smarter, looking at the groups for the mutation, but we leave it at this
+    // simple check for the moment.
+    return groups.size() > 0;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/example/CoveredColumnIndexSpecifierBuilder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/example/CoveredColumnIndexSpecifierBuilder.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/example/CoveredColumnIndexSpecifierBuilder.java
new file mode 100644
index 0000000..0c9f155
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/example/CoveredColumnIndexSpecifierBuilder.java
@@ -0,0 +1,166 @@
+package org.apache.hadoop.hbase.index.covered.example;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HTableDescriptor;
+
+import org.apache.hadoop.hbase.index.Indexer;
+import org.apache.hadoop.hbase.index.covered.CoveredColumnsIndexBuilder;
+import org.apache.hadoop.hbase.index.covered.IndexCodec;
+
+/**
+ * Helper to build the configuration for the {@link CoveredColumnIndexer}.
+ * <p>
+ * This class is NOT thread-safe; all concurrent access must be managed externally.
+ */
+public class CoveredColumnIndexSpecifierBuilder {
+
+  private static final String INDEX_TO_TABLE_CONF_PREFX = "hbase.index.covered.";
+  // number of index 'groups'. Each group represents a set of 'joined' columns. The data stored with
+  // each joined column are either just the columns in the group or all the most recent data in the
+  // row (a fully covered index).
+  private static final String COUNT = ".count";
+  private static final String INDEX_GROUPS_COUNT_KEY = INDEX_TO_TABLE_CONF_PREFX + ".groups" + COUNT;
+  private static final String INDEX_GROUP_PREFIX = INDEX_TO_TABLE_CONF_PREFX + "group.";
+  private static final String INDEX_GROUP_COVERAGE_SUFFIX = ".columns";
+  private static final String TABLE_SUFFIX = ".table";
+
+  // right now, we don't support this should be easy enough to add later
+  // private static final String INDEX_GROUP_FULLY_COVERED = ".covered";
+
+  List<ColumnGroup> groups = new ArrayList<ColumnGroup>();
+  private Map<String, String> specs = new HashMap<String, String>();
+
+  /**
+   * Add a group of columns to index
+   * @param columns Pairs of cf:cq (full specification of a column) to index
+   * @return the index of the group. This can be used to remove or modify the group via
+   *         {@link #remove(int)} or {@link #get(int)}, any time before building
+   */
+  public int addIndexGroup(ColumnGroup columns) {
+    if (columns == null || columns.size() == 0) {
+      throw new IllegalArgumentException("Must specify some columns to index!");
+    }
+    int size = this.groups.size();
+    this.groups.add(columns);
+    return size;
+  }
+
+  public void remove(int i) {
+    this.groups.remove(i);
+  }
+
+  public ColumnGroup get(int i) {
+    return this.groups.get(i);
+  }
+
+  /**
+   * Clear the stored {@link ColumnGroup}s for resuse.
+   */
+  public void reset() {
+    this.groups.clear();
+  }
+
+  Map<String, String> convertToMap() {
+    int total = this.groups.size();
+    // hbase.index.covered.groups = i
+    specs.put(INDEX_GROUPS_COUNT_KEY, Integer.toString(total));
+
+    int i = 0;
+    for (ColumnGroup group : groups) {
+      addIndexGroupToSpecs(specs, group, i++);
+    }
+
+    return specs;
+  }
+
+  /**
+   * @param specs
+   * @param columns
+   * @param index
+   */
+  private void addIndexGroupToSpecs(Map<String, String> specs, ColumnGroup columns, int index) {
+    // hbase.index.covered.group.<i>
+    String prefix = INDEX_GROUP_PREFIX + Integer.toString(index);
+
+    // set the table to which the group writes
+    // hbase.index.covered.group.<i>.table
+    specs.put(prefix + TABLE_SUFFIX, columns.getTable());
+    
+    // a different key for each column in the group
+    // hbase.index.covered.group.<i>.columns
+    String columnPrefix = prefix + INDEX_GROUP_COVERAGE_SUFFIX;
+    // hbase.index.covered.group.<i>.columns.count = <j>
+    String columnsSizeKey = columnPrefix + COUNT;
+    specs.put(columnsSizeKey, Integer.toString(columns.size()));
+    
+    // add each column in the group
+    int i=0; 
+    for (CoveredColumn column : columns) {
+      // hbase.index.covered.group.<i>.columns.<j>
+      String nextKey = columnPrefix + "." + Integer.toString(i);
+      String nextValue = column.serialize();
+      specs.put(nextKey, nextValue);
+      i++;
+    }
+  }
+
+  public void build(HTableDescriptor desc) throws IOException {
+    build(desc, CoveredColumnIndexCodec.class);
+  }
+
+  void build(HTableDescriptor desc, Class<? extends IndexCodec> clazz) throws IOException {
+    // add the codec for the index to the map of options
+    Map<String, String> opts = this.convertToMap();
+    opts.put(CoveredColumnsIndexBuilder.CODEC_CLASS_NAME_KEY, clazz.getName());
+    Indexer.enableIndexing(desc, CoveredColumnIndexer.class, opts);
+  }
+
+  static List<ColumnGroup> getColumns(Configuration conf) {
+    int count= conf.getInt(INDEX_GROUPS_COUNT_KEY, 0);
+    if (count ==0) {
+      return Collections.emptyList();
+    }
+
+    // parse out all the column groups we should index
+    List<ColumnGroup> columns = new ArrayList<ColumnGroup>(count);
+    for (int i = 0; i < count; i++) {
+      // parse out each group
+      String prefix = INDEX_GROUP_PREFIX + i;
+
+      // hbase.index.covered.group.<i>.table
+      String table = conf.get(prefix + TABLE_SUFFIX);
+      ColumnGroup group = new ColumnGroup(table);
+
+      // parse out each column in the group
+      // hbase.index.covered.group.<i>.columns
+      String columnPrefix = prefix + INDEX_GROUP_COVERAGE_SUFFIX;
+      // hbase.index.covered.group.<i>.columns.count = j
+      String columnsSizeKey = columnPrefix + COUNT;
+      int columnCount = conf.getInt(columnsSizeKey, 0);
+      for(int j=0; j< columnCount; j++){
+        String columnKey = columnPrefix + "." + j;
+        CoveredColumn column = CoveredColumn.parse(conf.get(columnKey));
+        group.add(column);
+      }
+
+      // add the group
+      columns.add(group);
+    }
+    return columns;
+  }
+
+  /**
+   * @param key
+   * @param value
+   */
+  public void addArbitraryConfigForTesting(String key, String value) {
+    this.specs.put(key, value);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/example/CoveredColumnIndexer.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/example/CoveredColumnIndexer.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/example/CoveredColumnIndexer.java
new file mode 100644
index 0000000..c7019c4
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/example/CoveredColumnIndexer.java
@@ -0,0 +1,146 @@
+package org.apache.hadoop.hbase.index.covered.example;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValue.Type;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+
+import org.apache.hadoop.hbase.index.covered.Batch;
+import org.apache.hadoop.hbase.index.covered.CoveredColumnsIndexBuilder;
+import org.apache.hadoop.hbase.index.covered.LocalTableState;
+import org.apache.hadoop.hbase.index.covered.update.IndexUpdateManager;
+
+/**
+ * Index maintainer that maintains multiple indexes based on '{@link ColumnGroup}s'. Each group is a
+ * fully covered within itself and stores the fully 'pre-joined' version of that values for that
+ * group of columns.
+ * <p>
+ * <h2>Index Layout</h2> The row key for a given index entry is the current state of the all the
+ * values of the columns in a column group, followed by the primary key (row key) of the original
+ * row, and then the length of each value and then finally the total number of values. This is then
+ * enough information to completely rebuild the latest value of row for each column in the group.
+ * <p>
+ * The family is always {@link CoveredColumnIndexCodec#INDEX_ROW_COLUMN_FAMILY}
+ * <p>
+ * The qualifier is prepended with the integer index (serialized with {@link Bytes#toBytes(int)}) of
+ * the column in the group. This index corresponds the index of the value for the group in the row
+ * key.
+ * 
+ * <pre>
+ *         ROW                            ||   FAMILY     ||    QUALIFIER     ||   VALUE
+ * (v1)(v2)...(vN)(pk)(L1)(L2)...(Ln)(#V) || INDEX_FAMILY ||     1Cf1:Cq1     ||  null
+ * (v1)(v2)...(vN)(pk)(L1)(L2)...(Ln)(#V) || INDEX_FAMILY ||     2Cf2:Cq2     ||  null
+ * ...
+ * (v1)(v2)...(vN)(pk)(L1)(L2)...(Ln)(#V) || INDEX_FAMILY ||     NCfN:CqN     ||  null
+ * </pre>
+ * 
+ * <h2>Index Maintenance</h2>
+ * <p>
+ * When making an insertion into the table, we also attempt to cleanup the index. This means that we
+ * need to remove the previous entry from the index. Generally, this is completed by inserting a
+ * delete at the previous value of the previous row.
+ * <p>
+ * The main caveat here is when dealing with custom timestamps. If there is no special timestamp
+ * specified, we can just insert the proper {@link Delete} at the current timestamp and move on.
+ * However, when the client specifies a timestamp, we could see updates out of order. In that case,
+ * we can do an insert using the specified timestamp, but a delete is different...
+ * <p>
+ * Taking the simple case, assume we do a single column in a group. Then if we get an out of order
+ * update, we need to check the current state of that column in the current row. If the current row
+ * is older, we can issue a delete as normal. If the current row is newer, however, we then have to
+ * issue a delete for the index update at the time of the current row. This ensures that the index
+ * update made for the 'future' time still covers the existing row.
+ * <p>
+ * <b>ASSUMPTION:</b> all key-values in a single {@link Delete}/{@link Put} have the same timestamp.
+ * This dramatically simplifies the logic needed to manage updating the index for out-of-order
+ * {@link Put}s as we don't need to manage multiple levels of timestamps across multiple columns.
+ * <p>
+ * We can extend this to multiple columns by picking the latest update of any column in group as the
+ * delete point.
+ * <p>
+ * <b>NOTE:</b> this means that we need to do a lookup (point {@link Get}) of the entire row
+ * <i>every time there is a write to the table</i>.
+ */
+public class CoveredColumnIndexer extends CoveredColumnsIndexBuilder {
+
+  /**
+   * Create the specified index table with the necessary columns
+   * @param admin {@link HBaseAdmin} to use when creating the table
+   * @param indexTable name of the index table.
+   * @throws IOException
+   */
+  public static void createIndexTable(HBaseAdmin admin, String indexTable) throws IOException {
+    createIndexTable(admin, new HTableDescriptor(indexTable));
+  }
+
+  /**
+   * @param admin to create the table
+   * @param index descriptor to update before creating table
+   */
+  public static void createIndexTable(HBaseAdmin admin, HTableDescriptor index) throws IOException {
+    HColumnDescriptor col =
+        new HColumnDescriptor(CoveredColumnIndexCodec.INDEX_ROW_COLUMN_FAMILY);
+    // ensure that we can 'see past' delete markers when doing scans
+    col.setKeepDeletedCells(true);
+    index.addFamily(col);
+    admin.createTable(index);
+  }
+
+  @Override
+  public Collection<Pair<Mutation, byte[]>> getIndexUpdateForFilteredRows(
+      Collection<KeyValue> filtered) throws IOException {
+
+    // stores all the return values
+    IndexUpdateManager updateMap = new IndexUpdateManager();
+    // batch the updates by row to make life easier and ordered
+    Collection<Batch> batches = batchByRow(filtered);
+
+    for (Batch batch : batches) {
+      Put p = new Put(batch.getKvs().iterator().next().getRow());
+      for (KeyValue kv : batch.getKvs()) {
+        // we only need to cleanup Put entries
+        byte type = kv.getType();
+        Type t = KeyValue.Type.codeToType(type);
+        if (!t.equals(Type.Put)) {
+          continue;
+        }
+
+        // add the kv independently
+        p.add(kv);
+      }
+
+      // do the usual thing as for deletes
+      Collection<Batch> timeBatch = createTimestampBatchesFromMutation(p);
+      LocalTableState state = new LocalTableState(env, localTable, p);
+      for (Batch entry : timeBatch) {
+        //just set the timestamp on the table - it already has all the future state
+        state.setCurrentTimestamp(entry.getTimestamp());
+        this.addDeleteUpdatesToMap(updateMap, state, entry.getTimestamp());
+      }
+    }
+    return updateMap.toMap();
+  }
+
+
+  /**
+   * @param filtered
+   * @return
+   */
+  private Collection<Batch>  batchByRow(Collection<KeyValue> filtered) {
+    Map<Long, Batch> batches = new HashMap<Long, Batch>();
+    createTimestampBatchesFromKeyValues(filtered, batches);
+    return batches.values();
+  }
+}
\ No newline at end of file


[31/51] [partial] Initial commit of master branch from github

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/OrderByExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/OrderByExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/OrderByExpression.java
new file mode 100644
index 0000000..67d7e99
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/OrderByExpression.java
@@ -0,0 +1,87 @@
+package org.apache.phoenix.expression;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableUtils;
+
+/**
+ * A container for a column that appears in ORDER BY clause.
+ */
+public class OrderByExpression implements Writable {
+    private Expression expression;
+    private boolean isNullsLast;
+    private boolean isAscending;
+    
+    public OrderByExpression() {
+    }
+    
+    public OrderByExpression(Expression expression, boolean isNullsLast, boolean isAcending) {
+        checkNotNull(expression);
+        this.expression = expression;
+        this.isNullsLast = isNullsLast;
+        this.isAscending = isAcending;
+    }
+
+    public Expression getExpression() {
+        return expression;
+    }
+    
+    public boolean isNullsLast() {
+        return isNullsLast;
+    }
+    
+    public boolean isAscending() {
+        return isAscending;
+    }
+    
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o != null && this.getClass() == o.getClass()) {
+            OrderByExpression that = (OrderByExpression)o;
+            return isNullsLast == that.isNullsLast
+                && isAscending == that.isAscending
+                && expression.equals(that.expression);
+        }
+        return false;
+    }
+    
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + (isNullsLast ? 0 : 1);
+        result = prime * result + (isAscending ? 0 : 1);
+        result = prime * result + expression.hashCode();
+        return result;
+    }
+    
+    @Override
+    public String toString() {
+        return this.getExpression() + (isAscending ? "" : " DESC") + (isNullsLast ? " NULLS LAST" : "");
+    }
+    
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        this.isNullsLast = input.readBoolean();
+        this.isAscending = input.readBoolean();
+        expression = ExpressionType.values()[WritableUtils.readVInt(input)].newInstance();
+        expression.readFields(input);
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+        output.writeBoolean(isNullsLast);
+        output.writeBoolean(isAscending);
+        WritableUtils.writeVInt(output, ExpressionType.valueOf(expression).ordinal());
+        expression.write(output);
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..2262882
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/ProjectedColumnExpression.java
@@ -0,0 +1,144 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.expression;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import org.apache.phoenix.expression.visitor.ExpressionVisitor;
+import org.apache.phoenix.join.ScanProjector;
+import org.apache.phoenix.schema.KeyValueSchema;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.ValueBitSet;
+import org.apache.phoenix.schema.KeyValueSchema.KeyValueSchemaBuilder;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.SchemaUtil;
+
+public class ProjectedColumnExpression extends ColumnExpression {
+	private KeyValueSchema schema;
+	ValueBitSet bitSet;
+	private int position;
+	private byte[] name; // for display purpose only
+	
+	public ProjectedColumnExpression() {
+	}
+
+	public ProjectedColumnExpression(PColumn column, PTable table) {
+		super(column);
+		this.schema = buildSchema(table);
+		this.bitSet = ValueBitSet.newInstance(schema);
+		this.position = column.getPosition() - table.getPKColumns().size();
+		this.name = column.getName().getBytes();
+	}
+    
+    private static KeyValueSchema buildSchema(PTable table) {
+    	KeyValueSchemaBuilder builder = new KeyValueSchemaBuilder(0);
+        for (PColumn column : table.getColumns()) {
+        	if (!SchemaUtil.isPKColumn(column)) {
+        		builder.addField(column);
+        	}
+        }
+        return builder.build();
+    }
+    
+    public KeyValueSchema getSchema() {
+    	return schema;
+    }
+    
+    public int getPosition() {
+    	return position;
+    }
+    
+    public byte[] getName() {
+    	return name;
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + schema.hashCode();
+        result = prime * result + position;
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) return true;
+        if (obj == null) return false;
+        if (getClass() != obj.getClass()) return false;
+        ProjectedColumnExpression other = (ProjectedColumnExpression)obj;
+        if (!schema.equals(other.schema)) return false;
+        if (position != other.position) return false;
+        return true;
+    }
+
+    @Override
+    public String toString() {
+        return Bytes.toString(name);
+    }
+	
+	@Override
+	public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        try {
+            ScanProjector.decodeProjectedValue(tuple, ptr);
+            int maxOffset = ptr.getOffset() + ptr.getLength();
+            bitSet.clear();
+            bitSet.or(ptr);
+            schema.iterator(ptr, position, bitSet);
+            Boolean hasValue = schema.next(ptr, position, maxOffset, bitSet);
+            if (hasValue == null || !hasValue.booleanValue())
+                return false;
+        } catch (IOException e) {
+            return false;
+        }
+		
+		return true;
+	}
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        super.readFields(input);
+        schema = new KeyValueSchema();
+        schema.readFields(input);
+        bitSet = ValueBitSet.newInstance(schema);
+        position = input.readInt();
+        name = Bytes.readByteArray(input);
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+        super.write(output);
+        schema.write(output);
+        output.writeInt(position);
+        Bytes.writeByteArray(output, name);
+    }
+
+    @Override
+    public final <T> T accept(ExpressionVisitor<T> visitor) {
+        return visitor.visit(this);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/RowKeyColumnExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/RowKeyColumnExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/RowKeyColumnExpression.java
new file mode 100644
index 0000000..383ef2e
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/RowKeyColumnExpression.java
@@ -0,0 +1,150 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.expression;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.expression.visitor.ExpressionVisitor;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.PDatum;
+import org.apache.phoenix.schema.RowKeyValueAccessor;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.ByteUtil;
+
+
+/**
+ * 
+ * Class to access a value stored in the row key
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class RowKeyColumnExpression  extends ColumnExpression {
+    private PDataType fromType;
+    private RowKeyValueAccessor accessor;
+    protected final String name;
+    
+    public RowKeyColumnExpression() {
+        name = null; // Only on client
+    }
+    
+    private RowKeyColumnExpression(PDatum datum, RowKeyValueAccessor accessor, PDataType fromType, String name) {
+        super(datum);
+        this.accessor = accessor;
+        this.fromType = fromType;
+        this.name = name;
+    }
+    
+    public RowKeyColumnExpression(PDatum datum, RowKeyValueAccessor accessor) {
+        this(datum, accessor, datum.getDataType(), datum.toString());
+    }
+    
+    public RowKeyColumnExpression(PDatum datum, RowKeyValueAccessor accessor, String name) {
+        this(datum, accessor, datum.getDataType(), name);
+    }
+    
+    public RowKeyColumnExpression(PDatum datum, RowKeyValueAccessor accessor, PDataType fromType) {
+        this(datum, accessor, fromType, datum.toString());
+    }
+    
+    public int getPosition() {
+        return accessor.getIndex();
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = super.hashCode();
+        result = prime * result + ((accessor == null) ? 0 : accessor.hashCode());
+        return result;
+    }
+
+    @Override
+    public String toString() {
+        return name == null ? "PK[" + accessor.getIndex() + "]" : name;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) return true;
+        if (!super.equals(obj)) return false;
+        if (getClass() != obj.getClass()) return false;
+        RowKeyColumnExpression other = (RowKeyColumnExpression)obj;
+        return accessor.equals(other.accessor);
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        tuple.getKey(ptr);
+        int offset = accessor.getOffset(ptr.get(), ptr.getOffset());
+        // Null is represented in the last expression of a multi-part key 
+        // by the bytes not being present.
+        int maxOffset = ptr.getOffset() + ptr.getLength();
+        if (offset < maxOffset) {
+            byte[] buffer = ptr.get();
+            int fixedByteSize = -1;
+            // FIXME: fixedByteSize <= maxByteSize ? fixedByteSize : 0 required because HBase passes bogus keys to filter to position scan (HBASE-6562)
+            if (fromType.isFixedWidth()) {
+                fixedByteSize = getByteSize();
+                fixedByteSize = fixedByteSize <= maxOffset ? fixedByteSize : 0;
+            }
+            int length = fixedByteSize >= 0 ? fixedByteSize  : accessor.getLength(buffer, offset, maxOffset);
+            // In the middle of the key, an empty variable length byte array represents null
+            if (length > 0) {
+                /*
+                if (type == fromType) {
+                    ptr.set(buffer,offset,length);
+                } else {
+                    ptr.set(type.toBytes(type.toObject(buffer, offset, length, fromType)));
+                }
+                */
+                ptr.set(buffer,offset,length);
+                type.coerceBytes(ptr, fromType, getColumnModifier(), getColumnModifier());
+            } else {
+                ptr.set(ByteUtil.EMPTY_BYTE_ARRAY);
+            }
+            return true;
+        }
+        return false;
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        super.readFields(input);
+        accessor = new RowKeyValueAccessor();
+        accessor.readFields(input);
+        fromType = type; // fromType only needed on client side
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+        super.write(output);
+        accessor.write(output);
+    }
+    
+    @Override
+    public final <T> T accept(ExpressionVisitor<T> visitor) {
+        return visitor.visit(this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/RowKeyExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/RowKeyExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/RowKeyExpression.java
new file mode 100644
index 0000000..c311932
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/RowKeyExpression.java
@@ -0,0 +1,25 @@
+package org.apache.phoenix.expression;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+public class RowKeyExpression extends BaseTerminalExpression {
+    public static final RowKeyExpression INSTANCE = new RowKeyExpression();
+    
+    private RowKeyExpression() {
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        tuple.getKey(ptr);
+        return true;
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return PDataType.VARBINARY;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/RowValueConstructorExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/RowValueConstructorExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/RowValueConstructorExpression.java
new file mode 100644
index 0000000..3e410cb
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/RowValueConstructorExpression.java
@@ -0,0 +1,358 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.
+ */
+
+/**
+ * Implementation for row value constructor (a,b,c) expression.
+ * 
+ * @author samarth.jain
+ * @since 0.1
+ */
+package org.apache.phoenix.expression;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import com.google.common.collect.Lists;
+import org.apache.phoenix.expression.function.CeilDecimalExpression;
+import org.apache.phoenix.expression.function.CeilTimestampExpression;
+import org.apache.phoenix.expression.function.FloorDateExpression;
+import org.apache.phoenix.expression.function.FloorDecimalExpression;
+import org.apache.phoenix.expression.function.TimeUnit;
+import org.apache.phoenix.expression.visitor.ExpressionVisitor;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.TypeMismatchException;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.TrustedByteArrayOutputStream;
+
+public class RowValueConstructorExpression extends BaseCompoundExpression {
+    
+    private ImmutableBytesWritable ptrs[];
+    private ImmutableBytesWritable literalExprPtr;
+    private int counter;
+    private int estimatedByteSize;
+    
+    public static interface ExpressionComparabilityWrapper {
+        public Expression wrap(Expression lhs, Expression rhs) throws SQLException;
+    }
+    /*
+     * Used to coerce the RHS to the expected type based on the LHS. In some circumstances,
+     * we may need to round the value up or down. For example:
+     * WHERE (a,b) < (2.4, 'foo')
+     * We take the ceiling of 2.4 to make it 3 if a is an INTEGER to prevent needing to coerce
+     * every time during evaluation.
+     */
+    private static ExpressionComparabilityWrapper[] WRAPPERS = new ExpressionComparabilityWrapper[CompareOp.values().length];
+    static {
+        WRAPPERS[CompareOp.LESS.ordinal()] = new ExpressionComparabilityWrapper() {
+
+            @Override
+            public Expression wrap(Expression lhs, Expression rhs) throws SQLException {
+                Expression e = rhs;
+                PDataType rhsType = rhs.getDataType();
+                PDataType lhsType = lhs.getDataType();
+                if (rhsType == PDataType.DECIMAL && lhsType != PDataType.DECIMAL) {
+                    e = FloorDecimalExpression.create(rhs);
+                } else if ((rhsType == PDataType.TIMESTAMP || rhsType == PDataType.UNSIGNED_TIMESTAMP)  && (lhsType != PDataType.TIMESTAMP && lhsType != PDataType.UNSIGNED_TIMESTAMP)) {
+                    e = FloorDateExpression.create(rhs, TimeUnit.MILLISECOND);
+                }
+                e = new CoerceExpression(e, lhsType, lhs.getColumnModifier(), lhs.getByteSize());
+                return e;
+            }
+            
+        };
+        WRAPPERS[CompareOp.LESS_OR_EQUAL.ordinal()] = WRAPPERS[CompareOp.LESS.ordinal()];
+        
+        WRAPPERS[CompareOp.GREATER.ordinal()] = new ExpressionComparabilityWrapper() {
+
+            @Override
+            public Expression wrap(Expression lhs, Expression rhs) throws SQLException {
+                Expression e = rhs;
+                PDataType rhsType = rhs.getDataType();
+                PDataType lhsType = lhs.getDataType();
+                if (rhsType == PDataType.DECIMAL && lhsType != PDataType.DECIMAL) {
+                    e = CeilDecimalExpression.create(rhs);
+                } else if ((rhsType == PDataType.TIMESTAMP || rhsType == PDataType.UNSIGNED_TIMESTAMP)  && (lhsType != PDataType.TIMESTAMP && lhsType != PDataType.UNSIGNED_TIMESTAMP)) {
+                    e = CeilTimestampExpression.create(rhs);
+                }
+                e = new CoerceExpression(e, lhsType, lhs.getColumnModifier(), lhs.getByteSize());
+                return e;
+            }
+            
+        };
+        WRAPPERS[CompareOp.GREATER_OR_EQUAL.ordinal()] = WRAPPERS[CompareOp.GREATER.ordinal()];
+    }
+    
+    private static ExpressionComparabilityWrapper getWrapper(CompareOp op) {
+        ExpressionComparabilityWrapper wrapper = WRAPPERS[op.ordinal()];
+        if (wrapper == null) {
+            throw new IllegalStateException("Unexpected compare op of " + op + " for row value constructor");
+        }
+        return wrapper;
+    }
+    
+    /**
+     * Recursively coerce the RHS to match the LHS type, throwing if the types are incompatible. The
+     * recursion occurs when the RHS or LHS is a row value constructor.
+     * TODO: this no longer needs to be recursive, as we flatten out rvc when we normalize the statement.
+     * @param lhs left hand side expression
+     * @param rhs right hand side expression
+     * @param op operator being used to compare the expressions, which can affect rounding we may need to do.
+     * @return the newly coerced expression
+     * @throws SQLException
+     */
+    public static Expression coerce(Expression lhs, Expression rhs, CompareOp op) throws SQLException {
+        return coerce(lhs, rhs, getWrapper(op));
+    }
+        
+    public static Expression coerce(Expression lhs, Expression rhs, ExpressionComparabilityWrapper wrapper) throws SQLException {
+        
+        if (lhs instanceof RowValueConstructorExpression && rhs instanceof RowValueConstructorExpression) {
+            int i = 0;
+            List<Expression> coercedNodes = Lists.newArrayListWithExpectedSize(Math.max(lhs.getChildren().size(), rhs.getChildren().size()));
+            for (; i < Math.min(lhs.getChildren().size(),rhs.getChildren().size()); i++) {
+                coercedNodes.add(coerce(lhs.getChildren().get(i), rhs.getChildren().get(i), wrapper));
+            }
+            for (; i < lhs.getChildren().size(); i++) {
+                coercedNodes.add(coerce(lhs.getChildren().get(i), null, wrapper));
+            }
+            for (; i < rhs.getChildren().size(); i++) {
+                coercedNodes.add(coerce(null, rhs.getChildren().get(i), wrapper));
+            }
+            trimTrailingNulls(coercedNodes);
+            return coercedNodes.equals(rhs.getChildren()) ? rhs : new RowValueConstructorExpression(coercedNodes, rhs.isStateless());
+        } else if (lhs instanceof RowValueConstructorExpression) {
+            List<Expression> coercedNodes = Lists.newArrayListWithExpectedSize(Math.max(rhs.getChildren().size(), lhs.getChildren().size()));
+            coercedNodes.add(coerce(lhs.getChildren().get(0), rhs, wrapper));
+            for (int i = 1; i < lhs.getChildren().size(); i++) {
+                coercedNodes.add(coerce(lhs.getChildren().get(i), null, wrapper));
+            }
+            trimTrailingNulls(coercedNodes);
+            return coercedNodes.equals(rhs.getChildren()) ? rhs : new RowValueConstructorExpression(coercedNodes, rhs.isStateless());
+        } else if (rhs instanceof RowValueConstructorExpression) {
+            List<Expression> coercedNodes = Lists.newArrayListWithExpectedSize(Math.max(rhs.getChildren().size(), lhs.getChildren().size()));
+            coercedNodes.add(coerce(lhs, rhs.getChildren().get(0), wrapper));
+            for (int i = 1; i < rhs.getChildren().size(); i++) {
+                coercedNodes.add(coerce(null, rhs.getChildren().get(i), wrapper));
+            }
+            trimTrailingNulls(coercedNodes);
+            return coercedNodes.equals(rhs.getChildren()) ? rhs : new RowValueConstructorExpression(coercedNodes, rhs.isStateless());
+        } else if (lhs == null) { 
+            return rhs;
+        } else if (rhs == null) {
+            return LiteralExpression.newConstant(null, lhs.getDataType(), lhs.isDeterministic());
+        } else {
+            if (rhs.getDataType() != null && lhs.getDataType() != null && !rhs.getDataType().isCastableTo(lhs.getDataType())) {
+                throw TypeMismatchException.newException(lhs.getDataType(), rhs.getDataType());
+            }
+            return wrapper.wrap(lhs, rhs);
+        }
+    }
+    
+    private static void trimTrailingNulls(List<Expression> expressions) {
+        for (int i = expressions.size() - 1; i >= 0; i--) {
+            Expression e = expressions.get(i);
+            if (e instanceof LiteralExpression && ((LiteralExpression)e).getValue() == null) {
+                expressions.remove(i);
+            } else {
+                break;
+            }
+        }
+    }
+
+
+    public RowValueConstructorExpression() {
+    }
+    
+    public RowValueConstructorExpression(List<Expression> children, boolean isConstant) {
+        super(children);
+        counter = 0;
+        estimatedByteSize = 0;
+        init(isConstant);
+    }
+
+    public int getEstimatedSize() {
+        return estimatedByteSize;
+    }
+    
+    @Override
+    public boolean isStateless() {
+        return literalExprPtr != null;
+    }
+    
+    @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;
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        super.readFields(input);
+        init(input.readBoolean());
+    }
+    
+    @Override
+    public void write(DataOutput output) throws IOException {
+        super.write(output);
+        output.writeBoolean(literalExprPtr != null);
+    }
+    
+    private void init(boolean isConstant) {
+        this.ptrs = new ImmutableBytesWritable[children.size()];
+        if(isConstant) {
+            ImmutableBytesWritable ptr = new ImmutableBytesWritable();
+            this.evaluate(null, ptr);
+            literalExprPtr = ptr;
+        }
+    }
+    
+    @Override
+    public PDataType getDataType() {
+        return PDataType.VARBINARY;
+    }
+    
+    @Override
+    public void reset() {
+        counter = 0;
+        estimatedByteSize = 0;
+        Arrays.fill(ptrs, null);
+    }
+    
+    private static int getExpressionByteCount(Expression e) {
+        PDataType childType = e.getDataType();
+        if (childType != null && !childType.isFixedWidth()) {
+            return 1;
+        } else {
+            // Write at least one null byte in the case of the child being null with a childType of null
+            Integer byteSize = e.getByteSize();
+            int bytesToWrite = byteSize == null ? 1 : Math.max(1, byteSize);
+            return bytesToWrite;
+        }
+    }
+    
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        if(literalExprPtr != null) {
+            // if determined during construction that the row value constructor is just comprised of literal expressions, 
+            // let's just return the ptr we have already computed and be done with evaluation.
+            ptr.set(literalExprPtr.get(), literalExprPtr.getOffset(), literalExprPtr.getLength());
+            return true;
+        }
+        try {
+            int j;
+            int expressionCount = counter;
+            for(j = counter; j < ptrs.length; j++) {
+                final Expression expression = children.get(j);
+                // TODO: handle overflow and underflow
+                if (expression.evaluate(tuple, ptr)) {
+                    if (ptr.getLength() == 0) {
+                        estimatedByteSize += getExpressionByteCount(expression);
+                    } else {
+                        expressionCount = j+1;
+                        ptrs[j] = new ImmutableBytesWritable();
+                        ptrs[j].set(ptr.get(), ptr.getOffset(), ptr.getLength());
+                        estimatedByteSize += ptr.getLength() + (expression.getDataType().isFixedWidth() ? 0 : 1); // 1 extra for the separator byte.
+                    }
+                    counter++;
+                } else if (tuple == null || tuple.isImmutable()) {
+                    estimatedByteSize += getExpressionByteCount(expression);
+                    counter++;
+                } else {
+                    return false;
+                }
+            }
+            
+            if (j == ptrs.length) {
+                if (expressionCount == 0) {
+                    ptr.set(ByteUtil.EMPTY_BYTE_ARRAY);
+                    return true;
+                }
+                if (expressionCount == 1) {
+                    ptr.set(ptrs[0].get(), ptrs[0].getOffset(), ptrs[0].getLength());
+                    return true;
+                }
+                TrustedByteArrayOutputStream output = new TrustedByteArrayOutputStream(estimatedByteSize);
+                try {
+                    boolean previousCarryOver = false;
+                    for (int i = 0; i< expressionCount; i++) {
+                        Expression child = getChildren().get(i);
+                        PDataType childType = child.getDataType();
+                        ImmutableBytesWritable tempPtr = ptrs[i];
+                        if (tempPtr == null) {
+                            // Since we have a null and have no representation for null,
+                            // we must decrement the value of the current. Otherwise,
+                            // we'd have an ambiguity if this value happened to be the
+                            // min possible value.
+                            previousCarryOver = childType == null || childType.isFixedWidth();
+                            int bytesToWrite = getExpressionByteCount(child);
+                            for (int m = 0; m < bytesToWrite; m++) {
+                                output.write(QueryConstants.SEPARATOR_BYTE);
+                            }
+                        } else {
+                            output.write(tempPtr.get(), tempPtr.getOffset(), tempPtr.getLength());
+                            if (!childType.isFixedWidth()) {
+                                output.write(QueryConstants.SEPARATOR_BYTE);
+                            }
+                            if (previousCarryOver) {
+                                previousCarryOver = !ByteUtil.previousKey(output.getBuffer(), output.size());
+                            }
+                        }
+                    }
+                    int outputSize = output.size();
+                    byte[] outputBytes = output.getBuffer();
+                    for (int k = expressionCount -1 ; 
+                            k >=0 &&  getChildren().get(k).getDataType() != null && !getChildren().get(k).getDataType().isFixedWidth() && outputBytes[outputSize-1] == QueryConstants.SEPARATOR_BYTE ; k--) {
+                        outputSize--;
+                    }
+                    ptr.set(outputBytes, 0, outputSize);
+                    return true;
+                } finally {
+                    output.close();
+                }
+            }  
+            return false;
+        } catch (IOException e) {
+            throw new RuntimeException(e); //Impossible.
+        }
+    }
+    
+    @Override
+    public final String toString() {
+        StringBuilder buf = new StringBuilder("(");
+        for (int i = 0; i < children.size() - 1; i++) {
+            buf.append(children.get(i) + ", ");
+        }
+        buf.append(children.get(children.size()-1) + ")");
+        return buf.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/StringConcatExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/StringConcatExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/StringConcatExpression.java
new file mode 100644
index 0000000..24e96f7
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/StringConcatExpression.java
@@ -0,0 +1,95 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.expression.visitor.ExpressionVisitor;
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.ByteUtil;
+
+
+/**
+ * 
+ * Implementation for || string concatenation expression.
+ * @author kmahadik
+ * @since 0.1
+ */
+
+public class StringConcatExpression extends BaseCompoundExpression {
+    public StringConcatExpression() {
+    }
+
+    public StringConcatExpression(List<Expression> children) {
+        super(children);
+    }
+
+    @Override
+    public String toString() {
+        StringBuilder buf = new StringBuilder("(");
+        for (int i = 0; i < children.size() - 1; i++) {
+            buf.append(children.get(i) + " || ");
+        }
+        buf.append(children.get(children.size()-1));
+        buf.append(')');
+        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;
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        byte[] result = ByteUtil.EMPTY_BYTE_ARRAY;
+        for (int i=0; i<children.size(); i++) {
+            if (children.get(i).getDataType() == null || !children.get(i).evaluate(tuple, ptr)) {
+                continue;
+            }
+            PDataType childType = children.get(i).getDataType();
+            ColumnModifier columnModifier = children.get(i).getColumnModifier();
+            // We could potentially not invert the bytes, but we might as well since we're allocating
+            // additional space here anyway.
+            if (childType.isCoercibleTo(PDataType.VARCHAR)) {
+                result = ByteUtil.concat(result, ByteUtil.concat(columnModifier, ptr));
+            } else {
+                result = ByteUtil.concat(result, PDataType.VARCHAR.toBytes(childType.toObject(ptr, columnModifier).toString()));
+            }
+        }
+        ptr.set(result);
+        return true;
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return PDataType.VARCHAR;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/SubtractExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/SubtractExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/SubtractExpression.java
new file mode 100644
index 0000000..a9c9c93
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/SubtractExpression.java
@@ -0,0 +1,60 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.math.BigDecimal;
+import java.util.List;
+
+import org.apache.phoenix.expression.visitor.ExpressionVisitor;
+import org.apache.phoenix.query.QueryConstants;
+
+
+/**
+ * 
+ * Subtract expression implementation
+ *
+ * @author kmahadik
+ * @since 0.1
+ */
+public abstract class SubtractExpression extends BaseAddSubtractExpression {
+    protected static final BigDecimal BD_MILLIS_IN_DAY = BigDecimal.valueOf(QueryConstants.MILLIS_IN_DAY);
+
+    public SubtractExpression() {
+    }
+
+    public SubtractExpression(List<Expression> children) {
+        super(children);
+    }
+
+    @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;
+    }
+    
+    @Override
+    public String getOperatorString() {
+        return " - ";
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/TimestampAddExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/TimestampAddExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/TimestampAddExpression.java
new file mode 100644
index 0000000..1206704
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/TimestampAddExpression.java
@@ -0,0 +1,89 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.math.BigDecimal;
+import java.math.RoundingMode;
+import java.sql.Timestamp;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.DateUtil;
+
+/**
+ * 
+ * Class to encapsulate addition arithmetic for {@link PDataType#TIMESTAMP}.
+ *
+ * @author samarth.jain
+ * @since 2.1.3
+ */
+
+public class TimestampAddExpression extends AddExpression {
+
+    public TimestampAddExpression() {
+    }
+
+    public TimestampAddExpression(List<Expression> children) {
+        super(children);
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        BigDecimal finalResult = BigDecimal.ZERO;
+        
+        for(int i=0; i<children.size(); i++) {
+            if (!children.get(i).evaluate(tuple, ptr)) {
+                return false;
+            }
+            if (ptr.getLength() == 0) {
+                return true;
+            }
+            BigDecimal value;
+            PDataType type = children.get(i).getDataType();
+            ColumnModifier columnModifier = children.get(i).getColumnModifier();
+            if(type == PDataType.TIMESTAMP || type == PDataType.UNSIGNED_TIMESTAMP) {
+                value = (BigDecimal)(PDataType.DECIMAL.toObject(ptr, type, columnModifier));
+            } else if (type.isCoercibleTo(PDataType.DECIMAL)) {
+                value = (((BigDecimal)PDataType.DECIMAL.toObject(ptr, columnModifier)).multiply(QueryConstants.BD_MILLIS_IN_DAY)).setScale(6, RoundingMode.HALF_UP);
+            } else if (type.isCoercibleTo(PDataType.DOUBLE)) {
+                value = ((BigDecimal.valueOf(type.getCodec().decodeDouble(ptr, columnModifier))).multiply(QueryConstants.BD_MILLIS_IN_DAY)).setScale(6, RoundingMode.HALF_UP);
+            } else {
+                value = BigDecimal.valueOf(type.getCodec().decodeLong(ptr, columnModifier));
+            } 
+            finalResult = finalResult.add(value);
+        }
+        Timestamp ts = DateUtil.getTimestamp(finalResult);
+        byte[] resultPtr = new byte[getDataType().getByteSize()];
+        PDataType.TIMESTAMP.toBytes(ts, resultPtr, 0);
+        ptr.set(resultPtr);
+        return true;
+    }
+
+    @Override
+    public final PDataType getDataType() {
+        return PDataType.TIMESTAMP;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/TimestampSubtractExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/TimestampSubtractExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/TimestampSubtractExpression.java
new file mode 100644
index 0000000..57450c0
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/TimestampSubtractExpression.java
@@ -0,0 +1,89 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.math.BigDecimal;
+import java.math.RoundingMode;
+import java.sql.Timestamp;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.DateUtil;
+/**
+ * 
+ * Class to encapsulate subtraction arithmetic for {@link PDataType#TIMESTAMP}.
+ *
+ * @author samarth.jain
+ * @since 2.1.3
+ */
+public class TimestampSubtractExpression extends SubtractExpression {
+
+    public TimestampSubtractExpression() {
+    }
+
+    public TimestampSubtractExpression(List<Expression> children) {
+        super(children);
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        BigDecimal finalResult = BigDecimal.ZERO;
+        
+        for(int i=0; i<children.size(); i++) {
+            if (!children.get(i).evaluate(tuple, ptr)) {
+                return false;
+            }
+            if (ptr.getLength() == 0) {
+                return true;
+            }
+            BigDecimal value;
+            PDataType type = children.get(i).getDataType();
+            ColumnModifier columnModifier = children.get(i).getColumnModifier();
+            if(type == PDataType.TIMESTAMP || type == PDataType.UNSIGNED_TIMESTAMP) {
+                value = (BigDecimal)(PDataType.DECIMAL.toObject(ptr, type, columnModifier));
+            } else if (type.isCoercibleTo(PDataType.DECIMAL)) {
+                value = (((BigDecimal)PDataType.DECIMAL.toObject(ptr, columnModifier)).multiply(BD_MILLIS_IN_DAY)).setScale(6, RoundingMode.HALF_UP);
+            } else if (type.isCoercibleTo(PDataType.DOUBLE)) {
+                value = ((BigDecimal.valueOf(type.getCodec().decodeDouble(ptr, columnModifier))).multiply(BD_MILLIS_IN_DAY)).setScale(6, RoundingMode.HALF_UP);
+            } else {
+                value = BigDecimal.valueOf(type.getCodec().decodeLong(ptr, columnModifier));
+            }
+            if (i == 0) {
+                finalResult = value;
+            } else {
+                finalResult = finalResult.subtract(value);
+            }
+        }
+        Timestamp ts = DateUtil.getTimestamp(finalResult);
+        byte[] resultPtr = new byte[getDataType().getByteSize()];
+        PDataType.TIMESTAMP.toBytes(ts, resultPtr, 0);
+        ptr.set(resultPtr);
+        return true;
+    }
+
+    @Override
+    public final PDataType getDataType() {
+        return PDataType.TIMESTAMP;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/Aggregator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/Aggregator.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/Aggregator.java
new file mode 100644
index 0000000..4067eb4
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/Aggregator.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.aggregator;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+
+/**
+ * 
+ * Interface to abstract the incremental calculation of an aggregated value.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public interface Aggregator extends Expression {
+    
+    /**
+     * Incrementally aggregate the value with the current row
+     * @param tuple the result containing all the key values of the row
+     * @param ptr the bytes pointer to the underlying result
+     */
+    public void aggregate(Tuple tuple, ImmutableBytesWritable ptr);
+    
+    /**
+     * Get the size in bytes
+     */
+    public int getSize();
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/Aggregators.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/Aggregators.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/Aggregators.java
new file mode 100644
index 0000000..18dfc4a
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/Aggregators.java
@@ -0,0 +1,133 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.aggregator;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.expression.function.SingleAggregateFunction;
+import org.apache.phoenix.schema.KeyValueSchema;
+import org.apache.phoenix.schema.KeyValueSchema.KeyValueSchemaBuilder;
+import org.apache.phoenix.schema.ValueBitSet;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.SizedUtil;
+
+
+/**
+ * 
+ * Represents an ordered list of Aggregators
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+abstract public class Aggregators {
+    protected final int estimatedByteSize;
+    protected final KeyValueSchema schema;
+    protected final ImmutableBytesWritable ptr = new ImmutableBytesWritable();
+    protected final ValueBitSet valueSet;
+    protected final Aggregator[] aggregators;
+    protected final SingleAggregateFunction[] functions;
+    
+    public int getEstimatedByteSize() {
+        return estimatedByteSize;
+    }
+    
+    public Aggregators(SingleAggregateFunction[] functions, Aggregator[] aggregators, int minNullableIndex) {
+        this.functions = functions;
+        this.aggregators = aggregators;
+        this.estimatedByteSize = calculateSize(aggregators);
+        this.schema = newValueSchema(aggregators, minNullableIndex);
+        this.valueSet = ValueBitSet.newInstance(schema);
+    }
+    
+    public KeyValueSchema getValueSchema() {
+        return schema;
+    }
+    
+    @Override
+    public String toString() {
+        StringBuilder buf = new StringBuilder(this.getClass().getName() + " [" + functions.length + "]:");
+        for (int i = 0; i < functions.length; i++) {
+            SingleAggregateFunction function = functions[i];
+            buf.append("\t" + i + ") " + function );
+        }
+        return buf.toString();
+    }
+    
+    /**
+     * Return the aggregate functions
+     */
+    public SingleAggregateFunction[] getFunctions() {
+        return functions;
+    }
+    
+    /**
+     * Aggregate over aggregators
+     * @param result the single row Result from scan iteration
+     */
+    abstract public void aggregate(Aggregator[] aggregators, Tuple result);
+
+    protected static int calculateSize(Aggregator[] aggregators) {
+        
+        int size = SizedUtil.ARRAY_SIZE /*aggregators[]*/  + (SizedUtil.POINTER_SIZE  * aggregators.length);
+        for (Aggregator aggregator : aggregators) {
+            size += aggregator.getSize();
+        }
+        return size;
+    }
+    
+    /**
+     * Get the ValueSchema for the Aggregators
+     */
+    private static KeyValueSchema newValueSchema(Aggregator[] aggregators, int minNullableIndex) {
+        KeyValueSchemaBuilder builder = new KeyValueSchemaBuilder(minNullableIndex);
+        for (int i = 0; i < aggregators.length; i++) {
+            Aggregator aggregator = aggregators[i];
+            builder.addField(aggregator);
+        }
+        return builder.build();
+    }
+
+    /**
+     * @return byte representation of the ValueSchema
+     */
+    public byte[] toBytes(Aggregator[] aggregators) {
+        return schema.toBytes(aggregators, valueSet, ptr);
+    }
+    
+    public int getAggregatorCount() {
+        return aggregators.length;
+    }
+
+    public Aggregator[] getAggregators() {
+        return aggregators;
+    }
+    
+    abstract public Aggregator[] newAggregators();
+    
+    public void reset(Aggregator[] aggregators) {
+        for (int i = 0; i < aggregators.length; i++) {
+            aggregators[i].reset();
+        }
+    }
+    
+    protected Aggregator getAggregator(int position) {
+        return aggregators[position];
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/BaseAggregator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/BaseAggregator.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/BaseAggregator.java
new file mode 100644
index 0000000..8dd6668
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/BaseAggregator.java
@@ -0,0 +1,59 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.aggregator;
+
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.expression.BaseTerminalExpression;
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.util.SizedUtil;
+
+/**
+ * Base class for Aggregator implementations
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public abstract class BaseAggregator extends BaseTerminalExpression implements Aggregator {
+    
+    protected final ColumnModifier columnModifier;    
+    
+    public BaseAggregator(ColumnModifier columnModifier) {
+        this.columnModifier = columnModifier;
+    }
+    
+    @Override
+    public boolean isNullable() {
+        return true;
+    }
+    
+    @Override
+    public int getSize() {
+        return SizedUtil.OBJECT_SIZE;
+    }
+    
+    ImmutableBytesWritable evalClientAggs(Aggregator clientAgg) {
+        CountAggregator ca = (CountAggregator)clientAgg;
+        ImmutableBytesWritable ptr = new ImmutableBytesWritable();
+        ca.evaluate(null, ptr);
+        return ptr;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/BaseDecimalStddevAggregator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/BaseDecimalStddevAggregator.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/BaseDecimalStddevAggregator.java
new file mode 100644
index 0000000..b28cb92
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/BaseDecimalStddevAggregator.java
@@ -0,0 +1,119 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.aggregator;
+
+import java.math.*;
+import java.util.List;
+import java.util.Map.Entry;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Pair;
+
+import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.expression.ColumnExpression;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.*;
+import org.apache.phoenix.util.BigDecimalUtil.Operation;
+
+/**
+ * 
+ * @author anoopsjohn
+ * @since 1.2.1
+ */
+public abstract class BaseDecimalStddevAggregator extends DistinctValueWithCountClientAggregator {
+
+    private BigDecimal cachedResult = null;
+    private int colPrecision;
+    private int colScale;
+
+    public BaseDecimalStddevAggregator(List<Expression> exps, ColumnModifier columnModifier) {
+        super(columnModifier);
+        ColumnExpression stdDevColExp = (ColumnExpression)exps.get(0);
+        this.colPrecision = stdDevColExp.getMaxLength();
+        this.colScale = stdDevColExp.getScale();
+    }
+
+    @Override
+    protected int getBufferLength() {
+        return PDataType.DECIMAL.getByteSize();
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        if (cachedResult == null) {
+            BigDecimal ssd = sumSquaredDeviation();
+            ssd = ssd.divide(new BigDecimal(getDataPointsCount()), PDataType.DEFAULT_MATH_CONTEXT);
+            // Calculate the precision for the stddev result.
+            // There are totalCount #Decimal values for which we are calculating the stddev
+            // The resultant precision depends on precision and scale of all these values. (See
+            // BigDecimalUtil.getResultPrecisionScale)
+            // As of now we are not using the actual precision and scale of individual values but just using the table
+            // column's max length(precision) and scale for each of the values.
+            int resultPrecision = colPrecision;
+            for (int i = 1; i < this.totalCount; i++) {
+                // Max precision that we can support is 38 See PDataType.MAX_PRECISION
+                if (resultPrecision >= PDataType.MAX_PRECISION) break;
+                Pair<Integer, Integer> precisionScale = BigDecimalUtil.getResultPrecisionScale(this.colPrecision,
+                        this.colScale, this.colPrecision, this.colScale, Operation.OTHERS);
+                resultPrecision = precisionScale.getFirst();
+            }
+            cachedResult = new BigDecimal(Math.sqrt(ssd.doubleValue()), new MathContext(resultPrecision,
+                    RoundingMode.HALF_UP));
+            cachedResult.setScale(this.colScale, RoundingMode.HALF_UP);
+        }
+        if (buffer == null) {
+            initBuffer();
+        }
+        buffer = PDataType.DECIMAL.toBytes(cachedResult);
+        ptr.set(buffer);
+        return true;
+    }
+
+    protected abstract long getDataPointsCount();
+
+    private BigDecimal sumSquaredDeviation() {
+        BigDecimal m = mean();
+        BigDecimal result = BigDecimal.ZERO;
+        for (Entry<ImmutableBytesPtr, Integer> entry : valueVsCount.entrySet()) {
+            BigDecimal colValue = (BigDecimal)PDataType.DECIMAL.toObject(entry.getKey());
+            BigDecimal delta = colValue.subtract(m);
+            result = result.add(delta.multiply(delta).multiply(new BigDecimal(entry.getValue())));
+        }
+        return result;
+    }
+
+    private BigDecimal mean() {
+        BigDecimal sum = BigDecimal.ZERO;
+        for (Entry<ImmutableBytesPtr, Integer> entry : valueVsCount.entrySet()) {
+            BigDecimal colValue = (BigDecimal)PDataType.DECIMAL.toObject(entry.getKey());
+            sum = sum.add(colValue.multiply(new BigDecimal(entry.getValue())));
+        }
+        return sum.divide(new BigDecimal(totalCount), PDataType.DEFAULT_MATH_CONTEXT);
+    }
+
+    @Override
+    public void reset() {
+        super.reset();
+        this.cachedResult = null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/BaseStddevAggregator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/BaseStddevAggregator.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/BaseStddevAggregator.java
new file mode 100644
index 0000000..eaafc79
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/BaseStddevAggregator.java
@@ -0,0 +1,96 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.aggregator;
+
+import java.math.BigDecimal;
+import java.util.List;
+import java.util.Map.Entry;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+/**
+ * 
+ * @author anoopsjohn
+ * @since 1.2.1
+ */
+public abstract class BaseStddevAggregator extends DistinctValueWithCountClientAggregator {
+
+    protected Expression stdDevColExp;
+    private BigDecimal cachedResult = null;
+
+    public BaseStddevAggregator(List<Expression> exps, ColumnModifier columnModifier) {
+        super(columnModifier);
+        this.stdDevColExp = exps.get(0);
+    }
+
+    @Override
+    protected int getBufferLength() {
+        return PDataType.DECIMAL.getByteSize();
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        if (cachedResult == null) {
+            double ssd = sumSquaredDeviation();
+            double result = Math.sqrt(ssd / getDataPointsCount());
+            cachedResult = new BigDecimal(result);
+        }
+        if (buffer == null) {
+            initBuffer();
+        }
+        buffer = PDataType.DECIMAL.toBytes(cachedResult);
+        ptr.set(buffer);
+        return true;
+    }
+    
+    protected abstract long getDataPointsCount();
+    
+    private double sumSquaredDeviation() {
+        double m = mean();
+        double result = 0.0;
+        for (Entry<ImmutableBytesPtr, Integer> entry : valueVsCount.entrySet()) {
+            double colValue = (Double)PDataType.DOUBLE.toObject(entry.getKey(), this.stdDevColExp.getDataType());
+            double delta = colValue - m;
+            result += (delta * delta) * entry.getValue();
+        }
+        return result;
+    }
+
+    private double mean() {
+        double sum = 0.0;
+        for (Entry<ImmutableBytesPtr, Integer> entry : valueVsCount.entrySet()) {
+            double colValue = (Double)PDataType.DOUBLE.toObject(entry.getKey(), this.stdDevColExp.getDataType());
+            sum += colValue * entry.getValue();
+        }
+        return sum / totalCount;
+    }
+    
+    @Override
+    public void reset() {
+        super.reset();
+        this.cachedResult = null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/ClientAggregators.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/ClientAggregators.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/ClientAggregators.java
new file mode 100644
index 0000000..0ac5baf
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/ClientAggregators.java
@@ -0,0 +1,80 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.aggregator;
+
+import java.util.List;
+
+import org.apache.phoenix.expression.function.SingleAggregateFunction;
+import org.apache.phoenix.schema.ValueBitSet;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.TupleUtil;
+
+
+
+/**
+ * 
+ * Aggregators that execute on the client-side
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class ClientAggregators extends Aggregators {
+    private final ValueBitSet tempValueSet; 
+  
+    private static Aggregator[] getAggregators(List<SingleAggregateFunction> aggFuncs) {
+        Aggregator[] aggregators = new Aggregator[aggFuncs.size()];
+        for (int i = 0; i < aggregators.length; i++) {
+            aggregators[i] = aggFuncs.get(i).getAggregator();
+        }
+        return aggregators;
+    }
+    
+    public ClientAggregators(List<SingleAggregateFunction> functions, int minNullableIndex) {
+        super(functions.toArray(new SingleAggregateFunction[functions.size()]), getAggregators(functions), minNullableIndex);
+        this.tempValueSet = ValueBitSet.newInstance(schema);
+    }
+    
+    @Override
+    public void aggregate(Aggregator[] aggregators, Tuple result) {
+        TupleUtil.getAggregateValue(result, ptr);
+        tempValueSet.clear();
+        tempValueSet.or(ptr);
+
+        int i = 0, maxOffset = ptr.getOffset() + ptr.getLength();
+        Boolean hasValue;
+        schema.iterator(ptr);
+        while ((hasValue=schema.next(ptr, i, maxOffset, tempValueSet)) != null) {
+            if (hasValue) {
+                aggregators[i].aggregate(result, ptr);
+            }
+            i++;
+        }
+    }
+    
+    @Override
+    public Aggregator[] newAggregators() {
+        Aggregator[] aggregators = new Aggregator[functions.length];
+        for (int i = 0; i < functions.length; i++) {
+            aggregators[i] = functions[i].newClientAggregator();
+        }
+        return aggregators;
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/CountAggregator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/CountAggregator.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/CountAggregator.java
new file mode 100644
index 0000000..33740c2
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/CountAggregator.java
@@ -0,0 +1,92 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.aggregator;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.SizedUtil;
+
+/**
+ * 
+ * Aggregator for COUNT aggregations
+ * 
+ * @author jtaylor
+ * @since 0.1
+ */
+public class CountAggregator extends BaseAggregator {
+
+    private long count = 0;
+    private byte[] buffer = null;
+
+    public CountAggregator() {
+        super(null);
+    }
+
+    public CountAggregator(LongSumAggregator clientAgg) {
+        this();
+        count = clientAgg.getSum();
+    }
+
+    @Override
+    public void aggregate(Tuple tuple, ImmutableBytesWritable ptr) {
+        count++;
+    }
+
+    @Override
+    public boolean isNullable() {
+        return false;
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        if (buffer == null) {
+            buffer = new byte[getDataType().getByteSize()];
+        }
+        getDataType().getCodec().encodeLong(count, buffer, 0);
+        ptr.set(buffer);
+        return true;
+    }
+
+    @Override
+    public final PDataType getDataType() {
+        return PDataType.LONG;
+    }
+
+    @Override
+    public void reset() {
+        count = 0;
+        buffer = null;
+        super.reset();
+    }
+
+    @Override
+    public String toString() {
+        return "COUNT [count=" + count + "]";
+    }
+
+    @Override
+    public int getSize() {
+        return super.getSize() + SizedUtil.LONG_SIZE + SizedUtil.ARRAY_SIZE
+                + getDataType().getByteSize();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/DecimalStddevPopAggregator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/DecimalStddevPopAggregator.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/DecimalStddevPopAggregator.java
new file mode 100644
index 0000000..c707057
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/DecimalStddevPopAggregator.java
@@ -0,0 +1,43 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.aggregator;
+
+import java.util.List;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.schema.ColumnModifier;
+
+/**
+ * Client side Aggregator for STDDEV_POP aggregations for DECIMAL data type.
+ * 
+ * @author anoopsjohn
+ * @since 1.2.1
+ */
+public class DecimalStddevPopAggregator extends BaseDecimalStddevAggregator {
+
+    public DecimalStddevPopAggregator(List<Expression> exps, ColumnModifier columnModifier) {
+        super(exps, columnModifier);
+    }
+
+    @Override
+    protected long getDataPointsCount() {
+        return totalCount;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/DecimalStddevSampAggregator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/DecimalStddevSampAggregator.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/DecimalStddevSampAggregator.java
new file mode 100644
index 0000000..0aa1928
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/DecimalStddevSampAggregator.java
@@ -0,0 +1,43 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.aggregator;
+
+import java.util.List;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.schema.ColumnModifier;
+
+/**
+ * Client side Aggregator for STDDEV_SAMP aggregations for DECIMAL data type.
+ * 
+ * @author anoopsjohn
+ * @since 1.2.1
+ */
+public class DecimalStddevSampAggregator extends BaseDecimalStddevAggregator {
+
+    public DecimalStddevSampAggregator(List<Expression> exps, ColumnModifier columnModifier) {
+        super(exps, columnModifier);
+    }
+
+    @Override
+    protected long getDataPointsCount() {
+        return totalCount - 1;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/DecimalSumAggregator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/DecimalSumAggregator.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/DecimalSumAggregator.java
new file mode 100644
index 0000000..5b91ef3
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/DecimalSumAggregator.java
@@ -0,0 +1,103 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.aggregator;
+
+import java.math.BigDecimal;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.SizedUtil;
+
+
+/**
+ * 
+ * Aggregator that sums BigDecimal values
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class DecimalSumAggregator extends BaseAggregator {
+    private BigDecimal sum = BigDecimal.ZERO;
+    private byte[] sumBuffer;
+    
+    public DecimalSumAggregator(ColumnModifier columnModifier, ImmutableBytesWritable ptr) {
+        super(columnModifier);
+        if (ptr != null) {
+            initBuffer();
+            sum = (BigDecimal)PDataType.DECIMAL.toObject(ptr);
+        }
+    }
+    
+    private PDataType getInputDataType() {
+        return PDataType.DECIMAL;
+    }
+    
+    private int getBufferLength() {
+        return getDataType().getByteSize();
+    }
+
+    private void initBuffer() {
+        sumBuffer = new byte[getBufferLength()];
+    }
+    
+    @Override
+    public void aggregate(Tuple tuple, ImmutableBytesWritable ptr) {
+        BigDecimal value = (BigDecimal)getDataType().toObject(ptr, getInputDataType(), columnModifier);
+        sum = sum.add(value);
+        if (sumBuffer == null) {
+            sumBuffer = new byte[getDataType().getByteSize()];
+        }
+    }
+    
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        if (sumBuffer == null) {
+            return false;
+        }
+        int len = getDataType().toBytes(sum, sumBuffer, 0);
+        ptr.set(sumBuffer, 0, len);
+        return true;
+    }
+    
+    @Override
+    public final PDataType getDataType() {
+        return PDataType.DECIMAL;
+    }
+    
+    @Override
+    public void reset() {
+        sum = BigDecimal.ZERO;
+        sumBuffer = null;
+        super.reset();
+    }
+
+    @Override
+    public String toString() {
+        return "DECIMAL SUM [sum=" + sum + "]";
+    }
+
+    @Override
+    public int getSize() {
+        return super.getSize() + SizedUtil.BIG_DECIMAL_SIZE + SizedUtil.ARRAY_SIZE + getDataType().getByteSize();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/DistinctCountClientAggregator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/DistinctCountClientAggregator.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/DistinctCountClientAggregator.java
new file mode 100644
index 0000000..d221e91
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/DistinctCountClientAggregator.java
@@ -0,0 +1,55 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.aggregator;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+/**
+ * Client side Aggregator for DISTINCT COUNT aggregations
+ * 
+ * @author anoopsjohn
+ * @since 1.2.1
+ */
+public class DistinctCountClientAggregator extends DistinctValueWithCountClientAggregator {
+
+    public DistinctCountClientAggregator(ColumnModifier columnModifier) {
+        super(columnModifier);
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        if (buffer == null) {
+            initBuffer();
+        }
+        long value = this.valueVsCount.size();
+        buffer = PDataType.LONG.toBytes(value);
+        ptr.set(buffer);
+        return true;
+    }
+
+    @Override
+    protected int getBufferLength() {
+        return PDataType.LONG.getByteSize();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/DistinctValueWithCountClientAggregator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/DistinctValueWithCountClientAggregator.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/DistinctValueWithCountClientAggregator.java
new file mode 100644
index 0000000..c009a28
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/DistinctValueWithCountClientAggregator.java
@@ -0,0 +1,132 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.aggregator;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.TreeMap;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.WritableUtils;
+
+import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+/**
+ * Client side Aggregator which will aggregate data and find distinct values with number of occurrences for each.
+ * 
+ * @author anoopsjohn
+ * @since 1.2.1
+ */
+public abstract class DistinctValueWithCountClientAggregator extends BaseAggregator {
+    protected Map<ImmutableBytesPtr, Integer> valueVsCount = new HashMap<ImmutableBytesPtr, Integer>();
+    protected byte[] buffer;
+    protected long totalCount = 0L;
+
+    public DistinctValueWithCountClientAggregator(ColumnModifier columnModifier) {
+        super(columnModifier);
+    }
+
+    @Override
+    public void aggregate(Tuple tuple, ImmutableBytesWritable ptr) {
+        InputStream is = new ByteArrayInputStream(ptr.get(), ptr.getOffset() + 1, ptr.getLength() - 1);
+        try {
+            if (Bytes.equals(ptr.get(), ptr.getOffset(), 1, DistinctValueWithCountServerAggregator.COMPRESS_MARKER, 0,
+                    1)) {
+                InputStream decompressionStream = DistinctValueWithCountServerAggregator.COMPRESS_ALGO
+                        .createDecompressionStream(is,
+                                DistinctValueWithCountServerAggregator.COMPRESS_ALGO.getDecompressor(), 0);
+                is = decompressionStream;
+            }
+            DataInputStream in = new DataInputStream(is);
+            int mapSize = WritableUtils.readVInt(in);
+            for (int i = 0; i < mapSize; i++) {
+                int keyLen = WritableUtils.readVInt(in);
+                byte[] keyBytes = new byte[keyLen];
+                in.read(keyBytes, 0, keyLen);
+                ImmutableBytesPtr key = new ImmutableBytesPtr(keyBytes);
+                int value = WritableUtils.readVInt(in);
+                Integer curCount = valueVsCount.get(key);
+                if (curCount == null) {
+                    valueVsCount.put(key, value);
+                } else {
+                    valueVsCount.put(key, curCount + value);
+                }
+                totalCount += value;
+            }
+        } catch (IOException ioe) {
+            throw new RuntimeException(ioe); // Impossible as we're using a ByteArrayInputStream
+        }
+        if (buffer == null) {
+            initBuffer();
+        }
+    }
+
+    protected abstract int getBufferLength();
+
+    protected void initBuffer() {
+        buffer = new byte[getBufferLength()];
+    }
+
+    @Override
+    public boolean isNullable() {
+        return false;
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return PDataType.VARBINARY;
+    }
+
+    @Override
+    public void reset() {
+        valueVsCount = new HashMap<ImmutableBytesPtr, Integer>();
+        buffer = null;
+        totalCount = 0L;
+        super.reset();
+    }
+    
+    protected Map<Object, Integer> getSortedValueVsCount(final boolean ascending, final PDataType type) {
+        // To sort the valueVsCount
+        Comparator<Object> comparator = new Comparator<Object>() {
+            @Override
+            public int compare(Object o1, Object o2) {
+                if (ascending) { 
+                    return type.compareTo(o1, o2); 
+                }
+                return type.compareTo(o2, o1);
+            }
+        };
+        Map<Object, Integer> sorted = new TreeMap<Object, Integer>(comparator);
+        for (Entry<ImmutableBytesPtr, Integer> entry : valueVsCount.entrySet()) {
+            sorted.put(type.toObject(entry.getKey(), columnModifier), entry.getValue());
+        }
+        return sorted;
+    }
+}


[35/51] [partial] Initial commit of master branch from github

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..9f01fff
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
@@ -0,0 +1,269 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.coprocessor;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.ipc.CoprocessorProtocol;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableUtils;
+
+import com.google.common.collect.Lists;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTableImpl;
+import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.MetaDataUtil;
+
+
+/**
+ * 
+ * Coprocessor protocol for Phoenix DDL. Phoenix stores the table metadata in
+ * an HBase table named SYSTEM.TABLE. Each table is represented by:
+ * - one row for the table
+ * - one row per column in the tabe
+ * Upto {@link #DEFAULT_MAX_META_DATA_VERSIONS} versions are kept. The time
+ * stamp of the metadata must always be increasing. The timestamp of the key
+ * values in the data row corresponds to the schema that it's using.
+ *
+ * TODO: dynamically prune number of schema version kept based on whether or
+ * not the data table still uses it (based on the min time stamp of the data
+ * table).
+ * 
+ * @author jtaylor
+ * @since 0.1
+ */
+public interface MetaDataProtocol extends CoprocessorProtocol {
+    public static final int PHOENIX_MAJOR_VERSION = 3;
+    public static final int PHOENIX_MINOR_VERSION = 0;
+    public static final int PHOENIX_PATCH_NUMBER = 0;
+    public static final int PHOENIX_VERSION = 
+            MetaDataUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER);
+    
+    public static final long MIN_TABLE_TIMESTAMP = 0;
+    // Increase MIN_SYSTEM_TABLE_TIMESTAMP by one for each schema change SYSTEM.TABLE schema changes.
+    // For 1.0,1.1,1.2,and 1.2.1 we used MetaDataProtocol.MIN_TABLE_TIMESTAMP+1
+    // For 2.0 and above, we use MetaDataProtocol.MIN_TABLE_TIMESTAMP+7 so that we can add the five new
+    // columns to the existing system table (three new columns in 1.2.1 and two new columns in 1.2)
+    // For 3.0 and above, we use MIN_TABLE_TIMESTAMP + 8 so that we can add the tenant_id column
+    // as the first column to the existing system table.
+    // For 3.1 (SNAPSHOT) and above, we use MIN_TABLE_TIMESTAMP + 9 so that we can add
+    // the multi_tenant and multi_type columns for multi tenancy
+    public static final long MIN_SYSTEM_TABLE_TIMESTAMP = MIN_TABLE_TIMESTAMP + 9;
+    public static final int DEFAULT_MAX_META_DATA_VERSIONS = 1000;
+
+    // TODO: pare this down to minimum, as we don't need duplicates for both table and column errors, nor should we need
+    // a different code for every type of error.
+    // ENTITY_ALREADY_EXISTS, ENTITY_NOT_FOUND, NEWER_ENTITY_FOUND, ENTITY_NOT_IN_REGION, CONCURRENT_MODIFICATION
+    // ILLEGAL_MUTATION (+ sql code)
+    public enum MutationCode {
+        TABLE_ALREADY_EXISTS,
+        TABLE_NOT_FOUND, 
+        COLUMN_NOT_FOUND, 
+        COLUMN_ALREADY_EXISTS,
+        CONCURRENT_TABLE_MUTATION,
+        TABLE_NOT_IN_REGION,
+        NEWER_TABLE_FOUND,
+        UNALLOWED_TABLE_MUTATION,
+        NO_PK_COLUMNS,
+        PARENT_TABLE_NOT_FOUND
+    };
+    
+    public static class MetaDataMutationResult implements Writable {
+        private MutationCode returnCode;
+        private long mutationTime;
+        private PTable table;
+        private List<byte[]> tableNamesToDelete;
+        private byte[] columnName;
+        private byte[] familyName;
+        private boolean wasUpdated;
+        
+        public MetaDataMutationResult() {
+        }
+
+        public MetaDataMutationResult(MutationCode returnCode, long currentTime, PTable table, PColumn column) {
+            this(returnCode, currentTime, table);
+            if(column != null){
+                this.columnName = column.getName().getBytes();
+                this.familyName = column.getFamilyName().getBytes();    
+            }
+            
+        }
+        
+        public MetaDataMutationResult(MutationCode returnCode, long currentTime, PTable table) {
+           this(returnCode, currentTime, table, Collections.<byte[]> emptyList());
+        }
+        
+        public MetaDataMutationResult(MutationCode returnCode, long currentTime, PTable table, List<byte[]> tableNamesToDelete) {
+            this.returnCode = returnCode;
+            this.mutationTime = currentTime;
+            this.table = table;
+            this.tableNamesToDelete = tableNamesToDelete;
+        }
+        
+        public MutationCode getMutationCode() {
+            return returnCode;
+        }
+        
+        public long getMutationTime() {
+            return mutationTime;
+        }
+        
+        public boolean wasUpdated() {
+            return wasUpdated;
+        }
+        
+        public PTable getTable() {
+            return table;
+        }
+ 
+        public void setTable(PTable table) {
+            this.table = table;
+        }
+ 
+        public List<byte[]> getTableNamesToDelete() {
+            return tableNamesToDelete;
+        }
+        
+        public byte[] getColumnName() {
+            return columnName;
+        }
+        
+        public byte[] getFamilyName() {
+            return familyName;
+        }        
+        
+        @Override
+        public void readFields(DataInput input) throws IOException {
+            this.returnCode = MutationCode.values()[WritableUtils.readVInt(input)];
+            this.mutationTime = input.readLong();
+            wasUpdated = input.readBoolean();
+            if (wasUpdated) {
+                this.table = new PTableImpl();
+                this.table.readFields(input);
+            }
+            columnName = Bytes.readByteArray(input);
+            if (columnName.length > 0) {
+                familyName = Bytes.readByteArray(input);
+            }
+            boolean hasTablesToDelete = input.readBoolean();
+            if (hasTablesToDelete) {
+                int count = input.readInt();
+                tableNamesToDelete = Lists.newArrayListWithExpectedSize(count);
+                for( int i = 0 ; i < count ; i++ ){
+                     byte[] tableName = Bytes.readByteArray(input);
+                     tableNamesToDelete.add(tableName);
+                }
+            }
+        }
+
+        @Override
+        public void write(DataOutput output) throws IOException {
+            WritableUtils.writeVInt(output, returnCode.ordinal());
+            output.writeLong(mutationTime);
+            output.writeBoolean(table != null);
+            if (table != null) {
+                table.write(output);
+            }
+            Bytes.writeByteArray(output, columnName == null ? ByteUtil.EMPTY_BYTE_ARRAY : columnName);
+            if (columnName != null) {
+                 Bytes.writeByteArray(output, familyName == null ? ByteUtil.EMPTY_BYTE_ARRAY : familyName);
+            }
+            if(tableNamesToDelete != null && tableNamesToDelete.size() > 0 ) {
+                output.writeBoolean(true);
+                output.writeInt(tableNamesToDelete.size());
+                for(byte[] tableName : tableNamesToDelete) {
+                    Bytes.writeByteArray(output,tableName);    
+                }
+                
+            } else {
+                output.writeBoolean(false);
+            }
+            
+        }
+    }
+    
+    /**
+     * The the latest Phoenix table at or before the given clientTimestamp. If the
+     * client already has the latest (based on the tableTimestamp), then no table
+     * is returned.
+     * @param tenantId
+     * @param schemaName
+     * @param tableName
+     * @param tableTimestamp
+     * @param clientTimestamp
+     * @return MetaDataMutationResult
+     * @throws IOException
+     */
+    MetaDataMutationResult getTable(byte[] tenantId, byte[] schemaName, byte[] tableName, long tableTimestamp, long clientTimestamp) throws IOException;
+
+    /**
+     * Create a new Phoenix table
+     * @param tableMetadata
+     * @return MetaDataMutationResult
+     * @throws IOException
+     */
+    MetaDataMutationResult createTable(List<Mutation> tableMetadata) throws IOException;
+
+    /**
+     * Drop an existing Phoenix table
+     * @param tableMetadata
+     * @param tableType
+     * @return MetaDataMutationResult
+     * @throws IOException
+     */
+    MetaDataMutationResult dropTable(List<Mutation> tableMetadata, String tableType) throws IOException;
+
+    /**
+     * Add a column to an existing Phoenix table
+     * @param tableMetadata
+     * @return MetaDataMutationResult
+     * @throws IOException
+     */
+    MetaDataMutationResult addColumn(List<Mutation> tableMetadata) throws IOException;
+    
+    /**
+     * Drop a column from an existing Phoenix table
+     * @param tableMetadata
+     * @return MetaDataMutationResult
+     * @throws IOException
+     */
+    MetaDataMutationResult dropColumn(List<Mutation> tableMetadata) throws IOException;
+    
+    MetaDataMutationResult updateIndexState(List<Mutation> tableMetadata) throws IOException;
+
+    /**
+     * Clears the server-side cache of table meta data. Used between test runs to
+     * ensure no side effects.
+     */
+    void clearCache();
+    
+    /**
+     * Get the version of the server-side HBase and phoenix.jar. Used when initially connecting
+     * to a cluster to ensure that the client and server jars are compatible.
+     */
+    long getVersion();
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..d0bd7c5
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
@@ -0,0 +1,40 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.coprocessor;
+
+import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+
+import org.apache.phoenix.cache.GlobalCache;
+
+
+/**
+ * Coprocessor for metadata related operations. This coprocessor would only be registered
+ * to SYSTEM.TABLE.
+ */
+public class MetaDataRegionObserver extends BaseRegionObserver {
+
+    @Override
+    public void preClose(final ObserverContext<RegionCoprocessorEnvironment> c,
+            boolean abortRequested) {
+        GlobalCache.getInstance(c.getEnvironment()).getMetaDataCache().clear();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ScanProjector.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ScanProjector.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ScanProjector.java
new file mode 100644
index 0000000..ee9e6e2
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ScanProjector.java
@@ -0,0 +1,202 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.coprocessor;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.io.WritableUtils;
+
+import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.KeyValueUtil;
+
+public class ScanProjector {
+    
+    public enum ProjectionType {TABLE, CF, CQ};
+    
+    private static final String SCAN_PROJECTOR = "scanProjector";
+    private static final byte[] SEPERATOR = Bytes.toBytes(":");
+    
+    private final ProjectionType type;
+    private final byte[] tablePrefix;
+    private final Map<ImmutableBytesPtr, byte[]> cfProjectionMap;
+    private final Map<ImmutableBytesPtr, Map<ImmutableBytesPtr, Pair<byte[], byte[]>>> cqProjectionMap;
+    
+    private ScanProjector(ProjectionType type, byte[] tablePrefix, 
+            Map<ImmutableBytesPtr, byte[]> cfProjectionMap, Map<ImmutableBytesPtr, 
+            Map<ImmutableBytesPtr, Pair<byte[], byte[]>>> cqProjectionMap) {
+        this.type = ProjectionType.TABLE;
+        this.tablePrefix = tablePrefix;
+        this.cfProjectionMap = cfProjectionMap;
+        this.cqProjectionMap = cqProjectionMap;
+    }
+    
+    public static void serializeProjectorIntoScan(Scan scan, ScanProjector projector) {
+        ByteArrayOutputStream stream = new ByteArrayOutputStream();
+        try {
+            DataOutputStream output = new DataOutputStream(stream);
+            WritableUtils.writeVInt(output, projector.type.ordinal());
+            switch (projector.type) {
+            case TABLE:
+                WritableUtils.writeCompressedByteArray(output, projector.tablePrefix);
+                break;
+            case CF:
+                WritableUtils.writeVInt(output, projector.cfProjectionMap.size());
+                for (Map.Entry<ImmutableBytesPtr, byte[]> entry : projector.cfProjectionMap.entrySet()) {
+                    WritableUtils.writeCompressedByteArray(output, entry.getKey().get());
+                    WritableUtils.writeCompressedByteArray(output, entry.getValue());
+                }
+                break;
+            case CQ:
+                WritableUtils.writeVInt(output, projector.cqProjectionMap.size());
+                for (Map.Entry<ImmutableBytesPtr, Map<ImmutableBytesPtr, Pair<byte[], byte[]>>> entry : 
+                    projector.cqProjectionMap.entrySet()) {
+                    WritableUtils.writeCompressedByteArray(output, entry.getKey().get());
+                    Map<ImmutableBytesPtr, Pair<byte[], byte[]>> map = entry.getValue();
+                    WritableUtils.writeVInt(output, map.size());
+                    for (Map.Entry<ImmutableBytesPtr, Pair<byte[], byte[]>> e : map.entrySet()) {
+                        WritableUtils.writeCompressedByteArray(output, e.getKey().get());
+                        WritableUtils.writeCompressedByteArray(output, e.getValue().getFirst());
+                        WritableUtils.writeCompressedByteArray(output, e.getValue().getSecond());
+                    }
+                }
+                break;
+            default:
+                throw new IOException("Unrecognized projection type '" + projector.type + "'");    
+            }
+            scan.setAttribute(SCAN_PROJECTOR, stream.toByteArray());
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        } finally {
+            try {
+                stream.close();
+            } catch (IOException e) {
+                throw new RuntimeException(e);
+            }
+        }
+        
+    }
+    
+    public static ScanProjector deserializeProjectorFromScan(Scan scan) {
+        byte[] proj = scan.getAttribute(SCAN_PROJECTOR);
+        if (proj == null) {
+            return null;
+        }
+        ByteArrayInputStream stream = new ByteArrayInputStream(proj);
+        try {
+            DataInputStream input = new DataInputStream(stream);
+            int t = WritableUtils.readVInt(input);
+            ProjectionType type = ProjectionType.values()[t];
+            if (type == ProjectionType.TABLE) {
+                byte[] tablePrefix = WritableUtils.readCompressedByteArray(input);
+                return new ScanProjector(type, tablePrefix, null, null);
+            }
+            if (type == ProjectionType.CF) {
+                int count = WritableUtils.readVInt(input);
+                Map<ImmutableBytesPtr, byte[]> cfMap = new HashMap<ImmutableBytesPtr, byte[]>();
+                for (int i = 0; i < count; i++) {
+                    byte[] cf = WritableUtils.readCompressedByteArray(input);
+                    byte[] renamed = WritableUtils.readCompressedByteArray(input);
+                    cfMap.put(new ImmutableBytesPtr(cf), renamed);
+                }
+                return new ScanProjector(type, null, cfMap, null);
+            }
+            
+            int count = WritableUtils.readVInt(input);
+            Map<ImmutableBytesPtr, Map<ImmutableBytesPtr, Pair<byte[], byte[]>>> cqMap = 
+                new HashMap<ImmutableBytesPtr, Map<ImmutableBytesPtr, Pair<byte[], byte[]>>>();
+            for (int i = 0; i < count; i++) {
+                byte[] cf = WritableUtils.readCompressedByteArray(input);
+                int nQuals = WritableUtils.readVInt(input);
+                Map<ImmutableBytesPtr, Pair<byte[], byte[]>> map = 
+                    new HashMap<ImmutableBytesPtr, Pair<byte[], byte[]>>();
+                for (int j = 0; j < nQuals; j++) {
+                    byte[] cq = WritableUtils.readCompressedByteArray(input);
+                    byte[] renamedCf = WritableUtils.readCompressedByteArray(input);
+                    byte[] renamedCq = WritableUtils.readCompressedByteArray(input);
+                    map.put(new ImmutableBytesPtr(cq), new Pair<byte[], byte[]>(renamedCf, renamedCq));
+                }
+                cqMap.put(new ImmutableBytesPtr(cf), map);
+            }
+            return new ScanProjector(type, null, null, cqMap);
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        } finally {
+            try {
+                stream.close();
+            } catch (IOException e) {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+    
+    public ProjectionType getType() {
+        return this.type;
+    }
+    
+    public byte[] getTablePrefix() {
+        return this.tablePrefix;
+    }
+    
+    public Map<ImmutableBytesPtr, byte[]> getCfProjectionMap() {
+        return this.cfProjectionMap;
+    }
+    
+    public Map<ImmutableBytesPtr, Map<ImmutableBytesPtr, Pair<byte[], byte[]>>> getCqProjectionMap() {
+        return this.cqProjectionMap;
+    }
+    
+    public KeyValue getProjectedKeyValue(KeyValue kv) {
+        if (type == ProjectionType.TABLE) {
+            byte[] cf = ByteUtil.concat(tablePrefix, SEPERATOR, kv.getFamily());
+            return KeyValueUtil.newKeyValue(kv.getBuffer(), kv.getKeyOffset(), kv.getKeyLength(), 
+                    cf, kv.getQualifier(), kv.getTimestamp(), kv.getBuffer(), kv.getValueOffset(), kv.getValueLength());
+        }
+        
+        if (type == ProjectionType.CF) {
+            byte[] cf = cfProjectionMap.get(new ImmutableBytesPtr(kv.getFamily()));
+            if (cf == null)
+                return kv;
+            return KeyValueUtil.newKeyValue(kv.getBuffer(), kv.getKeyOffset(), kv.getKeyLength(), 
+                    cf, kv.getQualifier(), kv.getTimestamp(), kv.getBuffer(), kv.getValueOffset(), kv.getValueLength());
+        }
+        
+        Map<ImmutableBytesPtr, Pair<byte[], byte[]>> map = cqProjectionMap.get(new ImmutableBytesPtr(kv.getFamily()));
+        if (map == null)
+            return kv;
+        
+        Pair<byte[], byte[]> col = map.get(new ImmutableBytesPtr(kv.getQualifier()));
+        if (col == null)
+            return kv;
+        
+        return KeyValueUtil.newKeyValue(kv.getBuffer(), kv.getKeyOffset(), kv.getKeyLength(), 
+                col.getFirst(), col.getSecond(), kv.getTimestamp(), kv.getBuffer(), kv.getValueOffset(), kv.getValueLength());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..cb76b70
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ScanRegionObserver.java
@@ -0,0 +1,313 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.coprocessor;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.WritableUtils;
+
+import com.google.common.collect.Lists;
+import org.apache.phoenix.cache.GlobalCache;
+import org.apache.phoenix.cache.TenantCache;
+import org.apache.phoenix.expression.OrderByExpression;
+import org.apache.phoenix.iterate.OrderedResultIterator;
+import org.apache.phoenix.iterate.RegionScannerResultIterator;
+import org.apache.phoenix.iterate.ResultIterator;
+import org.apache.phoenix.join.HashJoinInfo;
+import org.apache.phoenix.join.ScanProjector;
+import org.apache.phoenix.memory.MemoryManager.MemoryChunk;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.ScanUtil;
+import org.apache.phoenix.util.ServerUtil;
+
+
+/**
+ * 
+ * Wraps the scan performing a non aggregate query to prevent needless retries
+ * if a Phoenix bug is encountered from our custom filter expression evaluation.
+ * Unfortunately, until HBASE-7481 gets fixed, there's no way to do this from our
+ * custom filters.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class ScanRegionObserver extends BaseScannerRegionObserver {
+    public static final String NON_AGGREGATE_QUERY = "NonAggregateQuery";
+    private static final String TOPN = "TopN";
+
+    public static void serializeIntoScan(Scan scan, int thresholdBytes, int limit, List<OrderByExpression> orderByExpressions, int estimatedRowSize) {
+        ByteArrayOutputStream stream = new ByteArrayOutputStream(); // TODO: size?
+        try {
+            DataOutputStream output = new DataOutputStream(stream);
+            WritableUtils.writeVInt(output, thresholdBytes);
+            WritableUtils.writeVInt(output, limit);
+            WritableUtils.writeVInt(output, estimatedRowSize);
+            WritableUtils.writeVInt(output, orderByExpressions.size());
+            for (OrderByExpression orderingCol : orderByExpressions) {
+                orderingCol.write(output);
+            }
+            scan.setAttribute(TOPN, stream.toByteArray());
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        } finally {
+            try {
+                stream.close();
+            } catch (IOException e) {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+    
+    public static OrderedResultIterator deserializeFromScan(Scan scan, RegionScanner s) {
+        byte[] topN = scan.getAttribute(TOPN);
+        if (topN == null) {
+            return null;
+        }
+        ByteArrayInputStream stream = new ByteArrayInputStream(topN); // TODO: size?
+        try {
+            DataInputStream input = new DataInputStream(stream);
+            int thresholdBytes = WritableUtils.readVInt(input);
+            int limit = WritableUtils.readVInt(input);
+            int estimatedRowSize = WritableUtils.readVInt(input);
+            int size = WritableUtils.readVInt(input);
+            List<OrderByExpression> orderByExpressions = Lists.newArrayListWithExpectedSize(size);           
+            for (int i = 0; i < size; i++) {
+                OrderByExpression orderByExpression = new OrderByExpression();
+                orderByExpression.readFields(input);
+                orderByExpressions.add(orderByExpression);
+            }
+            ResultIterator inner = new RegionScannerResultIterator(s);
+            return new OrderedResultIterator(inner, orderByExpressions, thresholdBytes, limit >= 0 ? limit : null, estimatedRowSize);
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        } finally {
+            try {
+                stream.close();
+            } catch (IOException e) {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+
+    @Override
+    protected RegionScanner doPostScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c, final Scan scan, final RegionScanner s) throws Throwable {
+        byte[] isScanQuery = scan.getAttribute(NON_AGGREGATE_QUERY);
+
+        if (isScanQuery == null || Bytes.compareTo(PDataType.FALSE_BYTES, isScanQuery) == 0) {
+            return s;
+        }
+        
+        final ScanProjector p = ScanProjector.deserializeProjectorFromScan(scan);
+        final HashJoinInfo j = HashJoinInfo.deserializeHashJoinFromScan(scan);
+        final ImmutableBytesWritable tenantId = ScanUtil.getTenantId(scan);
+        
+        RegionScanner innerScanner = s;
+        if (p != null || j != null) {
+            innerScanner = new HashJoinRegionScanner(s, p, j, tenantId, c.getEnvironment());
+        }
+        
+        final OrderedResultIterator iterator = deserializeFromScan(scan,innerScanner);
+        if (iterator == null) {
+            return getWrappedScanner(c, innerScanner);
+        }
+        
+        return getTopNScanner(c, innerScanner, iterator, tenantId);
+    }
+    
+    /**
+     *  Return region scanner that does TopN.
+     *  We only need to call startRegionOperation and closeRegionOperation when
+     *  getting the first Tuple (which forces running through the entire region)
+     *  since after this everything is held in memory
+     */
+    private RegionScanner getTopNScanner(final ObserverContext<RegionCoprocessorEnvironment> c, final RegionScanner s, final OrderedResultIterator iterator, ImmutableBytesWritable tenantId) throws Throwable {
+        final Tuple firstTuple;
+        TenantCache tenantCache = GlobalCache.getTenantCache(c.getEnvironment(), tenantId);
+        long estSize = iterator.getEstimatedByteSize();
+        final MemoryChunk chunk = tenantCache.getMemoryManager().allocate(estSize);
+        final HRegion region = c.getEnvironment().getRegion();
+        region.startRegionOperation();
+        try {
+            // Once we return from the first call to next, we've run through and cached
+            // the topN rows, so we no longer need to start/stop a region operation.
+            firstTuple = iterator.next();
+            // Now that the topN are cached, we can resize based on the real size
+            long actualSize = iterator.getByteSize();
+            chunk.resize(actualSize);
+        } catch (Throwable t) {
+            ServerUtil.throwIOException(region.getRegionNameAsString(), t);
+            return null;
+        } finally {
+            region.closeRegionOperation();
+        }
+        return new BaseRegionScanner() {
+            private Tuple tuple = firstTuple;
+            
+            @Override
+            public boolean isFilterDone() {
+                return tuple == null; 
+            }
+
+            @Override
+            public HRegionInfo getRegionInfo() {
+                return s.getRegionInfo();
+            }
+
+            @Override
+            public boolean next(List<KeyValue> results) throws IOException {
+                try {
+                    if (isFilterDone()) {
+                        return false;
+                    }
+                    
+                    for (int i = 0; i < tuple.size(); i++) {
+                        results.add(tuple.getValue(i));
+                    }
+                    
+                    tuple = iterator.next();
+                    return !isFilterDone();
+                } catch (Throwable t) {
+                    ServerUtil.throwIOException(region.getRegionNameAsString(), t);
+                    return false;
+                }
+            }
+
+            @Override
+            public void close() throws IOException {
+                try {
+                    s.close();
+                } finally {
+                    chunk.close();                }
+            }
+        };
+    }
+        
+    /**
+     * Return wrapped scanner that catches unexpected exceptions (i.e. Phoenix bugs) and
+     * re-throws as DoNotRetryIOException to prevent needless retrying hanging the query
+     * for 30 seconds. Unfortunately, until HBASE-7481 gets fixed, there's no way to do
+     * the same from a custom filter.
+     */
+    private RegionScanner getWrappedScanner(final ObserverContext<RegionCoprocessorEnvironment> c, final RegionScanner s) {
+        return new RegionScanner() {
+
+            @Override
+            public boolean next(List<KeyValue> results) throws IOException {
+                try {
+                    return s.next(results);
+                } catch (Throwable t) {
+                    ServerUtil.throwIOException(c.getEnvironment().getRegion().getRegionNameAsString(), t);
+                    return false; // impossible
+                }
+            }
+
+            @Override
+            public boolean next(List<KeyValue> results, String metric) throws IOException {
+                try {
+                    return s.next(results, metric);
+                } catch (Throwable t) {
+                    ServerUtil.throwIOException(c.getEnvironment().getRegion().getRegionNameAsString(), t);
+                    return false; // impossible
+                }
+            }
+
+            @Override
+            public boolean next(List<KeyValue> result, int limit) throws IOException {
+                try {
+                    return s.next(result, limit);
+                } catch (Throwable t) {
+                    ServerUtil.throwIOException(c.getEnvironment().getRegion().getRegionNameAsString(), t);
+                    return false; // impossible
+                }
+            }
+
+            @Override
+            public boolean next(List<KeyValue> result, int limit, String metric) throws IOException {
+                try {
+                    return s.next(result, limit, metric);
+                } catch (Throwable t) {
+                    ServerUtil.throwIOException(c.getEnvironment().getRegion().getRegionNameAsString(), t);
+                    return false; // impossible
+                }
+            }
+
+            @Override
+            public void close() throws IOException {
+                s.close();
+            }
+
+            @Override
+            public HRegionInfo getRegionInfo() {
+                return s.getRegionInfo();
+            }
+
+            @Override
+            public boolean isFilterDone() {
+                return s.isFilterDone();
+            }
+
+            @Override
+            public boolean reseek(byte[] row) throws IOException {
+                return s.reseek(row);
+            }
+            
+            @Override
+            public long getMvccReadPoint() {
+                return s.getMvccReadPoint();
+            }
+
+            @Override
+            public boolean nextRaw(List<KeyValue> result, String metric) throws IOException {
+                try {
+                    return s.nextRaw(result, metric);
+                } catch (Throwable t) {
+                    ServerUtil.throwIOException(c.getEnvironment().getRegion().getRegionNameAsString(), t);
+                    return false; // impossible
+                }
+            }
+
+            @Override
+            public boolean nextRaw(List<KeyValue> result, int limit, String metric) throws IOException {
+                try {
+                    return s.nextRaw(result, limit, metric);
+                } catch (Throwable t) {
+                    ServerUtil.throwIOException(c.getEnvironment().getRegion().getRegionNameAsString(), t);
+                    return false; // impossible
+                }
+            }
+        };
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/SequenceRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/SequenceRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/SequenceRegionObserver.java
new file mode 100644
index 0000000..ed04cc3
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/SequenceRegionObserver.java
@@ -0,0 +1,258 @@
+package org.apache.phoenix.coprocessor;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Append;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Increment;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.io.TimeRange;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.Pair;
+
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.Sequence;
+import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.KeyValueUtil;
+import org.apache.phoenix.util.MetaDataUtil;
+import org.apache.phoenix.util.ServerUtil;
+
+/**
+ * 
+ * Region observer coprocessor for sequence operations:
+ * 1) For creating a sequence, as checkAndPut does not allow us to scope the
+ * Get done for the check with a TimeRange.
+ * 2) For incrementing a sequence, as increment does not a) allow us to set the
+ * timestamp of the key value being incremented and b) recognize when the key
+ * value being incremented does not exist
+ * 3) For deleting a sequence, as checkAndDelete does not allow us to scope
+ * the Get done for the check with a TimeRange.
+ *
+ * @author jtaylor
+ * @since 3.0.0
+ */
+public class SequenceRegionObserver extends BaseRegionObserver {
+    public enum Op {CREATE_SEQUENCE, DROP_SEQUENCE, RETURN_SEQUENCE};
+    public static final String OPERATION_ATTRIB = "SEQUENCE_OPERATION";
+    public static final String MAX_TIMERANGE_ATTRIB = "MAX_TIMERANGE";
+    public static final String CURRENT_VALUE_ATTRIB = "CURRENT_VALUE";
+    private static final byte[] SUCCESS_VALUE = PDataType.INTEGER.toBytes(Integer.valueOf(Sequence.SUCCESS));
+    
+    private static Result getErrorResult(byte[] row, long timestamp, int errorCode) {
+        byte[] errorCodeBuf = new byte[PDataType.INTEGER.getByteSize()];
+        PDataType.INTEGER.getCodec().encodeInt(errorCode, errorCodeBuf, 0);
+        return new Result(Collections.singletonList(
+                KeyValueUtil.newKeyValue(row, 
+                        PhoenixDatabaseMetaData.SEQUENCE_FAMILY_BYTES, 
+                        QueryConstants.EMPTY_COLUMN_BYTES, timestamp, errorCodeBuf)));
+    }
+    /**
+     * 
+     * Use PreIncrement hook of BaseRegionObserver to overcome deficiencies in Increment
+     * implementation (HBASE-10254):
+     * 1) Lack of recognition and identification of when the key value to increment doesn't exist
+     * 2) Lack of the ability to set the timestamp of the updated key value.
+     * Works the same as existing region.increment(), except assumes there is a single column to
+     * increment and uses Phoenix LONG encoding.
+     * @author jtaylor
+     * @since 3.0.0
+     */
+    @Override
+    public Result preIncrement(final ObserverContext<RegionCoprocessorEnvironment> e,
+        final Increment increment) throws IOException {
+        RegionCoprocessorEnvironment env = e.getEnvironment();
+        // We need to set this to prevent region.increment from being called
+        e.bypass();
+        e.complete();
+        HRegion region = env.getRegion();
+        byte[] row = increment.getRow();
+        TimeRange tr = increment.getTimeRange();
+        region.startRegionOperation();
+        try {
+            Integer lid = region.getLock(null, row, true);
+            try {
+                long maxTimestamp = tr.getMax();
+                if (maxTimestamp == HConstants.LATEST_TIMESTAMP) {
+                    maxTimestamp = EnvironmentEdgeManager.currentTimeMillis();
+                    tr = new TimeRange(tr.getMin(), maxTimestamp);
+                }
+                Get get = new Get(row);
+                get.setTimeRange(tr.getMin(), tr.getMax());
+                for (Map.Entry<byte[],NavigableMap<byte[], Long>> entry : increment.getFamilyMap().entrySet()) {
+                    byte[] cf = entry.getKey();
+                    for (byte[] cq : entry.getValue().keySet()) {
+                        get.addColumn(cf, cq);
+                    }
+                }
+                Result result = region.get(get);
+                if (result.isEmpty()) {
+                    return getErrorResult(row, maxTimestamp, SQLExceptionCode.SEQUENCE_UNDEFINED.getErrorCode());
+                }
+                KeyValue currentValueKV = Sequence.getCurrentValueKV(result);
+                KeyValue incrementByKV = Sequence.getIncrementByKV(result);
+                KeyValue cacheSizeKV = Sequence.getCacheSizeKV(result);
+                long value = PDataType.LONG.getCodec().decodeLong(currentValueKV.getBuffer(), currentValueKV.getValueOffset(), null);
+                long incrementBy = PDataType.LONG.getCodec().decodeLong(incrementByKV.getBuffer(), incrementByKV.getValueOffset(), null);
+                int cacheSize = PDataType.INTEGER.getCodec().decodeInt(cacheSizeKV.getBuffer(), cacheSizeKV.getValueOffset(), null);
+                value += incrementBy * cacheSize;
+                byte[] valueBuffer = new byte[PDataType.LONG.getByteSize()];
+                PDataType.LONG.getCodec().encodeLong(value, valueBuffer, 0);
+                Put put = new Put(row, currentValueKV.getTimestamp());
+                // Hold timestamp constant for sequences, so that clients always only see the latest value
+                // regardless of when they connect.
+                KeyValue newCurrentValueKV = KeyValueUtil.newKeyValue(row, currentValueKV.getFamily(), currentValueKV.getQualifier(), currentValueKV.getTimestamp(), valueBuffer);
+                put.add(newCurrentValueKV);
+                @SuppressWarnings("unchecked")
+                Pair<Mutation,Integer>[] mutations = new Pair[1];
+                mutations[0] = new Pair<Mutation,Integer>(put, lid);
+                region.batchMutate(mutations);
+                return Sequence.replaceCurrentValueKV(result, newCurrentValueKV);
+            } finally {
+                region.releaseRowLock(lid);
+            }
+        } catch (Throwable t) {
+            ServerUtil.throwIOException("Increment of sequence " + Bytes.toStringBinary(row), t);
+            return null; // Impossible
+        } finally {
+            region.closeRegionOperation();
+        }
+    }
+
+    /**
+     * Override the preAppend for checkAndPut and checkAndDelete, as we need the ability to
+     * a) set the TimeRange for the Get being done and
+     * b) return something back to the client to indicate success/failure
+     */
+    @SuppressWarnings("deprecation")
+    @Override
+    public Result preAppend(final ObserverContext<RegionCoprocessorEnvironment> e,
+            final Append append) throws IOException {
+        byte[] opBuf = append.getAttribute(OPERATION_ATTRIB);
+        if (opBuf == null) {
+            return null;
+        }
+        Op op = Op.values()[opBuf[0]];
+        KeyValue keyValue = append.getFamilyMap().values().iterator().next().iterator().next();
+
+        long clientTimestamp = HConstants.LATEST_TIMESTAMP;
+        long minGetTimestamp = MetaDataProtocol.MIN_TABLE_TIMESTAMP;
+        long maxGetTimestamp = HConstants.LATEST_TIMESTAMP;
+        boolean hadClientTimestamp;
+        byte[] clientTimestampBuf = null;
+        if (op == Op.RETURN_SEQUENCE) {
+            // When returning sequences, this allows us to send the expected timestamp
+            // of the sequence to make sure we don't reset any other sequence
+            hadClientTimestamp = true;
+            clientTimestamp = minGetTimestamp = keyValue.getTimestamp();
+            maxGetTimestamp = minGetTimestamp + 1;
+        } else {
+            clientTimestampBuf = append.getAttribute(MAX_TIMERANGE_ATTRIB);
+            if (clientTimestampBuf != null) {
+                clientTimestamp = maxGetTimestamp = Bytes.toLong(clientTimestampBuf);
+            }
+            hadClientTimestamp = (clientTimestamp != HConstants.LATEST_TIMESTAMP);
+            if (hadClientTimestamp) {
+                // Prevent race condition of creating two sequences at the same timestamp
+                // by looking for a sequence at or after the timestamp at which it'll be
+                // created.
+                if (op == Op.CREATE_SEQUENCE) {
+                    maxGetTimestamp = clientTimestamp + 1;
+                }            
+            } else {
+                clientTimestamp = maxGetTimestamp = EnvironmentEdgeManager.currentTimeMillis();
+                clientTimestampBuf = Bytes.toBytes(clientTimestamp);
+            }
+        }
+
+        RegionCoprocessorEnvironment env = e.getEnvironment();
+        // We need to set this to prevent region.append from being called
+        e.bypass();
+        e.complete();
+        HRegion region = env.getRegion();
+        byte[] row = append.getRow();
+        region.startRegionOperation();
+        try {
+            Integer lid = region.getLock(null, row, true);
+            try {
+                byte[] family = keyValue.getFamily();
+                byte[] qualifier = keyValue.getQualifier();
+
+                Get get = new Get(row);
+                get.setTimeRange(minGetTimestamp, maxGetTimestamp);
+                get.addColumn(family, qualifier);
+                Result result = region.get(get);
+                if (result.isEmpty()) {
+                    if (op == Op.DROP_SEQUENCE || op == Op.RETURN_SEQUENCE) {
+                        return getErrorResult(row, clientTimestamp, SQLExceptionCode.SEQUENCE_UNDEFINED.getErrorCode());
+                    }
+                } else {
+                    if (op == Op.CREATE_SEQUENCE) {
+                        return getErrorResult(row, clientTimestamp, SQLExceptionCode.SEQUENCE_ALREADY_EXIST.getErrorCode());
+                    }
+                }
+                Mutation m = null;
+                switch (op) {
+                case RETURN_SEQUENCE:
+                    KeyValue currentValueKV = result.raw()[0];
+                    long expectedValue = PDataType.LONG.getCodec().decodeLong(append.getAttribute(CURRENT_VALUE_ATTRIB), 0, null);
+                    long value = PDataType.LONG.getCodec().decodeLong(currentValueKV.getBuffer(), currentValueKV.getValueOffset(), null);
+                    // Timestamp should match exactly, or we may have the wrong sequence
+                    if (expectedValue != value || currentValueKV.getTimestamp() != clientTimestamp) {
+                        return new Result(Collections.singletonList(KeyValueUtil.newKeyValue(row, PhoenixDatabaseMetaData.SEQUENCE_FAMILY_BYTES, QueryConstants.EMPTY_COLUMN_BYTES, currentValueKV.getTimestamp(), ByteUtil.EMPTY_BYTE_ARRAY)));
+                    }
+                    m = new Put(row, currentValueKV.getTimestamp());
+                    m.getFamilyMap().putAll(append.getFamilyMap());
+                    break;
+                case DROP_SEQUENCE:
+                    m = new Delete(row, clientTimestamp, null);
+                    break;
+                case CREATE_SEQUENCE:
+                    m = new Put(row, clientTimestamp);
+                    m.getFamilyMap().putAll(append.getFamilyMap());
+                    break;
+                }
+                if (!hadClientTimestamp) {
+                    for (List<KeyValue> kvs : m.getFamilyMap().values()) {
+                        for (KeyValue kv : kvs) {
+                            kv.updateLatestStamp(clientTimestampBuf);
+                        }
+                    }
+                }
+                @SuppressWarnings("unchecked")
+                Pair<Mutation,Integer>[] mutations = new Pair[1];
+                mutations[0] = new Pair<Mutation,Integer>(m, lid);
+                region.batchMutate(mutations);
+                long serverTimestamp = MetaDataUtil.getClientTimeStamp(m);
+                // Return result with single KeyValue. The only piece of information
+                // the client cares about is the timestamp, which is the timestamp of
+                // when the mutation was actually performed (useful in the case of .
+                return new Result(Collections.singletonList(KeyValueUtil.newKeyValue(row, PhoenixDatabaseMetaData.SEQUENCE_FAMILY_BYTES, QueryConstants.EMPTY_COLUMN_BYTES, serverTimestamp, SUCCESS_VALUE)));
+            } finally {
+                region.releaseRowLock(lid);
+            }
+        } catch (Throwable t) {
+            ServerUtil.throwIOException("Increment of sequence " + Bytes.toStringBinary(row), t);
+            return null; // Impossible
+        } finally {
+            region.closeRegionOperation();
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..b27c14c
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ServerCachingEndpointImpl.java
@@ -0,0 +1,57 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.coprocessor;
+
+import java.sql.SQLException;
+
+import org.apache.hadoop.hbase.coprocessor.BaseEndpointCoprocessor;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.cache.GlobalCache;
+import org.apache.phoenix.cache.TenantCache;
+
+
+
+
+/**
+ * 
+ * Server-side implementation of {@link ServerCachingProtocol}
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class ServerCachingEndpointImpl extends BaseEndpointCoprocessor implements ServerCachingProtocol {
+
+    @Override
+    public boolean addServerCache(byte[] tenantId, byte[] cacheId, ImmutableBytesWritable cachePtr, ServerCacheFactory cacheFactory) throws SQLException {
+        TenantCache tenantCache = GlobalCache.getTenantCache((RegionCoprocessorEnvironment)this.getEnvironment(), tenantId == null ? null : new ImmutableBytesPtr(tenantId));
+        tenantCache.addServerCache(new ImmutableBytesPtr(cacheId), cachePtr, cacheFactory);
+        return true;
+    }
+
+    @Override
+    public boolean removeServerCache(byte[] tenantId, byte[] cacheId) throws SQLException {
+        TenantCache tenantCache = GlobalCache.getTenantCache((RegionCoprocessorEnvironment)this.getEnvironment(), tenantId == null ? null : new ImmutableBytesPtr(tenantId));
+        tenantCache.removeServerCache(new ImmutableBytesPtr(cacheId));
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..abda834
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ServerCachingProtocol.java
@@ -0,0 +1,63 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.coprocessor;
+
+import java.io.Closeable;
+import java.sql.SQLException;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.ipc.CoprocessorProtocol;
+import org.apache.hadoop.io.Writable;
+
+import org.apache.phoenix.memory.MemoryManager.MemoryChunk;
+
+/**
+ * 
+ * EndPoint coprocessor to send a cache to a region server.
+ * Used for:
+ * a) hash joins, to send the smaller side of the join to each region server
+ * b) secondary indexes, to send the necessary meta data to each region server
+ * @author jtaylor
+ * @since 0.1
+ */
+public interface ServerCachingProtocol extends CoprocessorProtocol {
+    public static interface ServerCacheFactory extends Writable {
+        public Closeable newCache(ImmutableBytesWritable cachePtr, MemoryChunk chunk) throws SQLException;
+    }
+    /**
+     * Add the cache to the region server cache.  
+     * @param tenantId the tenantId or null if not applicable
+     * @param cacheId unique identifier of the cache
+     * @param cachePtr pointer to the byte array of the cache
+     * @param cacheFactory factory that converts from byte array to object representation on the server side
+     * @return true on success and otherwise throws
+     * @throws SQLException 
+     */
+    public boolean addServerCache(byte[] tenantId, byte[] cacheId, ImmutableBytesWritable cachePtr, ServerCacheFactory cacheFactory) throws SQLException;
+    /**
+     * Remove the cache from the region server cache.  Called upon completion of
+     * the operation when cache is no longer needed.
+     * @param tenantId the tenantId or null if not applicable
+     * @param cacheId unique identifier of the cache
+     * @return true on success and otherwise throws
+     * @throws SQLException 
+     */
+    public boolean removeServerCache(byte[] tenantId, byte[] cacheId) throws SQLException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..c7c4575
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
@@ -0,0 +1,423 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.coprocessor;
+
+import static org.apache.phoenix.query.QueryConstants.AGG_TIMESTAMP;
+import static org.apache.phoenix.query.QueryConstants.SINGLE_COLUMN;
+import static org.apache.phoenix.query.QueryConstants.SINGLE_COLUMN_FAMILY;
+import static org.apache.phoenix.query.QueryConstants.UNGROUPED_AGG_ROW_KEY;
+import static org.apache.phoenix.query.QueryServices.MUTATE_BATCH_SIZE_ATTRIB;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.hadoop.hbase.CoprocessorEnvironment;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.MultiVersionConsistencyControl;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.apache.phoenix.client.KeyValueBuilder;
+import org.apache.phoenix.exception.ValueTypeIncompatibleException;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.ExpressionType;
+import org.apache.phoenix.expression.aggregator.Aggregator;
+import org.apache.phoenix.expression.aggregator.Aggregators;
+import org.apache.phoenix.expression.aggregator.ServerAggregators;
+import org.apache.phoenix.index.PhoenixIndexCodec;
+import org.apache.phoenix.join.HashJoinInfo;
+import org.apache.phoenix.join.ScanProjector;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.ConstraintViolationException;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.PRow;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTableImpl;
+import org.apache.phoenix.schema.tuple.MultiKeyValueTuple;
+import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.KeyValueUtil;
+import org.apache.phoenix.util.ScanUtil;
+import org.apache.phoenix.util.SchemaUtil;
+
+
+/**
+ * Region observer that aggregates ungrouped rows(i.e. SQL query with aggregation function and no GROUP BY).
+ * 
+ * @author jtaylor
+ * @since 0.1
+ */
+public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver {
+    private static final Logger logger = LoggerFactory.getLogger(UngroupedAggregateRegionObserver.class);
+    // TODO: move all constants into a single class
+    public static final String UNGROUPED_AGG = "UngroupedAgg";
+    public static final String DELETE_AGG = "DeleteAgg";
+    public static final String UPSERT_SELECT_TABLE = "UpsertSelectTable";
+    public static final String UPSERT_SELECT_EXPRS = "UpsertSelectExprs";
+    public static final String DELETE_CQ = "DeleteCQ";
+    public static final String DELETE_CF = "DeleteCF";
+    public static final String EMPTY_CF = "EmptyCF";
+    private KeyValueBuilder kvBuilder;
+    
+    @Override
+    public void start(CoprocessorEnvironment e) throws IOException {
+        super.start(e);
+        this.kvBuilder = KeyValueBuilder.get(e.getHBaseVersion());
+    }
+
+    private static void commitBatch(HRegion region, List<Pair<Mutation,Integer>> mutations, byte[] indexUUID) throws IOException {
+        if (indexUUID != null) {
+            for (Pair<Mutation,Integer> pair : mutations) {
+                pair.getFirst().setAttribute(PhoenixIndexCodec.INDEX_UUID, indexUUID);
+            }
+        }
+        @SuppressWarnings("unchecked")
+        Pair<Mutation,Integer>[] mutationArray = new Pair[mutations.size()];
+        // TODO: should we use the one that is all or none?
+        region.batchMutate(mutations.toArray(mutationArray));
+    }
+    
+    public static void serializeIntoScan(Scan scan) {
+        scan.setAttribute(UNGROUPED_AGG, QueryConstants.TRUE);
+    }
+
+    @Override
+    protected RegionScanner doPostScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c, final Scan scan, final RegionScanner s) throws IOException {
+        byte[] isUngroupedAgg = scan.getAttribute(UNGROUPED_AGG);
+        if (isUngroupedAgg == null) {
+            return s;
+        }
+        
+        final ScanProjector p = ScanProjector.deserializeProjectorFromScan(scan);
+        final HashJoinInfo j = HashJoinInfo.deserializeHashJoinFromScan(scan);
+        RegionScanner theScanner = s;
+        if (p != null || j != null)  {
+            theScanner = new HashJoinRegionScanner(s, p, j, ScanUtil.getTenantId(scan), c.getEnvironment());
+        }
+        final RegionScanner innerScanner = theScanner;
+        
+        byte[] indexUUID = scan.getAttribute(PhoenixIndexCodec.INDEX_UUID);
+        PTable projectedTable = null;
+        List<Expression> selectExpressions = null;
+        byte[] upsertSelectTable = scan.getAttribute(UPSERT_SELECT_TABLE);
+        boolean isUpsert = false;
+        boolean isDelete = false;
+        byte[] deleteCQ = null;
+        byte[] deleteCF = null;
+        byte[][] values = null;
+        byte[] emptyCF = null;
+        ImmutableBytesWritable ptr = null;
+        if (upsertSelectTable != null) {
+            isUpsert = true;
+            projectedTable = deserializeTable(upsertSelectTable);
+            selectExpressions = deserializeExpressions(scan.getAttribute(UPSERT_SELECT_EXPRS));
+            values = new byte[projectedTable.getPKColumns().size()][];
+            ptr = new ImmutableBytesWritable();
+        } else {
+            byte[] isDeleteAgg = scan.getAttribute(DELETE_AGG);
+            isDelete = isDeleteAgg != null && Bytes.compareTo(PDataType.TRUE_BYTES, isDeleteAgg) == 0;
+            if (!isDelete) {
+                deleteCF = scan.getAttribute(DELETE_CF);
+                deleteCQ = scan.getAttribute(DELETE_CQ);
+            }
+            emptyCF = scan.getAttribute(EMPTY_CF);
+        }
+        
+        int batchSize = 0;
+        long ts = scan.getTimeRange().getMax();
+        HRegion region = c.getEnvironment().getRegion();
+        List<Pair<Mutation,Integer>> mutations = Collections.emptyList();
+        if (isDelete || isUpsert || (deleteCQ != null && deleteCF != null) || emptyCF != null) {
+            // TODO: size better
+            mutations = Lists.newArrayListWithExpectedSize(1024);
+            batchSize = c.getEnvironment().getConfiguration().getInt(MUTATE_BATCH_SIZE_ATTRIB, QueryServicesOptions.DEFAULT_MUTATE_BATCH_SIZE);
+        }
+        Aggregators aggregators = ServerAggregators.deserialize(
+                scan.getAttribute(GroupedAggregateRegionObserver.AGGREGATORS), c.getEnvironment().getConfiguration());
+        Aggregator[] rowAggregators = aggregators.getAggregators();
+        boolean hasMore;
+        boolean hasAny = false;
+        MultiKeyValueTuple result = new MultiKeyValueTuple();
+        if (logger.isInfoEnabled()) {
+        	logger.info("Starting ungrouped coprocessor scan " + scan);
+        }
+        long rowCount = 0;
+        MultiVersionConsistencyControl.setThreadReadPoint(innerScanner.getMvccReadPoint());
+        region.startRegionOperation();
+        try {
+            do {
+                List<KeyValue> results = new ArrayList<KeyValue>();
+                // 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
+                hasMore = innerScanner.nextRaw(results, null);
+                if (!results.isEmpty()) {
+                	rowCount++;
+                    result.setKeyValues(results);
+                    try {
+                        if (isDelete) {
+                            @SuppressWarnings("deprecation") // FIXME: Remove when unintentionally deprecated method is fixed (HBASE-7870).
+                            // FIXME: the version of the Delete constructor without the lock args was introduced
+                            // in 0.94.4, thus if we try to use it here we can no longer use the 0.94.2 version
+                            // of the client.
+                            Delete delete = new Delete(results.get(0).getRow(),ts,null);
+                            mutations.add(new Pair<Mutation,Integer>(delete,null));
+                        } else if (isUpsert) {
+                            Arrays.fill(values, null);
+                            int i = 0;
+                            List<PColumn> projectedColumns = projectedTable.getColumns();
+                            for (; i < projectedTable.getPKColumns().size(); i++) {
+                                Expression expression = selectExpressions.get(i);
+                                if (expression.evaluate(result, ptr)) {
+                                    values[i] = ptr.copyBytes();
+                                    // If ColumnModifier from expression in SELECT doesn't match the
+                                    // column being projected into then invert the bits.
+                                    if (expression.getColumnModifier() != projectedColumns.get(i).getColumnModifier()) {
+                                        ColumnModifier.SORT_DESC.apply(values[i], 0, values[i], 0, values[i].length);
+                                    }
+                                }
+                            }
+                            projectedTable.newKey(ptr, values);
+                            PRow row = projectedTable.newRow(kvBuilder, ts, ptr);
+                            for (; i < projectedColumns.size(); i++) {
+                                Expression expression = selectExpressions.get(i);
+                                if (expression.evaluate(result, ptr)) {
+                                    PColumn column = projectedColumns.get(i);
+                                    byte[] bytes = ptr.copyBytes();
+                                    Object value = expression.getDataType().toObject(bytes, column.getColumnModifier());
+                                    // If ColumnModifier from expression in SELECT doesn't match the
+                                    // column being projected into then invert the bits.
+                                    if (expression.getColumnModifier() != column.getColumnModifier()) {
+                                        ColumnModifier.SORT_DESC.apply(bytes, 0, bytes, 0, bytes.length);
+                                    }
+                                    // We are guaranteed that the two column will have the same type.
+                                    if (!column.getDataType().isSizeCompatible(column.getDataType(),
+                                            value, bytes,
+                                            expression.getMaxLength(), column.getMaxLength(), 
+                                            expression.getScale(), column.getScale())) {
+                                        throw new ValueTypeIncompatibleException(column.getDataType(),
+                                                column.getMaxLength(), column.getScale());
+                                    }
+                                    bytes = column.getDataType().coerceBytes(bytes, value, expression.getDataType(),
+                                            expression.getMaxLength(), expression.getScale(), column.getMaxLength(), column.getScale());
+                                    row.setValue(column, bytes);
+                                }
+                            }
+                            for (Mutation mutation : row.toRowMutations()) {
+                                mutations.add(new Pair<Mutation,Integer>(mutation,null));
+                            }
+                        } else if (deleteCF != null && deleteCQ != null) {
+                            // No need to search for delete column, since we project only it
+                            // if no empty key value is being set
+                            if (emptyCF == null || result.getValue(deleteCF, deleteCQ) != null) {
+                                Delete delete = new Delete(results.get(0).getRow());
+                                delete.deleteColumns(deleteCF,  deleteCQ, ts);
+                                mutations.add(new Pair<Mutation,Integer>(delete,null));
+                            }
+                        }
+                        if (emptyCF != null) {
+                            /*
+                             * If we've specified an emptyCF, then we need to insert an empty
+                             * key value "retroactively" for any key value that is visible at
+                             * the timestamp that the DDL was issued. Key values that are not
+                             * visible at this timestamp will not ever be projected up to
+                             * scans past this timestamp, so don't need to be considered.
+                             * We insert one empty key value per row per timestamp.
+                             */
+                            Set<Long> timeStamps = Sets.newHashSetWithExpectedSize(results.size());
+                            for (KeyValue kv : results) {
+                                long kvts = kv.getTimestamp();
+                                if (!timeStamps.contains(kvts)) {
+                                    Put put = new Put(kv.getRow());
+                                    put.add(emptyCF, QueryConstants.EMPTY_COLUMN_BYTES, kvts, ByteUtil.EMPTY_BYTE_ARRAY);
+                                    mutations.add(new Pair<Mutation,Integer>(put,null));
+                                }
+                            }
+                        }
+                        // Commit in batches based on UPSERT_BATCH_SIZE_ATTRIB in config
+                        if (!mutations.isEmpty() && batchSize > 0 && mutations.size() % batchSize == 0) {
+                            commitBatch(region,mutations, indexUUID);
+                            mutations.clear();
+                        }
+                    } catch (ConstraintViolationException e) {
+                        // Log and ignore in count
+                        logger.error("Failed to create row in " + region.getRegionNameAsString() + " with values " + SchemaUtil.toString(values), e);
+                        continue;
+                    }
+                    aggregators.aggregate(rowAggregators, result);
+                    hasAny = true;
+                }
+            } while (hasMore);
+        } finally {
+            innerScanner.close();
+            region.closeRegionOperation();
+        }
+        
+        if (logger.isInfoEnabled()) {
+        	logger.info("Finished scanning " + rowCount + " rows for ungrouped coprocessor scan " + scan);
+        }
+
+        if (!mutations.isEmpty()) {
+            commitBatch(region,mutations, indexUUID);
+        }
+
+        final boolean hadAny = hasAny;
+        KeyValue keyValue = null;
+        if (hadAny) {
+            byte[] value = aggregators.toBytes(rowAggregators);
+            keyValue = KeyValueUtil.newKeyValue(UNGROUPED_AGG_ROW_KEY, SINGLE_COLUMN_FAMILY, SINGLE_COLUMN, AGG_TIMESTAMP, value, 0, value.length);
+        }
+        final KeyValue aggKeyValue = keyValue;
+        
+        RegionScanner scanner = new BaseRegionScanner() {
+            private boolean done = !hadAny;
+
+            @Override
+            public HRegionInfo getRegionInfo() {
+                return innerScanner.getRegionInfo();
+            }
+
+            @Override
+            public boolean isFilterDone() {
+                return done;
+            }
+
+            @Override
+            public void close() throws IOException {
+                innerScanner.close();
+            }
+
+            @Override
+            public boolean next(List<KeyValue> results) throws IOException {
+                if (done) return false;
+                done = true;
+                results.add(aggKeyValue);
+                return false;
+            }
+        };
+        return scanner;
+    }
+    
+    private static PTable deserializeTable(byte[] b) {
+        ByteArrayInputStream stream = new ByteArrayInputStream(b);
+        try {
+            DataInputStream input = new DataInputStream(stream);
+            PTable table = new PTableImpl();
+            table.readFields(input);
+            return table;
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        } finally {
+            try {
+                stream.close();
+            } catch (IOException e) {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+
+    private static List<Expression> deserializeExpressions(byte[] b) {
+        ByteArrayInputStream stream = new ByteArrayInputStream(b);
+        try {
+            DataInputStream input = new DataInputStream(stream);
+            int size = WritableUtils.readVInt(input);
+            List<Expression> selectExpressions = Lists.newArrayListWithExpectedSize(size);
+            for (int i = 0; i < size; i++) {
+                ExpressionType type = ExpressionType.values()[WritableUtils.readVInt(input)];
+                Expression selectExpression = type.newInstance();
+                selectExpression.readFields(input);
+                selectExpressions.add(selectExpression);
+            }
+            return selectExpressions;
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        } finally {
+            try {
+                stream.close();
+            } catch (IOException e) {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+
+    public static byte[] serialize(PTable projectedTable) {
+        ByteArrayOutputStream stream = new ByteArrayOutputStream();
+        try {
+            DataOutputStream output = new DataOutputStream(stream);
+            projectedTable.write(output);
+            return stream.toByteArray();
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        } finally {
+            try {
+                stream.close();
+            } catch (IOException e) {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+
+    public static byte[] serialize(List<Expression> selectExpressions) {
+        ByteArrayOutputStream stream = new ByteArrayOutputStream();
+        try {
+            DataOutputStream output = new DataOutputStream(stream);
+            WritableUtils.writeVInt(output, selectExpressions.size());
+            for (int i = 0; i < selectExpressions.size(); i++) {
+                Expression expression = selectExpressions.get(i);
+                WritableUtils.writeVInt(output, ExpressionType.valueOf(expression).ordinal());
+                expression.write(output);
+            }
+            return stream.toByteArray();
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        } finally {
+            try {
+                stream.close();
+            } catch (IOException e) {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/exception/PhoenixIOException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/exception/PhoenixIOException.java b/phoenix-core/src/main/java/org/apache/phoenix/exception/PhoenixIOException.java
new file mode 100644
index 0000000..d1b0b18
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/exception/PhoenixIOException.java
@@ -0,0 +1,33 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.exception;
+
+import java.sql.SQLException;
+
+
+public class PhoenixIOException extends SQLException {
+    private static final long serialVersionUID = 1L;
+    private static SQLExceptionCode code = SQLExceptionCode.IO_EXCEPTION;
+
+    public PhoenixIOException(Throwable e) {
+        super(e.getMessage(), code.getSQLState(), code.getErrorCode(), e);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/exception/PhoenixParserException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/exception/PhoenixParserException.java b/phoenix-core/src/main/java/org/apache/phoenix/exception/PhoenixParserException.java
new file mode 100644
index 0000000..fc96923
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/exception/PhoenixParserException.java
@@ -0,0 +1,112 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.exception;
+
+import java.sql.SQLSyntaxErrorException;
+
+import org.antlr.runtime.MismatchedTokenException;
+import org.antlr.runtime.MissingTokenException;
+import org.antlr.runtime.RecognitionException;
+import org.antlr.runtime.Token;
+import org.antlr.runtime.UnwantedTokenException;
+
+
+public class PhoenixParserException extends SQLSyntaxErrorException {
+    private static final long serialVersionUID = 1L;
+
+    public static final PhoenixParserException newException(Throwable cause, String[] tokens) {
+        return new PhoenixParserException(getErrorMessage(cause, tokens), cause);
+    }
+    
+    public PhoenixParserException(String msg, Throwable throwable) {
+        super(new SQLExceptionInfo.Builder(getErrorCode(throwable)).setRootCause(throwable)
+                .setMessage(msg).build().toString(),
+                getErrorCode(throwable).getSQLState(), getErrorCode(throwable).getErrorCode(), throwable);
+    }
+
+    public static String getLine(RecognitionException e) {
+        return Integer.toString(e.token.getLine());
+    }
+
+    public static String getColumn(RecognitionException e) {
+        return Integer.toString(e.token.getCharPositionInLine() + 1);
+    }
+
+    public static String getTokenLocation(RecognitionException e) {
+        return "line " + getLine(e) + ", column " + getColumn(e) + ".";
+    }
+
+    public static String getErrorMessage(Throwable e, String[] tokenNames) {
+        String msg;
+        if (e instanceof MissingTokenException) {
+            MissingTokenException mte = (MissingTokenException)e;
+            String tokenName;
+            if (mte.expecting== Token.EOF) {
+                tokenName = "EOF";
+            } else {
+                tokenName = tokenNames[mte.expecting];
+            }
+            msg = "Missing \""+ tokenName +"\" at "+ getTokenLocation(mte);
+        } else if (e instanceof UnwantedTokenException) {
+            UnwantedTokenException ute = (UnwantedTokenException)e;
+            String tokenName;
+            if (ute.expecting== Token.EOF) {
+                tokenName = "EOF";
+            } else {
+                tokenName = tokenNames[ute.expecting];
+            }
+            msg = "Unexpected input. Expecting \"" + tokenName + "\", got \"" + ute.getUnexpectedToken().getText() 
+                    + "\" at " + getTokenLocation(ute);
+        } else if (e instanceof MismatchedTokenException) {
+            MismatchedTokenException mte = (MismatchedTokenException)e;
+            String tokenName;
+            if (mte.expecting== Token.EOF) {
+                tokenName = "EOF";
+            } else {
+                tokenName = tokenNames[mte.expecting];
+            }
+            msg = "Mismatched input. Expecting \"" + tokenName + "\", got \"" + mte.token.getText()
+                    + "\" at " + getTokenLocation(mte);
+        } else if (e instanceof RecognitionException){
+            RecognitionException re = (RecognitionException) e;
+            msg = "Encountered \"" + re.token.getText() + "\" at " + getTokenLocation(re);
+        } else if (e instanceof UnknownFunctionException) {
+            UnknownFunctionException ufe = (UnknownFunctionException) e;
+            msg = "Unknown function: \"" + ufe.getFuncName() + "\".";
+        } else {
+            msg = e.getMessage();
+        }
+        return msg;
+    }
+
+    public static SQLExceptionCode getErrorCode(Throwable e) {
+        if (e instanceof MissingTokenException) {
+            return SQLExceptionCode.MISSING_TOKEN;
+        } else if (e instanceof UnwantedTokenException) {
+            return SQLExceptionCode.UNWANTED_TOKEN;
+        } else if (e instanceof MismatchedTokenException) {
+            return SQLExceptionCode.MISMATCHED_TOKEN;
+        } else if (e instanceof UnknownFunctionException) {
+            return SQLExceptionCode.UNKNOWN_FUNCTION;
+        } else {
+            return SQLExceptionCode.PARSER_ERROR;
+        }
+    }
+}


[12/51] [partial] Initial commit of master branch from github

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..f107350
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
@@ -0,0 +1,1739 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 com.google.common.collect.Lists.newArrayListWithExpectedSize;
+import static com.google.common.collect.Sets.newLinkedHashSet;
+import static com.google.common.collect.Sets.newLinkedHashSetWithExpectedSize;
+import static org.apache.phoenix.exception.SQLExceptionCode.INSUFFICIENT_MULTI_TENANT_COLUMNS;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ARRAY_SIZE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_COUNT;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_MODIFIER;
+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.DISABLE_WAL;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IMMUTABLE_ROWS;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_STATE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.LINK_TYPE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.MULTI_TENANT;
+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.SALT_BUCKETS;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_CAT_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_NAME_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_SCHEM_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_SEQ_NUM;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_TYPE_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TENANT_ID;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TYPE_SCHEMA;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TYPE_TABLE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_STATEMENT;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_TYPE;
+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.schema.PDataType.VARCHAR;
+
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.sql.Types;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.phoenix.compile.ColumnResolver;
+import org.apache.phoenix.compile.FromCompiler;
+import org.apache.phoenix.compile.MutationPlan;
+import org.apache.phoenix.compile.PostDDLCompiler;
+import org.apache.phoenix.compile.PostIndexDDLCompiler;
+import org.apache.phoenix.coprocessor.MetaDataProtocol;
+import org.apache.phoenix.coprocessor.MetaDataProtocol.MetaDataMutationResult;
+import org.apache.phoenix.coprocessor.MetaDataProtocol.MutationCode;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+import org.apache.phoenix.execute.MutationState;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.parse.AddColumnStatement;
+import org.apache.phoenix.parse.AlterIndexStatement;
+import org.apache.phoenix.parse.ColumnDef;
+import org.apache.phoenix.parse.ColumnName;
+import org.apache.phoenix.parse.CreateIndexStatement;
+import org.apache.phoenix.parse.CreateSequenceStatement;
+import org.apache.phoenix.parse.CreateTableStatement;
+import org.apache.phoenix.parse.DropColumnStatement;
+import org.apache.phoenix.parse.DropIndexStatement;
+import org.apache.phoenix.parse.DropSequenceStatement;
+import org.apache.phoenix.parse.DropTableStatement;
+import org.apache.phoenix.parse.NamedTableNode;
+import org.apache.phoenix.parse.ParseNodeFactory;
+import org.apache.phoenix.parse.PrimaryKeyConstraint;
+import org.apache.phoenix.parse.TableName;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.PTable.LinkType;
+import org.apache.phoenix.schema.PTable.ViewType;
+import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.IndexUtil;
+import org.apache.phoenix.util.MetaDataUtil;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.SchemaUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Objects;
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+
+public class MetaDataClient {
+    private static final Logger logger = LoggerFactory.getLogger(MetaDataClient.class);
+
+    private static final ParseNodeFactory FACTORY = new ParseNodeFactory();
+    private static final String CREATE_TABLE =
+            "UPSERT INTO " + TYPE_SCHEMA + ".\"" + TYPE_TABLE + "\"( " +
+            TENANT_ID + "," +
+            TABLE_SCHEM_NAME + "," +
+            TABLE_NAME_NAME + "," +
+            TABLE_TYPE_NAME + "," +
+            TABLE_SEQ_NUM + "," +
+            COLUMN_COUNT + "," +
+            SALT_BUCKETS + "," +
+            PK_NAME + "," +
+            DATA_TABLE_NAME + "," +
+            INDEX_STATE + "," +
+            IMMUTABLE_ROWS + "," +
+            DEFAULT_COLUMN_FAMILY_NAME + "," +
+            VIEW_STATEMENT + "," +
+            DISABLE_WAL + "," +
+            MULTI_TENANT + "," +
+            VIEW_TYPE +
+            ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
+    private static final String CREATE_LINK =
+            "UPSERT INTO " + TYPE_SCHEMA + ".\"" + TYPE_TABLE + "\"( " +
+            TENANT_ID + "," +
+            TABLE_SCHEM_NAME + "," +
+            TABLE_NAME_NAME + "," +
+            TABLE_CAT_NAME + "," +
+            LINK_TYPE +
+            ") VALUES (?, ?, ?, ?, ?)";
+    private static final String INCREMENT_SEQ_NUM =
+            "UPSERT INTO " + TYPE_SCHEMA + ".\"" + TYPE_TABLE + "\"( " + 
+            TENANT_ID + "," +
+            TABLE_SCHEM_NAME + "," +
+            TABLE_NAME_NAME + "," +
+            TABLE_SEQ_NUM  +
+            ") VALUES (?, ?, ?, ?)";
+    private static final String MUTATE_TABLE =
+        "UPSERT INTO " + TYPE_SCHEMA + ".\"" + TYPE_TABLE + "\"( " + 
+        TENANT_ID + "," +
+        TABLE_SCHEM_NAME + "," +
+        TABLE_NAME_NAME + "," +
+        TABLE_TYPE_NAME + "," +
+        TABLE_SEQ_NUM + "," +
+        COLUMN_COUNT + "," +
+        IMMUTABLE_ROWS + "," +
+        DISABLE_WAL + "," +
+        MULTI_TENANT +
+        ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?)";
+    private static final String UPDATE_INDEX_STATE =
+            "UPSERT INTO " + TYPE_SCHEMA + ".\"" + TYPE_TABLE + "\"( " + 
+            TENANT_ID + "," +
+            TABLE_SCHEM_NAME + "," +
+            TABLE_NAME_NAME + "," +
+            INDEX_STATE +
+            ") VALUES (?, ?, ?, ?)";
+    private static final String INSERT_COLUMN =
+        "UPSERT INTO " + TYPE_SCHEMA + ".\"" + TYPE_TABLE + "\"( " + 
+        TENANT_ID + "," +
+        TABLE_SCHEM_NAME + "," +
+        TABLE_NAME_NAME + "," +
+        COLUMN_NAME + "," +
+        TABLE_CAT_NAME + "," +
+        DATA_TYPE + "," +
+        NULLABLE + "," +
+        COLUMN_SIZE + "," +
+        DECIMAL_DIGITS + "," +
+        ORDINAL_POSITION + "," + 
+        COLUMN_MODIFIER + "," +
+        DATA_TABLE_NAME + "," + // write this both in the column and table rows for access by metadata APIs
+        ARRAY_SIZE +
+        ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
+    private static final String UPDATE_COLUMN_POSITION =
+        "UPSERT INTO " + TYPE_SCHEMA + ".\"" + TYPE_TABLE + "\" ( " + 
+        TENANT_ID + "," +
+        TABLE_SCHEM_NAME + "," +
+        TABLE_NAME_NAME + "," +
+        COLUMN_NAME + "," +
+        TABLE_CAT_NAME + "," +
+        ORDINAL_POSITION +
+        ") VALUES (?, ?, ?, ?, ?, ?)";
+    
+    private final PhoenixConnection connection;
+
+    public MetaDataClient(PhoenixConnection connection) {
+        this.connection = connection;
+    }
+    
+    public PhoenixConnection getConnection() {
+        return connection;
+    }
+
+    public long getCurrentTime(String schemaName, String tableName) throws SQLException {
+        MetaDataMutationResult result = updateCache(schemaName, tableName, true);
+        return result.getMutationTime();
+    }
+    
+    /**
+     * Update the cache with the latest as of the connection scn.
+     * @param schemaName
+     * @param tableName
+     * @return the timestamp from the server, negative if the table was added to the cache and positive otherwise
+     * @throws SQLException
+     */
+    public MetaDataMutationResult updateCache(String schemaName, String tableName) throws SQLException {
+        return updateCache(schemaName, tableName, false);
+    }
+    
+    private static final MetaDataMutationResult SYSTEM_TABLE_RESULT = new MetaDataMutationResult(MutationCode.TABLE_ALREADY_EXISTS,QueryConstants.UNSET_TIMESTAMP,null);
+    
+    private MetaDataMutationResult updateCache(String schemaName, String tableName, boolean alwaysHitServer) throws SQLException { // TODO: pass byte[] here
+        Long scn = connection.getSCN();
+        long clientTimeStamp = scn == null ? HConstants.LATEST_TIMESTAMP : scn;
+        if (TYPE_SCHEMA.equals(schemaName) && !alwaysHitServer) {
+            return SYSTEM_TABLE_RESULT;
+        }
+        PTable table = null;
+        String fullTableName = SchemaUtil.getTableName(schemaName, tableName);
+        long tableTimestamp = HConstants.LATEST_TIMESTAMP;
+        PName tenantIdName = connection.getTenantId();
+        try {
+            table = connection.getPMetaData().getTable(fullTableName);
+            tableTimestamp = table.getTimeStamp();
+        } catch (TableNotFoundException e) {
+            // TODO: Try again on services cache, as we may be looking for
+            // a global multi-tenant table
+        }
+        // Don't bother with server call: we can't possibly find a newer table
+        if (table != null && tableTimestamp == clientTimeStamp - 1 && !alwaysHitServer) {
+            return new MetaDataMutationResult(MutationCode.TABLE_ALREADY_EXISTS,QueryConstants.UNSET_TIMESTAMP,table);
+        }
+        
+        byte[] tenantId = null;
+        int maxTryCount = 1;
+        if (tenantIdName != null) {
+            tenantId = tenantIdName.getBytes();
+            maxTryCount = 2;
+        }
+        int tryCount = 0;
+        MetaDataMutationResult result;
+        
+        do {
+            final byte[] schemaBytes = PDataType.VARCHAR.toBytes(schemaName);
+            final byte[] tableBytes = PDataType.VARCHAR.toBytes(tableName);
+            result = connection.getQueryServices().getTable(tenantId, schemaBytes, tableBytes, tableTimestamp, clientTimeStamp);
+            
+            if (TYPE_SCHEMA.equals(schemaName)) {
+                return result;
+            }
+            MutationCode code = result.getMutationCode();
+            PTable resultTable = result.getTable();
+            // We found an updated table, so update our cache
+            if (resultTable != null) {
+                // Don't cache the table unless it has the same tenantId
+                // as the connection or it's not multi-tenant.
+                if (tryCount == 0 || !resultTable.isMultiTenant()) {
+                    connection.addTable(resultTable);
+                    return result;
+                }
+            } else {
+                // if (result.getMutationCode() == MutationCode.NEWER_TABLE_FOUND) {
+                // TODO: No table exists at the clientTimestamp, but a newer one exists.
+                // Since we disallow creation or modification of a table earlier than the latest
+                // timestamp, we can handle this such that we don't ask the
+                // server again.
+                // If table was not found at the current time stamp and we have one cached, remove it.
+                // Otherwise, we're up to date, so there's nothing to do.
+                if (table != null) {
+                    result.setTable(table);
+                    if (code == MutationCode.TABLE_ALREADY_EXISTS) {
+                        return result;
+                    }
+                    if (code == MutationCode.TABLE_NOT_FOUND && tryCount + 1 == maxTryCount) {
+                        connection.removeTable(fullTableName);
+                    }
+                }
+            }
+            tenantId = null;
+        } while (++tryCount < maxTryCount);
+        
+        return result;
+    }
+
+
+    private void addColumnMutation(String schemaName, String tableName, PColumn column, PreparedStatement colUpsert, String parentTableName) throws SQLException {
+        colUpsert.setString(1, connection.getTenantId() == null ? null : connection.getTenantId().getString());
+        colUpsert.setString(2, schemaName);
+        colUpsert.setString(3, tableName);
+        colUpsert.setString(4, column.getName().getString());
+        colUpsert.setString(5, column.getFamilyName() == null ? null : column.getFamilyName().getString());
+        colUpsert.setInt(6, column.getDataType().getSqlType());
+        colUpsert.setInt(7, column.isNullable() ? ResultSetMetaData.columnNullable : ResultSetMetaData.columnNoNulls);
+        if (column.getMaxLength() == null) {
+            colUpsert.setNull(8, Types.INTEGER);
+        } else {
+            colUpsert.setInt(8, column.getMaxLength());
+        }
+        if (column.getScale() == null) {
+            colUpsert.setNull(9, Types.INTEGER);
+        } else {
+            colUpsert.setInt(9, column.getScale());
+        }
+        colUpsert.setInt(10, column.getPosition() + 1);
+        colUpsert.setInt(11, ColumnModifier.toSystemValue(column.getColumnModifier()));
+        colUpsert.setString(12, parentTableName);
+        if (column.getArraySize() == null) {
+            colUpsert.setNull(13, Types.INTEGER);
+        } else {
+            colUpsert.setInt(13, column.getArraySize());
+        }
+        colUpsert.execute();
+    }
+
+    private PColumn newColumn(int position, ColumnDef def, PrimaryKeyConstraint pkConstraint) throws SQLException {
+        try {
+            ColumnName columnDefName = def.getColumnDefName();
+            ColumnModifier columnModifier = def.getColumnModifier();
+            boolean isPK = def.isPK();
+            if (pkConstraint != null) {
+                Pair<ColumnName,ColumnModifier> pkColumnModifier = pkConstraint.getColumn(columnDefName);
+                if (pkColumnModifier != null) {
+                    isPK = true;
+                    columnModifier = pkColumnModifier.getSecond();
+                }
+            }
+            
+            String columnName = columnDefName.getColumnName();
+            PName familyName = null;
+            if (def.isPK() && !pkConstraint.getColumnNames().isEmpty() ) {
+                throw new SQLExceptionInfo.Builder(SQLExceptionCode.PRIMARY_KEY_ALREADY_EXISTS)
+                    .setColumnName(columnName).build().buildException();
+            }
+            if (def.getColumnDefName().getFamilyName() != null) {
+                String family = def.getColumnDefName().getFamilyName();
+                if (isPK) {
+                    throw new SQLExceptionInfo.Builder(SQLExceptionCode.PRIMARY_KEY_WITH_FAMILY_NAME)
+                        .setColumnName(columnName).setFamilyName(family).build().buildException();
+                } else if (!def.isNull()) {
+                    throw new SQLExceptionInfo.Builder(SQLExceptionCode.KEY_VALUE_NOT_NULL)
+                        .setColumnName(columnName).setFamilyName(family).build().buildException();
+                }
+                familyName = PNameFactory.newName(family);
+            } else if (!isPK) {
+                familyName = PNameFactory.newName(QueryConstants.DEFAULT_COLUMN_FAMILY);
+            }
+            
+            PColumn column = new PColumnImpl(PNameFactory.newName(columnName), familyName, def.getDataType(),
+                    def.getMaxLength(), def.getScale(), def.isNull(), position, columnModifier, def.getArraySize());
+            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) throws SQLException {
+        PTable table = createTableInternal(statement, splits, parent, viewStatement, viewType);
+        if (table == null || table.getType() == PTableType.VIEW) {
+            return new MutationState(0,connection);
+        }
+        
+        // Hack to get around the case when an SCN is specified on the connection.
+        // In this case, we won't see the table we just created yet, so we hack
+        // around it by forcing the compiler to not resolve anything.
+        PostDDLCompiler compiler = new PostDDLCompiler(connection);
+        //connection.setAutoCommit(true);
+        // Execute any necessary data updates
+        Long scn = connection.getSCN();
+        long ts = (scn == null ? table.getTimeStamp() : scn);
+        // Getting the schema through the current connection doesn't work when the connection has an scn specified
+        // Since the table won't be added to the current connection.
+        TableRef tableRef = new TableRef(null, table, ts, false);
+        byte[] emptyCF = SchemaUtil.getEmptyColumnFamily(table.getColumnFamilies());
+        MutationPlan plan = compiler.compile(Collections.singletonList(tableRef), emptyCF, null, null, tableRef.getTimeStamp());
+        return connection.getQueryServices().updateData(plan);
+    }
+
+    private MutationState buildIndexAtTimeStamp(PTable index, NamedTableNode dataTableNode) throws SQLException {
+        // If our connection is at a fixed point-in-time, we need to open a new
+        // connection so that our new index table is visible.
+        Properties props = new Properties(connection.getClientInfo());
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(connection.getSCN()+1));
+        PhoenixConnection conn = DriverManager.getConnection(connection.getURL(), props).unwrap(PhoenixConnection.class);
+        MetaDataClient newClientAtNextTimeStamp = new MetaDataClient(conn);
+        
+        // Re-resolve the tableRef from the now newer connection
+        conn.setAutoCommit(true);
+        ColumnResolver resolver = FromCompiler.getResolver(dataTableNode, conn);
+        TableRef tableRef = resolver.getTables().get(0);
+        boolean success = false;
+        SQLException sqlException = null;
+        try {
+            MutationState state = newClientAtNextTimeStamp.buildIndex(index, tableRef);
+            success = true;
+            return state;
+        } catch (SQLException e) {
+            sqlException = e;
+        } finally {
+            try {
+                conn.close();
+            } catch (SQLException e) {
+                if (sqlException == null) {
+                    // If we're not in the middle of throwing another exception
+                    // then throw the exception we got on close.
+                    if (success) {
+                        sqlException = e;
+                    }
+                } else {
+                    sqlException.setNextException(e);
+                }
+            }
+            if (sqlException != null) {
+                throw sqlException;
+            }
+        }
+        throw new IllegalStateException(); // impossible
+    }
+    
+    private MutationState buildIndex(PTable index, TableRef dataTableRef) throws SQLException {
+        PostIndexDDLCompiler compiler = new PostIndexDDLCompiler(connection, dataTableRef);
+        MutationPlan plan = compiler.compile(index);
+        MutationState state = connection.getQueryServices().updateData(plan);
+        AlterIndexStatement indexStatement = FACTORY.alterIndex(FACTORY.namedTable(null, 
+                TableName.create(index.getSchemaName().getString(), index.getTableName().getString())),
+                dataTableRef.getTable().getTableName().getString(), false, PIndexState.ACTIVE);
+        alterIndex(indexStatement);
+        return state;
+    }
+
+    /**
+     * Create an index table by morphing the CreateIndexStatement into a CreateTableStatement and calling
+     * MetaDataClient.createTable. In doing so, we perform the following translations:
+     * 1) Change the type of any columns being indexed to types that support null if the column is nullable.
+     *    For example, a BIGINT type would be coerced to a DECIMAL type, since a DECIMAL type supports null
+     *    when it's in the row key while a BIGINT does not.
+     * 2) Append any row key column from the data table that is not in the indexed column list. Our indexes
+     *    rely on having a 1:1 correspondence between the index and data rows.
+     * 3) Change the name of the columns to include the column family. For example, if you have a column
+     *    named "B" in a column family named "A", the indexed column name will be "A:B". This makes it easy
+     *    to translate the column references in a query to the correct column references in an index table
+     *    regardless of whether the column reference is prefixed with the column family name or not. It also
+     *    has the side benefit of allowing the same named column in different column families to both be
+     *    listed as an index column.
+     * @param statement
+     * @param splits
+     * @return MutationState from population of index table from data table
+     * @throws SQLException
+     */
+    public MutationState createIndex(CreateIndexStatement statement, byte[][] splits) throws SQLException {
+        PrimaryKeyConstraint pk = statement.getIndexConstraint();
+        TableName indexTableName = statement.getIndexTableName();
+        
+        List<Pair<ColumnName, ColumnModifier>> indexedPkColumns = pk.getColumnNames();
+        List<ColumnName> includedColumns = statement.getIncludeColumns();
+        TableRef tableRef = null;
+        PTable table = null;
+        boolean retry = true;
+        while (true) {
+            try {
+                ColumnResolver resolver = FromCompiler.getResolver(statement, connection);
+                tableRef = resolver.getTables().get(0);
+                PTable dataTable = tableRef.getTable();
+                int hbaseVersion = connection.getQueryServices().getLowestClusterHBaseVersion();
+                if (!dataTable.isImmutableRows()) {
+                    if (hbaseVersion < PhoenixDatabaseMetaData.MUTABLE_SI_VERSION_THRESHOLD) {
+                        throw new SQLExceptionInfo.Builder(SQLExceptionCode.NO_MUTABLE_INDEXES).setTableName(indexTableName.getTableName()).build().buildException();
+                    }
+                    if (connection.getQueryServices().hasInvalidIndexConfiguration()) {
+                        throw new SQLExceptionInfo.Builder(SQLExceptionCode.INVALID_MUTABLE_INDEX_CONFIG).setTableName(indexTableName.getTableName()).build().buildException();
+                    }
+                }
+                Set<PColumn> unusedPkColumns;
+                if (dataTable.getBucketNum() != null) { // Ignore SALT column
+                    unusedPkColumns = new LinkedHashSet<PColumn>(dataTable.getPKColumns().subList(1, dataTable.getPKColumns().size()));
+                } else {
+                    unusedPkColumns = new LinkedHashSet<PColumn>(dataTable.getPKColumns());
+                }
+                List<Pair<ColumnName, ColumnModifier>> allPkColumns = Lists.newArrayListWithExpectedSize(unusedPkColumns.size());
+                List<ColumnDef> columnDefs = Lists.newArrayListWithExpectedSize(includedColumns.size() + indexedPkColumns.size());
+                
+                // First columns are the indexed ones
+                for (Pair<ColumnName, ColumnModifier> pair : indexedPkColumns) {
+                    ColumnName colName = pair.getFirst();
+                    PColumn col = resolver.resolveColumn(null, colName.getFamilyName(), colName.getColumnName()).getColumn();
+                    unusedPkColumns.remove(col);
+                    PDataType dataType = IndexUtil.getIndexColumnDataType(col);
+                    colName = ColumnName.caseSensitiveColumnName(IndexUtil.getIndexColumnName(col));
+                    allPkColumns.add(new Pair<ColumnName, ColumnModifier>(colName, pair.getSecond()));
+                    columnDefs.add(FACTORY.columnDef(colName, dataType.getSqlTypeName(), col.isNullable(), col.getMaxLength(), col.getScale(), false, null));
+                }
+                
+                // Next all the PK columns from the data table that aren't indexed
+                if (!unusedPkColumns.isEmpty()) {
+                    for (PColumn col : unusedPkColumns) {
+                        ColumnName colName = ColumnName.caseSensitiveColumnName(IndexUtil.getIndexColumnName(col));
+                        allPkColumns.add(new Pair<ColumnName, ColumnModifier>(colName, col.getColumnModifier()));
+                        PDataType dataType = IndexUtil.getIndexColumnDataType(col);
+                        columnDefs.add(FACTORY.columnDef(colName, dataType.getSqlTypeName(), col.isNullable(), col.getMaxLength(), col.getScale(), false, col.getColumnModifier()));
+                    }
+                }
+                pk = FACTORY.primaryKey(null, allPkColumns);
+                
+                // Last all the included columns (minus any PK columns)
+                for (ColumnName colName : includedColumns) {
+                    PColumn col = resolver.resolveColumn(null, colName.getFamilyName(), colName.getColumnName()).getColumn();
+                    if (SchemaUtil.isPKColumn(col)) {
+                        if (!unusedPkColumns.contains(col)) {
+                            throw new SQLExceptionInfo.Builder(SQLExceptionCode.COLUMN_EXIST_IN_DEF).build().buildException();
+                        }
+                    } else {
+                        colName = ColumnName.caseSensitiveColumnName(IndexUtil.getIndexColumnName(col));
+                        // Check for duplicates between indexed and included columns
+                        if (pk.contains(colName)) {
+                            throw new SQLExceptionInfo.Builder(SQLExceptionCode.COLUMN_EXIST_IN_DEF).build().buildException();
+                        }
+                        if (!SchemaUtil.isPKColumn(col)) {
+                            // Need to re-create ColumnName, since the above one won't have the column family name
+                            colName = ColumnName.caseSensitiveColumnName(col.getFamilyName().getString(), IndexUtil.getIndexColumnName(col));
+                            columnDefs.add(FACTORY.columnDef(colName, col.getDataType().getSqlTypeName(), col.isNullable(), col.getMaxLength(), col.getScale(), false, col.getColumnModifier()));
+                        }
+                    }
+                }
+                
+                CreateTableStatement tableStatement = FACTORY.createTable(indexTableName, statement.getProps(), columnDefs, pk, statement.getSplitNodes(), PTableType.INDEX, statement.ifNotExists(), null, null, statement.getBindCount());
+                table = createTableInternal(tableStatement, splits, tableRef.getTable(), null, null); // TODO: tenant-specific index
+                break;
+            } catch (ConcurrentTableMutationException e) { // Can happen if parent data table changes while above is in progress
+                if (retry) {
+                    retry = false;
+                    continue;
+                }
+                throw e;
+            }
+        }
+        if (table == null) {
+            return new MutationState(0,connection);
+        }
+        // If our connection is at a fixed point-in-time, we need to open a new
+        // connection so that our new index table is visible.
+        if (connection.getSCN() != null) {
+            return buildIndexAtTimeStamp(table, statement.getTable());
+        }
+        
+        return buildIndex(table, tableRef);
+    }
+
+    public MutationState dropSequence(DropSequenceStatement statement) throws SQLException {
+        Long scn = connection.getSCN();
+        long timestamp = scn == null ? HConstants.LATEST_TIMESTAMP : scn;
+        String schemaName = statement.getSequenceName().getSchemaName();
+        String sequenceName = statement.getSequenceName().getTableName();
+        String tenantId = connection.getTenantId() == null ? null : connection.getTenantId().getString();
+        try {
+            connection.getQueryServices().dropSequence(tenantId, schemaName, sequenceName, timestamp);
+        } catch (SequenceNotFoundException e) {
+            if (statement.ifExists()) {
+                return new MutationState(0, connection);
+            }
+            throw e;
+        }
+        return new MutationState(1, connection);
+    }
+    
+    public MutationState createSequence(CreateSequenceStatement statement, long startWith, long incrementBy, int cacheSize) throws SQLException {
+        Long scn = connection.getSCN();
+        long timestamp = scn == null ? HConstants.LATEST_TIMESTAMP : scn;
+        String schemaName = statement.getSequenceName().getSchemaName();
+        String sequenceName = statement.getSequenceName().getTableName();
+        String tenantId = connection.getTenantId() == null ? null : connection.getTenantId().getString();
+        try {
+            connection.getQueryServices().createSequence(tenantId, schemaName, sequenceName, startWith, incrementBy, cacheSize, timestamp);
+        } catch (SequenceAlreadyExistsException e) {
+            if (statement.ifNotExists()) {
+                return new MutationState(0, connection);
+            }
+            throw e;
+        }
+        return new MutationState(1, connection);
+    }
+    
+    private static ColumnDef findColumnDefOrNull(List<ColumnDef> colDefs, ColumnName colName) {
+        for (ColumnDef colDef : colDefs) {
+            if (colDef.getColumnDefName().getColumnName().equals(colName.getColumnName())) {
+                return colDef;
+            }
+        }
+        return null;
+    }
+    
+    private PTable createTableInternal(CreateTableStatement statement, byte[][] splits, final PTable parent, String viewStatement, ViewType viewType) throws SQLException {
+        final PTableType tableType = statement.getTableType();
+        boolean wasAutoCommit = connection.getAutoCommit();
+        connection.rollback();
+        try {
+            connection.setAutoCommit(false);
+            List<Mutation> tableMetaData = Lists.newArrayListWithExpectedSize(statement.getColumnDefs().size() + 3);
+            
+            TableName tableNameNode = statement.getTableName();
+            String schemaName = tableNameNode.getSchemaName();
+            String tableName = tableNameNode.getTableName();
+            String parentTableName = null;
+            String tenantId = connection.getTenantId() == null ? null : connection.getTenantId().getString();
+            boolean isParentImmutableRows = false;
+            if (parent != null && tableType == PTableType.INDEX) {
+                isParentImmutableRows = parent.isImmutableRows();
+                parentTableName = parent.getTableName().getString();
+                // Pass through data table sequence number so we can check it hasn't changed
+                PreparedStatement incrementStatement = connection.prepareStatement(INCREMENT_SEQ_NUM);
+                incrementStatement.setString(1, connection.getTenantId() == null ? null : connection.getTenantId().getString());
+                incrementStatement.setString(2, schemaName);
+                incrementStatement.setString(3, parentTableName);
+                incrementStatement.setLong(4, parent.getSequenceNumber());
+                incrementStatement.execute();
+                // Get list of mutations and add to table meta data that will be passed to server
+                // to guarantee order. This row will always end up last
+                tableMetaData.addAll(connection.getMutationState().toMutations().next().getSecond());
+                connection.rollback();
+
+                // Add row linking from data table row to index table row
+                PreparedStatement linkStatement = connection.prepareStatement(CREATE_LINK);
+                linkStatement.setString(1, connection.getTenantId() == null ? null : connection.getTenantId().getString());
+                linkStatement.setString(2, schemaName);
+                linkStatement.setString(3, parentTableName);
+                linkStatement.setString(4, tableName);
+                linkStatement.setByte(5, LinkType.INDEX_TABLE.getSerializedValue());
+                linkStatement.execute();
+            }
+            
+            PrimaryKeyConstraint pkConstraint = statement.getPrimaryKeyConstraint();
+            String pkName = null;
+            List<Pair<ColumnName,ColumnModifier>> pkColumnsNames = Collections.<Pair<ColumnName,ColumnModifier>>emptyList();
+            Iterator<Pair<ColumnName,ColumnModifier>> pkColumnsIterator = Iterators.emptyIterator();
+            if (pkConstraint != null) {
+                pkColumnsNames = pkConstraint.getColumnNames();
+                pkColumnsIterator = pkColumnsNames.iterator();
+                pkName = pkConstraint.getName();
+            }
+            
+            Map<String,Object> tableProps = Maps.newHashMapWithExpectedSize(statement.getProps().size());
+            Map<String,Object> commonFamilyProps = Collections.emptyMap();
+            // Somewhat hacky way of determining if property is for HColumnDescriptor or HTableDescriptor
+            HColumnDescriptor defaultDescriptor = new HColumnDescriptor(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES);
+            if (!statement.getProps().isEmpty()) {
+                commonFamilyProps = Maps.newHashMapWithExpectedSize(statement.getProps().size());
+                
+                Collection<Pair<String,Object>> props = statement.getProps().get(QueryConstants.ALL_FAMILY_PROPERTIES_KEY);
+                for (Pair<String,Object> prop : props) {
+                    if (defaultDescriptor.getValue(prop.getFirst()) == null) {
+                        tableProps.put(prop.getFirst(), prop.getSecond());
+                    } else {
+                        commonFamilyProps.put(prop.getFirst(), prop.getSecond());
+                    }
+                }
+            }
+            
+            boolean isSalted = false;
+            Integer saltBucketNum = null;
+            String defaultFamilyName = null;
+            boolean isImmutableRows = false;
+            boolean multiTenant = false;
+            // Although unusual, it's possible to set a mapped VIEW as having immutable rows.
+            // This tells Phoenix that you're managing the index maintenance yourself.
+            if (tableType != PTableType.VIEW || viewType == ViewType.MAPPED) {
+                Boolean isImmutableRowsProp = (Boolean) tableProps.remove(PTable.IS_IMMUTABLE_ROWS_PROP_NAME);
+                if (isImmutableRowsProp == null) {
+                    isImmutableRows = connection.getQueryServices().getProps().getBoolean(QueryServices.IMMUTABLE_ROWS_ATTRIB, QueryServicesOptions.DEFAULT_IMMUTABLE_ROWS);
+                } else {
+                    isImmutableRows = isImmutableRowsProp;
+                }
+            }
+            
+            if (tableType != PTableType.VIEW) {
+                saltBucketNum = (Integer) tableProps.remove(PhoenixDatabaseMetaData.SALT_BUCKETS);
+                if (saltBucketNum != null && (saltBucketNum < 0 || saltBucketNum > SaltingUtil.MAX_BUCKET_NUM)) {
+                    throw new SQLExceptionInfo.Builder(SQLExceptionCode.INVALID_BUCKET_NUM).build().buildException();
+                }
+                // Salt the index table if the data table is salted
+                if (saltBucketNum == null) {
+                    if (parent != null) {
+                        saltBucketNum = parent.getBucketNum();
+                    }
+                } else if (saltBucketNum.intValue() == 0) {
+                    saltBucketNum = null; // Provides a way for an index to not be salted if its data table is salted
+                }
+                isSalted = (saltBucketNum != null);
+                
+                defaultFamilyName = (String)tableProps.remove(PhoenixDatabaseMetaData.DEFAULT_COLUMN_FAMILY_NAME);
+                if (defaultFamilyName == null) {
+                    // Until we change this default, we need to have all tables set it
+                    defaultFamilyName = QueryConstants.DEFAULT_COLUMN_FAMILY;
+                }
+                
+                Boolean multiTenantProp = (Boolean) tableProps.remove(PhoenixDatabaseMetaData.MULTI_TENANT);
+                multiTenant = Boolean.TRUE.equals(multiTenantProp);
+            }
+            
+            boolean disableWAL = false;
+            Boolean disableWALProp = (Boolean) tableProps.remove(PhoenixDatabaseMetaData.DISABLE_WAL);
+            if (disableWALProp == null) {
+                disableWAL = isParentImmutableRows; // By default, disable WAL for immutable indexes
+            } else {
+                disableWAL = disableWALProp;
+            }
+            // Delay this check as it is supported to have IMMUTABLE_ROWS and SALT_BUCKETS defined on views
+            if (statement.getTableType() == PTableType.VIEW && !tableProps.isEmpty()) {
+                throw new SQLExceptionInfo.Builder(SQLExceptionCode.VIEW_WITH_PROPERTIES).build().buildException();
+            }
+            
+            List<ColumnDef> colDefs = statement.getColumnDefs();
+            List<PColumn> columns;
+            LinkedHashSet<PColumn> pkColumns;    
+            
+            if (tenantId != null && tableType != PTableType.VIEW) {
+                throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_CREATE_TENANT_SPECIFIC_TABLE)
+                    .setSchemaName(schemaName).setTableName(tableName).build().buildException();
+            }
+            
+            List<PName> physicalNames = Collections.emptyList();
+            if (tableType == PTableType.VIEW) {
+                physicalNames = Collections.singletonList(PNameFactory.newName(parent.getPhysicalName().getString()));
+                if (viewType == ViewType.MAPPED) {
+                    columns = newArrayListWithExpectedSize(colDefs.size());
+                    pkColumns = newLinkedHashSetWithExpectedSize(colDefs.size());  
+                } else {
+                    // Propagate property values to VIEW.
+                    // TODO: formalize the known set of these properties
+                    multiTenant = parent.isMultiTenant();
+                    saltBucketNum = parent.getBucketNum();
+                    isImmutableRows = parent.isImmutableRows();
+                    disableWAL = (disableWALProp == null ? parent.isWALDisabled() : disableWALProp);
+                    defaultFamilyName = parent.getDefaultFamilyName() == null ? null : parent.getDefaultFamilyName().getString();
+                    columns = newArrayListWithExpectedSize(parent.getColumns().size() + colDefs.size());
+                    columns.addAll(parent.getColumns());
+                    pkColumns = newLinkedHashSet(parent.getPKColumns());
+
+                    // Add row linking from data table row to physical table row
+                    PreparedStatement linkStatement = connection.prepareStatement(CREATE_LINK);
+                    for (PName physicalName : physicalNames) {
+                        linkStatement.setString(1, connection.getTenantId() == null ? null : connection.getTenantId().getString());
+                        linkStatement.setString(2, schemaName);
+                        linkStatement.setString(3, tableName);
+                        linkStatement.setString(4, physicalName.getString());
+                        linkStatement.setByte(5, LinkType.PHYSICAL_TABLE.getSerializedValue());
+                        linkStatement.execute();
+                    }
+                }
+            } else {
+                columns = newArrayListWithExpectedSize(colDefs.size());
+                pkColumns = newLinkedHashSetWithExpectedSize(colDefs.size() + 1); // in case salted  
+            }
+            
+            PreparedStatement colUpsert = connection.prepareStatement(INSERT_COLUMN);
+            Map<String, PName> familyNames = Maps.newLinkedHashMap();
+            boolean isPK = false;
+            
+            int positionOffset = columns.size();
+            if (isSalted) {
+                positionOffset++;
+                pkColumns.add(SaltingUtil.SALTING_COLUMN);
+            }
+            int position = positionOffset;
+            
+            for (ColumnDef colDef : colDefs) {
+                if (colDef.isPK()) {
+                    if (isPK) {
+                        throw new SQLExceptionInfo.Builder(SQLExceptionCode.PRIMARY_KEY_ALREADY_EXISTS)
+                            .setColumnName(colDef.getColumnDefName().getColumnName()).build().buildException();
+                    }
+                    isPK = true;
+                }
+                PColumn column = newColumn(position++, colDef, pkConstraint);
+                if (SchemaUtil.isPKColumn(column)) {
+                    // TODO: remove this constraint?
+                    if (pkColumnsIterator.hasNext() && !column.getName().getString().equals(pkColumnsIterator.next().getFirst().getColumnName())) {
+                        throw new SQLExceptionInfo.Builder(SQLExceptionCode.PRIMARY_KEY_OUT_OF_ORDER)
+                            .setSchemaName(schemaName)
+                            .setTableName(tableName)
+                            .setColumnName(column.getName().getString())
+                            .build().buildException();
+                    }
+                    if (tableType == PTableType.VIEW && viewType != ViewType.MAPPED) {
+                        throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_DEFINE_PK_FOR_VIEW)
+                            .setSchemaName(schemaName)
+                            .setTableName(tableName)
+                            .setColumnName(colDef.getColumnDefName().getColumnName())
+                            .build().buildException();
+                    }
+                    // disallow array type usage in primary key constraint
+                    if (colDef.isArray()) {
+                        throw new SQLExceptionInfo.Builder(SQLExceptionCode.ARRAY_NOT_ALLOWED_IN_PRIMARY_KEY)
+                        .setSchemaName(schemaName)
+                        .setTableName(tableName)
+                        .setColumnName(colDef.getColumnDefName().getColumnName())
+                        .build().buildException();
+                    }
+                    if (!pkColumns.add(column)) {
+                        throw new ColumnAlreadyExistsException(schemaName, tableName, column.getName().getString());
+                    }
+                }
+                if (tableType == PTableType.VIEW && hasColumnWithSameNameAndFamily(columns, column)) {
+                    // we only need to check for dup columns for views because they inherit columns from parent
+                    throw new ColumnAlreadyExistsException(schemaName, tableName, column.getName().getString());
+                }
+                columns.add(column);
+                if (colDef.getDataType() == PDataType.VARBINARY 
+                        && SchemaUtil.isPKColumn(column)
+                        && pkColumnsNames.size() > 1 
+                        && column.getPosition() < pkColumnsNames.size() - 1) {
+                    throw new SQLExceptionInfo.Builder(SQLExceptionCode.VARBINARY_IN_ROW_KEY)
+                        .setSchemaName(schemaName)
+                        .setTableName(tableName)
+                        .setColumnName(column.getName().getString())
+                        .build().buildException();
+                }
+                if (column.getFamilyName() != null) {
+                    familyNames.put(column.getFamilyName().getString(),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)
+                    .setSchemaName(schemaName)
+                    .setTableName(tableName)
+                    .build().buildException();
+            }
+            if (!pkColumnsNames.isEmpty() && pkColumnsNames.size() != pkColumns.size() - positionOffset) { // Then a column name in the primary key constraint wasn't resolved
+                Iterator<Pair<ColumnName,ColumnModifier>> pkColumnNamesIterator = pkColumnsNames.iterator();
+                while (pkColumnNamesIterator.hasNext()) {
+                    ColumnName colName = pkColumnNamesIterator.next().getFirst();
+                    ColumnDef colDef = findColumnDefOrNull(colDefs, colName);
+                    if (colDef == null) {
+                        throw new ColumnNotFoundException(schemaName, tableName, null, colName.getColumnName());
+                    }
+                    if (colDef.getColumnDefName().getFamilyName() != null) {
+                        throw new SQLExceptionInfo.Builder(SQLExceptionCode.PRIMARY_KEY_WITH_FAMILY_NAME)
+                        .setSchemaName(schemaName)
+                        .setTableName(tableName)
+                        .setColumnName(colDef.getColumnDefName().getColumnName() )
+                        .setFamilyName(colDef.getColumnDefName().getFamilyName())
+                        .build().buildException();
+                    }
+                }
+                // The above should actually find the specific one, but just in case...
+                throw new SQLExceptionInfo.Builder(SQLExceptionCode.INVALID_PRIMARY_KEY_CONSTRAINT)
+                    .setSchemaName(schemaName)
+                    .setTableName(tableName)
+                    .build().buildException();
+            }
+            
+            List<Pair<byte[],Map<String,Object>>> familyPropList = Lists.newArrayListWithExpectedSize(familyNames.size());
+            if (!statement.getProps().isEmpty()) {
+                for (String familyName : statement.getProps().keySet()) {
+                    if (!familyName.equals(QueryConstants.ALL_FAMILY_PROPERTIES_KEY)) {
+                        if (familyNames.get(familyName) == null) {
+                            throw new SQLExceptionInfo.Builder(SQLExceptionCode.PROPERTIES_FOR_FAMILY)
+                                .setFamilyName(familyName).build().buildException();
+                        } else if (statement.getTableType() == PTableType.VIEW) {
+                            throw new SQLExceptionInfo.Builder(SQLExceptionCode.VIEW_WITH_PROPERTIES).build().buildException();
+                        }
+                    }
+                }
+            }
+            throwIfInsufficientColumns(schemaName, tableName, pkColumns, isSalted, multiTenant);
+            
+            for (PName familyName : familyNames.values()) {
+                Collection<Pair<String,Object>> props = statement.getProps().get(familyName.getString());
+                if (props.isEmpty()) {
+                    familyPropList.add(new Pair<byte[],Map<String,Object>>(familyName.getBytes(),commonFamilyProps));
+                } else {
+                    Map<String,Object> combinedFamilyProps = Maps.newHashMapWithExpectedSize(props.size() + commonFamilyProps.size());
+                    combinedFamilyProps.putAll(commonFamilyProps);
+                    for (Pair<String,Object> prop : props) {
+                        combinedFamilyProps.put(prop.getFirst(), prop.getSecond());
+                    }
+                    familyPropList.add(new Pair<byte[],Map<String,Object>>(familyName.getBytes(),combinedFamilyProps));
+                }
+            }
+            
+            
+            // Bootstrapping for our SYSTEM.TABLE that creates itself before it exists 
+            if (SchemaUtil.isMetaTable(schemaName,tableName)) {
+                PTable table = PTableImpl.makePTable(PNameFactory.newName(schemaName),PNameFactory.newName(tableName), tableType, null,
+                        MetaDataProtocol.MIN_TABLE_TIMESTAMP, PTable.INITIAL_SEQ_NUM, PNameFactory.newName(QueryConstants.SYSTEM_TABLE_PK_NAME),
+                        null, columns, null, Collections.<PTable>emptyList(), isImmutableRows, 
+                        Collections.<PName>emptyList(), defaultFamilyName == null ? null : PNameFactory.newName(defaultFamilyName),
+                        null, Boolean.TRUE.equals(disableWAL), false, null);
+                connection.addTable(table);
+            } else if (tableType == PTableType.INDEX) {
+                if (tableProps.get(HTableDescriptor.MAX_FILESIZE) == null) {
+                    int nIndexRowKeyColumns = isPK ? 1 : pkColumnsNames.size();
+                    int nIndexKeyValueColumns = columns.size() - nIndexRowKeyColumns;
+                    int nBaseRowKeyColumns = parent.getPKColumns().size() - (parent.getBucketNum() == null ? 0 : 1);
+                    int nBaseKeyValueColumns = parent.getColumns().size() - parent.getPKColumns().size();
+                    /* 
+                     * Approximate ratio between index table size and data table size:
+                     * More or less equal to the ratio between the number of key value columns in each. We add one to
+                     * the key value column count to take into account our empty key value. We add 1/4 for any key
+                     * value data table column that was moved into the index table row key.
+                     */
+                    double ratio = (1+nIndexKeyValueColumns + (nIndexRowKeyColumns - nBaseRowKeyColumns)/4d)/(1+nBaseKeyValueColumns);
+                    HTableDescriptor descriptor = connection.getQueryServices().getTableDescriptor(parent.getPhysicalName().getBytes());
+                    if (descriptor != null) { // Is null for connectionless
+                        long maxFileSize = descriptor.getMaxFileSize();
+                        if (maxFileSize == -1) { // If unset, use default
+                            maxFileSize = HConstants.DEFAULT_MAX_FILE_SIZE;
+                        }
+                        tableProps.put(HTableDescriptor.MAX_FILESIZE, (long)(maxFileSize * ratio));
+                    }
+                }
+            }
+            
+            for (PColumn column : columns) {
+                addColumnMutation(schemaName, tableName, column, colUpsert, parentTableName);
+            }
+            
+            tableMetaData.addAll(connection.getMutationState().toMutations().next().getSecond());
+            connection.rollback();
+            
+            String dataTableName = parent == null || tableType == PTableType.VIEW ? null : parent.getTableName().getString();
+            PIndexState indexState = parent == null || tableType == PTableType.VIEW  ? null : PIndexState.BUILDING;
+            PreparedStatement tableUpsert = connection.prepareStatement(CREATE_TABLE);
+            tableUpsert.setString(1, tenantId);
+            tableUpsert.setString(2, schemaName);
+            tableUpsert.setString(3, tableName);
+            tableUpsert.setString(4, tableType.getSerializedValue());
+            tableUpsert.setLong(5, PTable.INITIAL_SEQ_NUM);
+            tableUpsert.setInt(6, position);
+            if (saltBucketNum != null) {
+                tableUpsert.setInt(7, saltBucketNum);
+            } else {
+                tableUpsert.setNull(7, Types.INTEGER);
+            }
+            tableUpsert.setString(8, pkName);
+            tableUpsert.setString(9, dataTableName);
+            tableUpsert.setString(10, indexState == null ? null : indexState.getSerializedValue());
+            tableUpsert.setBoolean(11, isImmutableRows);
+            tableUpsert.setString(12, defaultFamilyName);
+            tableUpsert.setString(13, viewStatement);
+            tableUpsert.setBoolean(14, disableWAL);
+            tableUpsert.setBoolean(15, multiTenant);
+            if (viewType == null) {
+                tableUpsert.setNull(16, Types.TINYINT);
+            } else {
+                tableUpsert.setByte(16, viewType.getSerializedValue());
+            }
+            tableUpsert.execute();
+            
+            tableMetaData.addAll(connection.getMutationState().toMutations().next().getSecond());
+            connection.rollback();
+            
+            /*
+             * The table metadata must be in the following order:
+             * 1) table header row
+             * 2) everything else
+             * 3) parent table header row
+             */
+            Collections.reverse(tableMetaData);
+            
+            splits = SchemaUtil.processSplits(splits, pkColumns, saltBucketNum, connection.getQueryServices().getProps().getBoolean(
+                    QueryServices.ROW_KEY_ORDER_SALTED_TABLE_ATTRIB, QueryServicesOptions.DEFAULT_ROW_KEY_ORDER_SALTED_TABLE));
+            MetaDataMutationResult result = connection.getQueryServices().createTable(
+                    tableMetaData, 
+                    viewType == ViewType.MAPPED ? physicalNames.get(0).getBytes() : null,
+                    tableType, tableProps, familyPropList, splits);
+            MutationCode code = result.getMutationCode();
+            switch(code) {
+            case TABLE_ALREADY_EXISTS:
+                connection.addTable(result.getTable());
+                if (!statement.ifNotExists()) {
+                    throw new TableAlreadyExistsException(schemaName, tableName);
+                }
+                return null;
+            case PARENT_TABLE_NOT_FOUND:
+                throw new TableNotFoundException(schemaName, parent.getName().getString());
+            case NEWER_TABLE_FOUND:
+                throw new NewerTableAlreadyExistsException(schemaName, tableName);
+            case UNALLOWED_TABLE_MUTATION:
+                throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_MUTATE_TABLE)
+                    .setSchemaName(schemaName).setTableName(tableName).build().buildException();
+            case CONCURRENT_TABLE_MUTATION:
+                connection.addTable(result.getTable());
+                throw new ConcurrentTableMutationException(schemaName, tableName);
+            default:
+                PTable table =  PTableImpl.makePTable(
+                        PNameFactory.newName(schemaName), PNameFactory.newName(tableName), tableType, indexState, result.getMutationTime(), PTable.INITIAL_SEQ_NUM, 
+                        pkName == null ? null : PNameFactory.newName(pkName), saltBucketNum, columns, dataTableName == null ? null : PNameFactory.newName(dataTableName), 
+                        Collections.<PTable>emptyList(), isImmutableRows, physicalNames,
+                        defaultFamilyName == null ? null : PNameFactory.newName(defaultFamilyName),
+                                viewStatement, Boolean.TRUE.equals(disableWAL), multiTenant, viewType);
+                connection.addTable(table);
+                return table;
+            }
+        } finally {
+            connection.setAutoCommit(wasAutoCommit);
+        }
+    }
+    
+    private static boolean hasColumnWithSameNameAndFamily(Collection<PColumn> columns, PColumn column) {
+        for (PColumn currColumn : columns) {
+           if (Objects.equal(currColumn.getFamilyName(), column.getFamilyName()) &&
+               Objects.equal(currColumn.getName(), column.getName())) {
+               return true;
+           }
+        }
+        return false;
+    }
+    
+    /**
+     * A table can be a parent table to tenant-specific tables if all of the following conditions are true:
+     * <p>
+     * FOR TENANT-SPECIFIC TABLES WITH TENANT_TYPE_ID SPECIFIED:
+     * <ol>
+     * <li>It has 3 or more PK columns AND
+     * <li>First PK (tenant id) column is not nullible AND 
+     * <li>Firsts PK column's data type is either VARCHAR or CHAR AND
+     * <li>Second PK (tenant type id) column is not nullible AND
+     * <li>Second PK column data type is either VARCHAR or CHAR
+     * </ol>
+     * FOR TENANT-SPECIFIC TABLES WITH NO TENANT_TYPE_ID SPECIFIED:
+     * <ol>
+     * <li>It has 2 or more PK columns AND
+     * <li>First PK (tenant id) column is not nullible AND 
+     * <li>Firsts PK column's data type is either VARCHAR or CHAR
+     * </ol>
+     */
+    private static void throwIfInsufficientColumns(String schemaName, String tableName, Collection<PColumn> columns, boolean isSalted, boolean isMultiTenant) throws SQLException {
+        if (!isMultiTenant) {
+            return;
+        }
+        int nPKColumns = columns.size() - (isSalted ? 1 : 0);
+        if (nPKColumns < 2) {
+            throw new SQLExceptionInfo.Builder(INSUFFICIENT_MULTI_TENANT_COLUMNS).setSchemaName(schemaName).setTableName(tableName).build().buildException();
+        }
+        Iterator<PColumn> iterator = columns.iterator();
+        if (isSalted) {
+            iterator.next();
+        }
+        PColumn tenantIdCol = iterator.next();
+        if (!tenantIdCol.getDataType().isCoercibleTo(VARCHAR)) {
+            throw new SQLExceptionInfo.Builder(INSUFFICIENT_MULTI_TENANT_COLUMNS).setSchemaName(schemaName).setTableName(tableName).build().buildException();
+        }
+    }
+
+    public MutationState dropTable(DropTableStatement statement) throws SQLException {
+        String schemaName = statement.getTableName().getSchemaName();
+        String tableName = statement.getTableName().getTableName();
+        return dropTable(schemaName, tableName, null, statement.getTableType(), statement.ifExists());
+    }
+
+    public MutationState dropIndex(DropIndexStatement statement) throws SQLException {
+        String schemaName = statement.getTableName().getSchemaName();
+        String tableName = statement.getIndexName().getName();
+        String parentTableName = statement.getTableName().getTableName();
+        return dropTable(schemaName, tableName, parentTableName, PTableType.INDEX, statement.ifExists());
+    }
+
+    private MutationState dropTable(String schemaName, String tableName, String parentTableName, PTableType tableType, boolean ifExists) throws SQLException {
+        connection.rollback();
+        boolean wasAutoCommit = connection.getAutoCommit();
+        try {
+            String tenantId = connection.getTenantId() == null ? null : connection.getTenantId().getString();
+            byte[] key = SchemaUtil.getTableKey(tenantId, schemaName, tableName);
+            Long scn = connection.getSCN();
+            long clientTimeStamp = scn == null ? HConstants.LATEST_TIMESTAMP : scn;
+            List<Mutation> tableMetaData = Lists.newArrayListWithExpectedSize(2);
+            @SuppressWarnings("deprecation") // FIXME: Remove when unintentionally deprecated method is fixed (HBASE-7870).
+            Delete tableDelete = new Delete(key, clientTimeStamp, null);
+            tableMetaData.add(tableDelete);
+            if (parentTableName != null) {
+                byte[] linkKey = MetaDataUtil.getParentLinkKey(tenantId, schemaName, parentTableName, tableName);
+                @SuppressWarnings("deprecation") // FIXME: Remove when unintentionally deprecated method is fixed (HBASE-7870).
+                Delete linkDelete = new Delete(linkKey, clientTimeStamp, null);
+                tableMetaData.add(linkDelete);
+            }
+
+            MetaDataMutationResult result = connection.getQueryServices().dropTable(tableMetaData, tableType);
+            MutationCode code = result.getMutationCode();
+            switch(code) {
+                case TABLE_NOT_FOUND:
+                    if (!ifExists) {
+                        throw new TableNotFoundException(schemaName, tableName);
+                    }
+                    break;
+                case NEWER_TABLE_FOUND:
+                    throw new NewerTableAlreadyExistsException(schemaName, tableName);
+                case UNALLOWED_TABLE_MUTATION:
+                    throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_MUTATE_TABLE)
+                        .setSchemaName(schemaName).setTableName(tableName).build().buildException();
+                default:
+                    try {
+                        // TODO: should we update the parent table by removing the index?
+                        connection.removeTable(tableName);
+                    } catch (TableNotFoundException ignore) { } // Ignore - just means wasn't cached
+                    
+                    boolean dropMetaData = connection.getQueryServices().getProps().getBoolean(DROP_METADATA_ATTRIB, DEFAULT_DROP_METADATA);
+                    if (result.getTable() != null && tableType != PTableType.VIEW && !dropMetaData) {
+                        connection.setAutoCommit(true);
+                        // Delete everything in the column. You'll still be able to do queries at earlier timestamps
+                        long ts = (scn == null ? result.getMutationTime() : scn);
+                        // Create empty table and schema - they're only used to get the name from
+                        // PName name, PTableType type, long timeStamp, long sequenceNumber, List<PColumn> columns
+                        PTable table = result.getTable();
+                        List<TableRef> tableRefs = Lists.newArrayListWithExpectedSize(1 + table.getIndexes().size());
+                        tableRefs.add(new TableRef(null, table, ts, false));
+                        // TODO: Let the standard mutable secondary index maintenance handle this?
+                        for (PTable index: table.getIndexes()) {
+                            tableRefs.add(new TableRef(null, index, ts, false));
+                        }
+                        MutationPlan plan = new PostDDLCompiler(connection).compile(tableRefs, null, null, Collections.<PColumn>emptyList(), ts);
+                        return connection.getQueryServices().updateData(plan);
+                    }
+                    break;
+                }
+                 return new MutationState(0,connection);
+        } finally {
+            connection.setAutoCommit(wasAutoCommit);
+        }
+    }
+
+    private MutationCode processMutationResult(String schemaName, String tableName, MetaDataMutationResult result) throws SQLException {
+        final MutationCode mutationCode = result.getMutationCode();
+        switch (mutationCode) {
+        case TABLE_NOT_FOUND:
+            connection.removeTable(tableName);
+            throw new TableNotFoundException(schemaName, tableName);
+        case UNALLOWED_TABLE_MUTATION:
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_MUTATE_TABLE)
+                .setSchemaName(schemaName).setTableName(tableName).build().buildException();
+        case COLUMN_ALREADY_EXISTS:
+        case COLUMN_NOT_FOUND:
+            break;
+        case CONCURRENT_TABLE_MUTATION:
+            connection.addTable(result.getTable());
+            if (logger.isDebugEnabled()) {
+                logger.debug("CONCURRENT_TABLE_MUTATION for table " + SchemaUtil.getTableName(schemaName, tableName));
+            }
+            throw new ConcurrentTableMutationException(schemaName, tableName);
+        case NEWER_TABLE_FOUND:
+            if (result.getTable() != null) {
+                connection.addTable(result.getTable());
+            }
+            throw new NewerTableAlreadyExistsException(schemaName, tableName);
+        case NO_PK_COLUMNS:
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode.PRIMARY_KEY_MISSING)
+                .setSchemaName(schemaName).setTableName(tableName).build().buildException();
+        case TABLE_ALREADY_EXISTS:
+            break;
+        default:
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode.UNEXPECTED_MUTATION_CODE).setSchemaName(schemaName)
+                .setTableName(tableName).setMessage("mutation code: " + mutationCode).build().buildException();
+        }
+        return mutationCode;
+    }
+
+    private  long incrementTableSeqNum(PTable table, PTableType expectedType, int columnCountDelta) throws SQLException {
+        return incrementTableSeqNum(table, expectedType, table.isImmutableRows(), table.isWALDisabled(), table.isMultiTenant(), columnCountDelta);
+    }
+    
+    private long incrementTableSeqNum(PTable table, PTableType expectedType, boolean isImmutableRows, boolean disableWAL, boolean isMultiTenant, int columnCountDelta) throws SQLException {
+        String schemaName = table.getSchemaName().getString();
+        String tableName = table.getTableName().getString();
+        // Ordinal position is 1-based and we don't count SALT column in ordinal position
+        int totalColumnCount = table.getColumns().size() + (table.getBucketNum() == null ? 0 : -1);
+        final long seqNum = table.getSequenceNumber() + 1;
+        PreparedStatement tableUpsert = connection.prepareStatement(MUTATE_TABLE);
+        try {
+            tableUpsert.setString(1, connection.getTenantId() == null ? null : connection.getTenantId().getString());
+            tableUpsert.setString(2, schemaName);
+            tableUpsert.setString(3, tableName);
+            tableUpsert.setString(4, expectedType.getSerializedValue());
+            tableUpsert.setLong(5, seqNum);
+            tableUpsert.setInt(6, totalColumnCount + columnCountDelta);
+            tableUpsert.setBoolean(7, isImmutableRows);
+            tableUpsert.setBoolean(8, disableWAL);
+            tableUpsert.setBoolean(9, isMultiTenant);
+            tableUpsert.execute();
+        } finally {
+            tableUpsert.close();
+        }
+        return seqNum;
+    }
+    
+    public MutationState addColumn(AddColumnStatement statement) throws SQLException {
+        connection.rollback();
+        boolean wasAutoCommit = connection.getAutoCommit();
+        try {
+            connection.setAutoCommit(false);
+            TableName tableNameNode = statement.getTable().getName();
+            String schemaName = tableNameNode.getSchemaName();
+            String tableName = tableNameNode.getTableName();
+            
+            boolean retried = false;
+            while (true) {
+                List<Mutation> tableMetaData = Lists.newArrayListWithExpectedSize(2);
+                ColumnResolver resolver = FromCompiler.getResolver(statement, connection);
+                PTable table = resolver.getTables().get(0).getTable();
+                if (logger.isDebugEnabled()) {
+                    logger.debug("Resolved table to " + table.getName().getString() + " with seqNum " + table.getSequenceNumber() + " at timestamp " + table.getTimeStamp() + " with " + table.getColumns().size() + " columns: " + table.getColumns());
+                }
+                
+                int position = table.getColumns().size();
+                
+                List<PColumn> currentPKs = table.getPKColumns();
+                PColumn lastPK = currentPKs.get(currentPKs.size()-1);
+                // Disallow adding columns if the last column is VARBIANRY.
+                if (lastPK.getDataType() == PDataType.VARBINARY) {
+                    throw new SQLExceptionInfo.Builder(SQLExceptionCode.VARBINARY_LAST_PK)
+                        .setColumnName(lastPK.getName().getString()).build().buildException();
+                }
+                // Disallow adding columns if last column is fixed width and nullable.
+                if (lastPK.isNullable() && lastPK.getDataType().isFixedWidth()) {
+                    throw new SQLExceptionInfo.Builder(SQLExceptionCode.NULLABLE_FIXED_WIDTH_LAST_PK)
+                        .setColumnName(lastPK.getName().getString()).build().buildException();
+                }
+                          
+                boolean isImmutableRows = table.isImmutableRows();
+                Boolean isImmutableRowsProp = (Boolean)statement.getProps().remove(PTable.IS_IMMUTABLE_ROWS_PROP_NAME);
+                if (isImmutableRowsProp != null) {
+                    isImmutableRows = isImmutableRowsProp;
+                }
+                boolean multiTenant = table.isMultiTenant();
+                Boolean multiTenantProp = (Boolean) statement.getProps().remove(PhoenixDatabaseMetaData.MULTI_TENANT);
+                if (multiTenantProp != null) {
+                    multiTenant = Boolean.TRUE.equals(multiTenantProp);
+                }
+                
+                boolean disableWAL = Boolean.TRUE.equals(statement.getProps().remove(DISABLE_WAL));
+                if (statement.getProps().get(PhoenixDatabaseMetaData.SALT_BUCKETS) != null) {
+                    throw new SQLExceptionInfo.Builder(SQLExceptionCode.SALT_ONLY_ON_CREATE_TABLE)
+                    .setTableName(table.getName().getString()).build().buildException();
+                }
+                if (statement.getProps().get(PhoenixDatabaseMetaData.DEFAULT_COLUMN_FAMILY_NAME) != null) {
+                    throw new SQLExceptionInfo.Builder(SQLExceptionCode.DEFAULT_COLUMN_FAMILY_ONLY_ON_CREATE_TABLE)
+                    .setTableName(table.getName().getString()).build().buildException();
+                }
+                
+                boolean isAddingPKColumn = false;
+                PreparedStatement colUpsert = connection.prepareStatement(INSERT_COLUMN);
+                
+                List<ColumnDef> columnDefs = statement.getColumnDefs();
+                if (columnDefs == null) {                    
+                    columnDefs = Lists.newArrayListWithExpectedSize(1);
+                }
+
+                List<Pair<byte[],Map<String,Object>>> families = Lists.newArrayList();                
+                List<PColumn> columns = Lists.newArrayListWithExpectedSize(columnDefs.size());
+
+                if ( columnDefs.size() > 0 ) {
+                    for( ColumnDef colDef : columnDefs) {
+                        if (colDef != null && !colDef.isNull()) {
+                            if(colDef.isPK()) {
+                                throw new SQLExceptionInfo.Builder(SQLExceptionCode.NOT_NULLABLE_COLUMN_IN_ROW_KEY)
+                                .setColumnName(colDef.getColumnDefName().getColumnName()).build().buildException();
+                            } else {
+                                throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ADD_NOT_NULLABLE_COLUMN)
+                                .setColumnName(colDef.getColumnDefName().getColumnName()).build().buildException();
+                            }
+                        }                        
+                        throwIfAlteringViewPK(colDef, table);
+                        PColumn column = newColumn(position++, colDef, PrimaryKeyConstraint.EMPTY);
+                        columns.add(column);
+                        addColumnMutation(schemaName, tableName, column, colUpsert, null);
+
+                        // TODO: support setting properties on other families?
+                        if (column.getFamilyName() != null) {
+                            families.add(new Pair<byte[],Map<String,Object>>(column.getFamilyName().getBytes(),statement.getProps()));
+                        } else { // If adding to primary key, then add the same column to all indexes on the table
+                            isAddingPKColumn = true;
+                            for (PTable index : table.getIndexes()) {
+                                int indexColPosition = index.getColumns().size();
+                                PDataType indexColDataType = IndexUtil.getIndexColumnDataType(column);
+                                ColumnName indexColName = ColumnName.caseSensitiveColumnName(IndexUtil.getIndexColumnName(column));
+                                ColumnDef indexColDef = FACTORY.columnDef(indexColName, indexColDataType.getSqlTypeName(), column.isNullable(), column.getMaxLength(), column.getScale(), true, column.getColumnModifier());
+                                PColumn indexColumn = newColumn(indexColPosition, indexColDef, PrimaryKeyConstraint.EMPTY);
+                                addColumnMutation(schemaName, index.getTableName().getString(), indexColumn, colUpsert, index.getParentTableName().getString());
+                            }
+                        }
+
+                        tableMetaData.addAll(connection.getMutationState().toMutations().next().getSecond());
+                        connection.rollback();
+                    }
+                } else {
+                 // Only support setting IMMUTABLE_ROWS=true and DISABLE_WAL=true on ALTER TABLE SET command
+                    if (!statement.getProps().isEmpty()) {
+                        throw new SQLExceptionInfo.Builder(SQLExceptionCode.SET_UNSUPPORTED_PROP_ON_ALTER_TABLE)
+                        .setTableName(table.getName().getString()).build().buildException();
+                    }
+                    // Check that HBase configured properly for mutable secondary indexing
+                    // if we're changing from an immutable table to a mutable table and we
+                    // have existing indexes.
+                    if (isImmutableRowsProp != null && !isImmutableRows && table.isImmutableRows() && !table.getIndexes().isEmpty()) {
+                        int hbaseVersion = connection.getQueryServices().getLowestClusterHBaseVersion();
+                        if (hbaseVersion < PhoenixDatabaseMetaData.MUTABLE_SI_VERSION_THRESHOLD) {
+                            throw new SQLExceptionInfo.Builder(SQLExceptionCode.NO_MUTABLE_INDEXES).setSchemaName(schemaName).setTableName(tableName).build().buildException();
+                        }
+                        if (connection.getQueryServices().hasInvalidIndexConfiguration()) {
+                            throw new SQLExceptionInfo.Builder(SQLExceptionCode.INVALID_MUTABLE_INDEX_CONFIG).setSchemaName(schemaName).setTableName(tableName).build().buildException();
+                        }
+                    }
+                    // TODO: if switching table to multiTenant or multiType, do some error checking
+                }
+                
+                if (isAddingPKColumn && !table.getIndexes().isEmpty()) {
+                    for (PTable index : table.getIndexes()) {
+                        incrementTableSeqNum(index, index.getType(), 1);
+                    }
+                    tableMetaData.addAll(connection.getMutationState().toMutations().next().getSecond());
+                    connection.rollback();
+                }
+                long seqNum = incrementTableSeqNum(table, statement.getTableType(), isImmutableRows, disableWAL, multiTenant, 1);
+                
+                tableMetaData.addAll(connection.getMutationState().toMutations().next().getSecond());
+                connection.rollback();
+                // Force the table header row to be first
+                Collections.reverse(tableMetaData);
+                
+                Pair<byte[],Map<String,Object>> family = families.size() > 0 ? families.get(0) : null;
+                
+                // Figure out if the empty column family is changing as a result of adding the new column
+                // The empty column family of an index will never change as a result of adding a new data column
+                byte[] emptyCF = null;
+                byte[] projectCF = null;
+                if (table.getType() != PTableType.VIEW && family != null) {
+                    if (table.getColumnFamilies().isEmpty()) {
+                        emptyCF = family.getFirst();
+                    } else {
+                        try {
+                            table.getColumnFamily(family.getFirst());
+                        } catch (ColumnFamilyNotFoundException e) {
+                            projectCF = family.getFirst();
+                            emptyCF = SchemaUtil.getEmptyColumnFamily(table.getColumnFamilies());
+                        }
+                    }
+                }
+                MetaDataMutationResult result = connection.getQueryServices().addColumn(tableMetaData, statement.getTableType(), families);
+                try {
+                    MutationCode code = processMutationResult(schemaName, tableName, result);
+                    if (code == MutationCode.COLUMN_ALREADY_EXISTS) {
+                        connection.addTable(result.getTable());
+                        if (!statement.ifNotExists()) {
+                            throw new ColumnAlreadyExistsException(schemaName, tableName, SchemaUtil.findExistingColumn(result.getTable(), columns));
+                        }
+                        return new MutationState(0,connection);
+                    }
+                    // Only update client side cache if we aren't adding a PK column to a table with indexes.
+                    // We could update the cache manually then too, it'd just be a pain.
+                    if (!isAddingPKColumn || table.getIndexes().isEmpty()) {
+                        connection.addColumn(SchemaUtil.getTableName(schemaName, tableName), columns, result.getMutationTime(), seqNum, isImmutableRows);
+                    }
+                    if (emptyCF != null) {
+                        Long scn = connection.getSCN();
+                        connection.setAutoCommit(true);
+                        // Delete everything in the column. You'll still be able to do queries at earlier timestamps
+                        long ts = (scn == null ? result.getMutationTime() : scn);
+                        MutationPlan plan = new PostDDLCompiler(connection).compile(Collections.singletonList(new TableRef(null, table, ts, false)), emptyCF, projectCF, null, ts);
+                        return connection.getQueryServices().updateData(plan);
+                    }
+                    return new MutationState(0,connection);
+                } catch (ConcurrentTableMutationException e) {
+                    if (retried) {
+                        throw e;
+                    }
+                    if (logger.isDebugEnabled()) {
+                        logger.debug("Caught ConcurrentTableMutationException for table " + SchemaUtil.getTableName(schemaName, tableName) + ". Will try again...");
+                    }
+                    retried = true;
+                }
+            }
+        } finally {
+            connection.setAutoCommit(wasAutoCommit);
+        }
+    }
+
+
+    private String dropColumnMutations(PTable table, List<PColumn> columnsToDrop, List<Mutation> tableMetaData) throws SQLException {
+        String tenantId = connection.getTenantId() == null ? null : connection.getTenantId().getString();
+        String schemaName = table.getSchemaName().getString();
+        String tableName = table.getTableName().getString();
+        String familyName = null;
+        StringBuilder buf = new StringBuilder("DELETE FROM " + TYPE_SCHEMA + ".\"" + TYPE_TABLE + "\" WHERE ");
+        buf.append(TENANT_ID);
+        if (tenantId == null || tenantId.length() == 0) {
+            buf.append(" IS NULL AND ");
+        } else {
+            buf.append(" = ? AND ");
+        }
+        buf.append(TABLE_SCHEM_NAME);
+        if (schemaName == null || schemaName.length() == 0) {
+            buf.append(" IS NULL AND ");
+        } else {
+            buf.append(" = ? AND ");
+        }
+        buf.append (TABLE_NAME_NAME + " = ? AND " + COLUMN_NAME + " = ? AND " + TABLE_CAT_NAME);
+        buf.append(" = ?");
+        
+        // TODO: when DeleteCompiler supports running an fully qualified IN query on the client-side,
+        // we can use a single IN query here instead of executing a different query per column being dropped.
+        PreparedStatement colDelete = connection.prepareStatement(buf.toString());
+        try {
+            for(PColumn columnToDrop : columnsToDrop) {
+                int i = 1;
+                if (tenantId != null && tenantId.length() > 0) {
+                    colDelete.setString(i++, tenantId);
+                }
+                if (schemaName != null & schemaName.length() > 0) {
+                    colDelete.setString(i++, schemaName);    
+                }
+                colDelete.setString(i++, tableName);
+                colDelete.setString(i++, columnToDrop.getName().getString());
+                colDelete.setString(i++, columnToDrop.getFamilyName() == null ? null : columnToDrop.getFamilyName().getString());
+                colDelete.execute();
+            }
+        } finally {
+            if(colDelete != null) {
+                colDelete.close();
+            }
+        }
+        
+       Collections.sort(columnsToDrop,new Comparator<PColumn> () {
+           @Override
+            public int compare(PColumn left, PColumn right) {
+               return Ints.compare(left.getPosition(), right.getPosition());
+            }
+        });
+    
+        int columnsToDropIndex = 0;
+        PreparedStatement colUpdate = connection.prepareStatement(UPDATE_COLUMN_POSITION);
+        colUpdate.setString(1, connection.getTenantId() == null ? null : connection.getTenantId().getString());
+        colUpdate.setString(2, schemaName);
+        colUpdate.setString(3, tableName);
+        for (int i = columnsToDrop.get(columnsToDropIndex).getPosition() + 1; i < table.getColumns().size(); i++) {
+            PColumn column = table.getColumns().get(i);
+            if(columnsToDrop.contains(column)) {
+                columnsToDropIndex++;
+                continue;
+            }
+            colUpdate.setString(4, column.getName().getString());
+            colUpdate.setString(5, column.getFamilyName() == null ? null : column.getFamilyName().getString());
+            colUpdate.setInt(6, column.getPosition() - columnsToDropIndex);
+            colUpdate.execute();
+        }
+       return familyName;
+    }
+    
+    /**
+     * Calculate what the new column family will be after the column is dropped, returning null
+     * if unchanged.
+     * @param table table containing column to drop
+     * @param columnToDrop column being dropped
+     * @return the new column family or null if unchanged.
+     */
+    private static byte[] getNewEmptyColumnFamilyOrNull (PTable table, PColumn columnToDrop) {
+        if (table.getType() != PTableType.VIEW && !SchemaUtil.isPKColumn(columnToDrop) && table.getColumnFamilies().get(0).getName().equals(columnToDrop.getFamilyName()) && table.getColumnFamilies().get(0).getColumns().size() == 1) {
+            return SchemaUtil.getEmptyColumnFamily(table.getColumnFamilies().subList(1, table.getColumnFamilies().size()));
+        }
+        // If unchanged, return null
+        return null;
+    }
+    
+    public MutationState dropColumn(DropColumnStatement statement) throws SQLException {
+        connection.rollback();
+        boolean wasAutoCommit = connection.getAutoCommit();
+        try {
+            connection.setAutoCommit(false);
+            TableName tableNameNode = statement.getTable().getName();
+            String schemaName = tableNameNode.getSchemaName();
+            String tableName = tableNameNode.getTableName();
+            String fullTableName = SchemaUtil.getTableName(schemaName, tableName);
+            boolean retried = false;
+            while (true) {
+                final ColumnResolver resolver = FromCompiler.getResolver(statement, connection);
+                PTable table = resolver.getTables().get(0).getTable();
+                List<ColumnName> columnRefs = statement.getColumnRefs();
+                if(columnRefs == null) {
+                    columnRefs = Lists.newArrayListWithCapacity(0);
+                }
+                TableRef tableRef = null;
+                List<ColumnRef> columnsToDrop = Lists.newArrayListWithExpectedSize(columnRefs.size() + table.getIndexes().size());
+                List<TableRef> indexesToDrop = Lists.newArrayListWithExpectedSize(table.getIndexes().size());
+                List<Mutation> tableMetaData = Lists.newArrayListWithExpectedSize((table.getIndexes().size() + 1) * (1 + table.getColumns().size() - columnRefs.size()));
+                List<PColumn>  tableColumnsToDrop = Lists.newArrayListWithExpectedSize(columnRefs.size());
+                
+                for(ColumnName column : columnRefs) {
+                    ColumnRef columnRef = null;
+                    try {
+                        columnRef = resolver.resolveColumn(null, column.getFamilyName(), column.getColumnName());
+                    } catch (ColumnNotFoundException e) {
+                        if (statement.ifExists()) {
+                            return new MutationState(0,connection);
+                        }
+                        throw e;
+                    }
+                    tableRef = columnRef.getTableRef();
+                    PColumn columnToDrop = columnRef.getColumn();
+                    tableColumnsToDrop.add(columnToDrop);
+                    if (SchemaUtil.isPKColumn(columnToDrop)) {
+                        throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_DROP_PK)
+                            .setColumnName(columnToDrop.getName().getString()).build().buildException();
+                    }
+                    columnsToDrop.add(new ColumnRef(tableRef, columnToDrop.getPosition()));
+                }
+                
+                dropColumnMutations(table, tableColumnsToDrop, tableMetaData);
+                for (PTable index : table.getIndexes()) {
+                    List<PColumn> indexColumnsToDrop = Lists.newArrayListWithExpectedSize(columnRefs.size());
+                    for(PColumn columnToDrop : tableColumnsToDrop) {
+                        String indexColumnName = IndexUtil.getIndexColumnName(columnToDrop);
+                        try {
+                            PColumn indexColumn = index.getColumn(indexColumnName);
+                            if (SchemaUtil.isPKColumn(indexColumn)) {
+                

<TRUNCATED>

[11/51] [partial] Initial commit of master branch from github

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataEntityNotFoundException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataEntityNotFoundException.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataEntityNotFoundException.java
new file mode 100644
index 0000000..e304e4f
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataEntityNotFoundException.java
@@ -0,0 +1,30 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.sql.SQLException;
+
+public abstract class MetaDataEntityNotFoundException extends SQLException {
+    private static final long serialVersionUID = 1L;
+
+    public MetaDataEntityNotFoundException(String reason, String sqlState, int code, Throwable cause) {
+        super(reason, sqlState, code, cause);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataSplitPolicy.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataSplitPolicy.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataSplitPolicy.java
new file mode 100644
index 0000000..a5a9bd2
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataSplitPolicy.java
@@ -0,0 +1,51 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.regionserver.ConstantSizeRegionSplitPolicy;
+
+import org.apache.phoenix.util.SchemaUtil;
+
+
+public class MetaDataSplitPolicy extends ConstantSizeRegionSplitPolicy {
+
+    @Override
+    protected byte[] getSplitPoint() {
+        byte[] splitPoint = super.getSplitPoint();
+        int offset = SchemaUtil.getVarCharLength(splitPoint, 0, splitPoint.length);
+        // Split only on Phoenix schema name, so this is ok b/c we won't be splitting
+        // in the middle of a Phoenix table.
+        if (offset == splitPoint.length) {
+            return splitPoint;
+        }
+//        offset = SchemaUtil.getVarCharLength(splitPoint, offset+1, splitPoint.length-offset-1);
+//        // Split only on Phoenix schema and table name, so this is ok b/c we won't be splitting
+//        // in the middle of a Phoenix table.
+//        if (offset == splitPoint.length) {
+//            return splitPoint;
+//        }
+        // Otherwise, an attempt is being made to split in the middle of a table.
+        // Just return a split point at the schema boundary instead
+        byte[] newSplitPoint = new byte[offset + 1];
+        System.arraycopy(splitPoint, 0, newSplitPoint, 0, offset+1);
+        return newSplitPoint;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/NewerTableAlreadyExistsException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/NewerTableAlreadyExistsException.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/NewerTableAlreadyExistsException.java
new file mode 100644
index 0000000..b5979e8
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/NewerTableAlreadyExistsException.java
@@ -0,0 +1,29 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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;
+
+public class NewerTableAlreadyExistsException extends TableAlreadyExistsException {
+	private static final long serialVersionUID = 1L;
+
+    public NewerTableAlreadyExistsException(String schemaName, String tableName) {
+        super(schemaName, tableName);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/PArrayDataType.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PArrayDataType.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PArrayDataType.java
new file mode 100644
index 0000000..dbff35b
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PArrayDataType.java
@@ -0,0 +1,386 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.nio.ByteBuffer;
+import java.sql.Types;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import org.apache.phoenix.util.ByteUtil;
+
+/**
+ * The datatype for PColummns that are Arrays
+ */
+public class PArrayDataType {
+
+    private static final int MAX_POSSIBLE_VINT_LENGTH = 2;
+    private static final byte ARRAY_SERIALIZATION_VERSION = 1;
+	public PArrayDataType() {
+	}
+
+	public byte[] toBytes(Object object, PDataType baseType) {
+		if(object == null) {
+			throw new ConstraintViolationException(this + " may not be null");
+		}
+	    int size = PDataType.fromTypeId((baseType.getSqlType() + Types.ARRAY)).estimateByteSize(object);
+	    
+        int noOfElements = ((PhoenixArray)object).numElements;
+        if(noOfElements == 0) {
+        	return ByteUtil.EMPTY_BYTE_ARRAY;
+        }
+        ByteBuffer buffer;
+        int capacity = 0;
+		if (!baseType.isFixedWidth() || baseType.isCoercibleTo(PDataType.VARCHAR)) {
+			// variable
+			if (calculateMaxOffset(size)) {
+				// Use Short to represent the offset
+				capacity = initOffsetArray(noOfElements, Bytes.SIZEOF_SHORT);
+			} else {
+				capacity = initOffsetArray(noOfElements, Bytes.SIZEOF_INT);
+				// Negate the number of elements
+				noOfElements = -noOfElements;
+			}
+			buffer = ByteBuffer.allocate(size + capacity + Bytes.SIZEOF_INT+ Bytes.SIZEOF_BYTE);
+		} else {
+			buffer = ByteBuffer.allocate(size);
+		}
+		return bytesFromByteBuffer((PhoenixArray)object, buffer, noOfElements, baseType, capacity);
+	}
+
+	private boolean calculateMaxOffset(int size) {
+		// If the total size + Offset postion ptr + Numelements in Vint is less than Short
+		if ((size + Bytes.SIZEOF_INT + MAX_POSSIBLE_VINT_LENGTH) <= (2 * Short.MAX_VALUE)) {
+			return true;
+		}
+		return false;
+	}
+
+	public int toBytes(Object object, byte[] bytes, int offset) {
+	    PhoenixArray array = (PhoenixArray)object;
+        if (array == null || array.baseType == null) {
+            return 0;
+        }
+	    return PDataType.fromTypeId((array.baseType.getSqlType() + Types.ARRAY)).estimateByteSize(object);
+	}
+
+	public boolean isCoercibleTo(PDataType targetType, Object value) {
+	    return targetType.isCoercibleTo(targetType, value);
+	}
+	
+	public boolean isCoercibleTo(PDataType targetType, PDataType expectedTargetType) {
+		if(!targetType.isArrayType()) {
+			return false;
+		} else {
+			PDataType targetElementType = PDataType.fromTypeId(targetType.getSqlType()
+					- Types.ARRAY);
+			PDataType expectedTargetElementType = PDataType.fromTypeId(expectedTargetType
+					.getSqlType() - Types.ARRAY);
+			return expectedTargetElementType.isCoercibleTo(targetElementType);
+		}
+    }
+	
+	public boolean isSizeCompatible(PDataType srcType, Object value,
+			byte[] b, Integer maxLength, Integer desiredMaxLength,
+			Integer scale, Integer desiredScale) {
+		PhoenixArray pArr = (PhoenixArray) value;
+		Object[] charArr = (Object[]) pArr.array;
+		PDataType baseType = PDataType.fromTypeId(srcType.getSqlType()
+				- Types.ARRAY);
+		for (int i = 0 ; i < charArr.length; i++) {
+			if (!baseType.isSizeCompatible(baseType, value, b, maxLength,
+					desiredMaxLength, scale, desiredScale)) {
+				return false;
+			}
+		}
+		return true;
+	}
+
+
+    public Object toObject(String value) {
+		// TODO: Do this as done in CSVLoader
+		throw new IllegalArgumentException("This operation is not suppported");
+	}
+
+	public Object toObject(byte[] bytes, int offset, int length, PDataType baseType, 
+			ColumnModifier columnModifier) {
+		return createPhoenixArray(bytes, offset, length, columnModifier,
+				baseType);
+	}
+	
+	public static void positionAtArrayElement(ImmutableBytesWritable ptr, int arrayIndex, PDataType baseDataType) {
+		byte[] bytes = ptr.get();
+		int initPos = ptr.getOffset();
+		int noOfElements = 0;
+		noOfElements = Bytes.toInt(bytes, ptr.getOffset() + Bytes.SIZEOF_BYTE, Bytes.SIZEOF_INT);
+		int noOFElementsSize = Bytes.SIZEOF_INT;
+		if(arrayIndex >= noOfElements) {
+			throw new IndexOutOfBoundsException(
+					"Invalid index "
+							+ arrayIndex
+							+ " specified, greater than the no of elements in the array: "
+							+ noOfElements);
+		}
+		boolean useShort = true;
+		int baseSize = Bytes.SIZEOF_SHORT;
+		if (noOfElements < 0) {
+			noOfElements = -noOfElements;
+			baseSize = Bytes.SIZEOF_INT;
+			useShort = false;
+		}
+
+		if (baseDataType.getByteSize() == null) {
+			int offset = ptr.getOffset() + noOFElementsSize + Bytes.SIZEOF_BYTE;
+			int indexOffset = Bytes.toInt(bytes, offset) + ptr.getOffset();
+			int valArrayPostion = offset + Bytes.SIZEOF_INT;
+			offset += Bytes.SIZEOF_INT;
+			int currOff = 0;
+			if (noOfElements > 1) {
+				while (offset <= (initPos+ptr.getLength())) {
+					int nextOff = 0;
+					// 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.
+					if(useShort) {
+						// If the arrayIndex is already the last element then read the last before one element and the last element
+						offset = indexOffset + (Bytes.SIZEOF_SHORT * arrayIndex);
+						if (arrayIndex == (noOfElements - 1)) {
+							currOff = Bytes.toShort(bytes, offset, baseSize) + Short.MAX_VALUE;
+							nextOff = indexOffset;
+							offset += baseSize;
+						} else {
+							currOff = Bytes.toShort(bytes, offset, baseSize) + Short.MAX_VALUE;
+							offset += baseSize;
+							nextOff = Bytes.toShort(bytes, offset, baseSize) + Short.MAX_VALUE;
+							offset += baseSize;
+						}
+					} else {
+						// If the arrayIndex is already the last element then read the last before one element and the last element
+						offset = indexOffset + (Bytes.SIZEOF_INT * arrayIndex);
+						if (arrayIndex == (noOfElements - 1)) {
+							currOff = Bytes.toInt(bytes, offset, baseSize);
+							nextOff = indexOffset;
+							offset += baseSize;
+						} else {
+							currOff = Bytes.toInt(bytes, offset, baseSize);
+							offset += baseSize;
+							nextOff = Bytes.toInt(bytes, offset, baseSize);
+							offset += baseSize;
+						}
+					}
+					int elementLength = nextOff - currOff;
+					ptr.set(bytes, currOff + initPos, elementLength);
+					break;
+				}
+			} else {
+				ptr.set(bytes, valArrayPostion + initPos, indexOffset - valArrayPostion);
+			}
+		} else {
+			ptr.set(bytes,
+					ptr.getOffset() + arrayIndex * baseDataType.getByteSize()
+							+ noOFElementsSize + Bytes.SIZEOF_BYTE, baseDataType.getByteSize());
+		}
+	}
+
+	public Object toObject(byte[] bytes, int offset, int length, PDataType baseType) {
+		return toObject(bytes, offset, length, baseType, null);
+	}
+	
+	public Object toObject(Object object, PDataType actualType) {
+		return object;
+	}
+
+	public Object toObject(Object object, PDataType actualType, ColumnModifier sortOrder) {
+		// How to use the sortOrder ? Just reverse the elements
+		return toObject(object, actualType);
+	}
+	
+	// Making this private
+	/**
+	 * The format of the byte buffer looks like this for variable length array elements
+	 * <noofelements><Offset of the index array><elements><offset array>
+	 * where <noOfelements> - vint
+	 * <offset of the index array> - int
+	 * <elements>  - these are the values
+	 * <offset array> - offset of every element written as INT/SHORT
+	 * 
+	 * @param array
+	 * @param buffer
+	 * @param noOfElements
+	 * @param byteSize
+	 * @param capacity 
+	 * @return
+	 */
+	private byte[] bytesFromByteBuffer(PhoenixArray array, ByteBuffer buffer,
+			int noOfElements, PDataType baseType, int capacity) {
+		int temp = noOfElements;
+        if (buffer == null) return null;
+        buffer.put(ARRAY_SERIALIZATION_VERSION);
+        buffer.putInt(noOfElements);
+        if (!baseType.isFixedWidth() || baseType.isCoercibleTo(PDataType.VARCHAR)) {
+            int fillerForOffsetByteArray = buffer.position();
+            buffer.position(fillerForOffsetByteArray + Bytes.SIZEOF_INT);
+            ByteBuffer offsetArray = ByteBuffer.allocate(capacity);
+            if(noOfElements < 0){
+            	noOfElements = -noOfElements;
+            }
+            for (int i = 0; i < noOfElements; i++) {
+                // Not fixed width
+				if (temp < 0) {
+					offsetArray.putInt(buffer.position());
+				} else {
+					offsetArray.putShort((short)(buffer.position() - Short.MAX_VALUE));
+				}
+                byte[] bytes = array.toBytes(i);
+                buffer.put(bytes);
+            }
+            int offsetArrayPosition = buffer.position();
+            buffer.put(offsetArray.array());
+            buffer.position(fillerForOffsetByteArray);
+            buffer.putInt(offsetArrayPosition);
+        } else {
+            for (int i = 0; i < noOfElements; i++) {
+                byte[] bytes = array.toBytes(i);
+                buffer.put(bytes);
+            }
+        }
+        return buffer.array();
+	}
+
+	private static int initOffsetArray(int noOfElements, int baseSize) {
+		// for now create an offset array equal to the noofelements
+		return noOfElements * baseSize;
+    }
+
+	private Object createPhoenixArray(byte[] bytes, int offset, int length,
+			ColumnModifier columnModifier, PDataType baseDataType) {
+		if(bytes == null || bytes.length == 0) {
+			return null;
+		}
+		ByteBuffer buffer = ByteBuffer.wrap(bytes, offset, length);
+		int initPos = buffer.position();
+		buffer.get();
+		int noOfElements = buffer.getInt();
+		boolean useShort = true;
+		int baseSize = Bytes.SIZEOF_SHORT;
+		if(noOfElements < 0) {
+			noOfElements = -noOfElements;
+			baseSize = Bytes.SIZEOF_INT;
+			useShort = false;
+		}
+		Object[] elements = (Object[]) java.lang.reflect.Array.newInstance(
+				baseDataType.getJavaClass(), noOfElements);
+		if (!baseDataType.isFixedWidth() || baseDataType.isCoercibleTo(PDataType.VARCHAR)) {
+			int indexOffset = buffer.getInt();
+			int valArrayPostion = buffer.position();
+			buffer.position(indexOffset + initPos);
+			ByteBuffer indexArr = ByteBuffer
+					.allocate(initOffsetArray(noOfElements, baseSize));
+			byte[] array = indexArr.array();
+			buffer.get(array);
+			int countOfElementsRead = 0;
+			int i = 0;
+			int currOff = -1;
+			int nextOff = -1;
+			if (noOfElements > 1) {
+				while (indexArr.hasRemaining()) {
+					if (countOfElementsRead < noOfElements) {
+						if (currOff == -1) {
+							if ((indexArr.position() + 2 * baseSize) <= indexArr
+									.capacity()) {
+								if (useShort) {
+									currOff = indexArr.getShort() + Short.MAX_VALUE;
+									nextOff = indexArr.getShort() + Short.MAX_VALUE;
+								} else {
+									currOff = indexArr.getInt();
+									nextOff = indexArr.getInt();
+								}
+								countOfElementsRead += 2;
+							}
+						} else {
+							currOff = nextOff;
+							if(useShort) {
+								nextOff = indexArr.getShort() + Short.MAX_VALUE;
+							} else {
+								nextOff = indexArr.getInt();
+							}
+							countOfElementsRead += 1;
+						}
+						int elementLength = nextOff - currOff;
+						buffer.position(currOff + initPos);
+						byte[] val = new byte[elementLength];
+						buffer.get(val);
+						elements[i++] = baseDataType.toObject(val,
+								columnModifier);
+					}
+				}
+				buffer.position(nextOff + initPos);
+				byte[] val = new byte[indexOffset - nextOff];
+				buffer.get(val);
+				elements[i++] = baseDataType.toObject(val, columnModifier);
+			} else {
+				byte[] val = new byte[indexOffset - valArrayPostion];
+				buffer.position(valArrayPostion + initPos);
+				buffer.get(val);
+				elements[i++] = baseDataType.toObject(val, columnModifier);
+			}
+		} else {
+			for (int i = 0; i < noOfElements; i++) {
+				byte[] val;
+				if (baseDataType.getByteSize() == null) {
+					val = new byte[length];
+				} else {
+					val = new byte[baseDataType.getByteSize()];
+				}
+				buffer.get(val);
+				elements[i] = baseDataType.toObject(val, columnModifier);
+			}
+		}
+		return PArrayDataType
+				.instantiatePhoenixArray(baseDataType, elements);
+	}
+	
+    public static PhoenixArray instantiatePhoenixArray(PDataType actualType, Object[] elements) {
+        return PDataType.instantiatePhoenixArray(actualType, elements);
+    }
+	
+	public int compareTo(Object lhs, Object rhs) {
+		PhoenixArray lhsArr = (PhoenixArray) lhs;
+		PhoenixArray rhsArr = (PhoenixArray) rhs;
+		if(lhsArr.equals(rhsArr)) {
+			return 0;
+		}
+		return 1;
+	}
+
+	public static int getArrayLength(ImmutableBytesWritable ptr,
+			PDataType baseType) {
+		byte[] bytes = ptr.get();
+		if(baseType.isFixedWidth()) {
+			return ((ptr.getLength() - (Bytes.SIZEOF_BYTE + Bytes.SIZEOF_INT))/baseType.getByteSize());
+		}
+		return Bytes.toInt(bytes, ptr.getOffset() + Bytes.SIZEOF_BYTE);
+	}
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/PBaseColumn.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PBaseColumn.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PBaseColumn.java
new file mode 100644
index 0000000..5169e29
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PBaseColumn.java
@@ -0,0 +1,52 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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;
+
+
+/**
+ * 
+ * Base class for PColumn implementors that provides
+ * some reasonable default implementations.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public abstract class PBaseColumn implements PColumn {
+
+    @Override
+    public final Integer getByteSize() {
+        return getDataType().getByteSize();
+    }
+
+    @Override
+    public final Integer getMaxLength() {
+        return null;
+    }
+
+    @Override
+    public final Integer getScale() {
+        return null;
+    }
+
+    @Override
+    public boolean isNullable() {
+        return false;
+    }
+ }

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..4436a11
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumn.java
@@ -0,0 +1,51 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.io.Writable;
+
+/**
+ * Definition of a Phoenix column
+ *
+ * @author wmacklem,jtaylor
+ * @since 0.1
+ */
+public interface PColumn extends PDatum, Writable {
+
+    /**
+     * @return the name of the column qualifier
+     */
+    PName getName();
+
+    /**
+     * @return the name of the column family
+     */
+    PName getFamilyName();
+
+    /**
+     * @return the zero-based ordinal position of the column
+     */
+    int getPosition();
+    
+    /**
+     * @return the declared array size or zero if this is not an array
+     */
+    Integer getArraySize();
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..6927c63
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnFamily.java
@@ -0,0 +1,54 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.util.Collection;
+
+/**
+ * 
+ * Definition of a Phoenix Column Family
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public interface PColumnFamily {
+    
+    /**
+     * @return The column family name.
+     */
+    PName getName();
+    
+    /**
+     * @return All the PColumns in this column family.
+     */
+    Collection<PColumn> getColumns();
+    
+    /**
+     * @return The PColumn for the specified column qualifier.
+     * @throws ColumnNotFoundException if the column cannot be found
+     */
+    PColumn getColumn(byte[] qualifier) throws ColumnNotFoundException;
+    
+    /**
+     * @return The PColumn for the specified column qualifier.
+     * @throws ColumnNotFoundException if the column cannot be found
+     */
+    PColumn getColumn(String name) throws ColumnNotFoundException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..4b09762
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnFamilyImpl.java
@@ -0,0 +1,75 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.util.Bytes;
+
+import com.google.common.collect.*;
+
+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;
+    
+    public PColumnFamilyImpl(PName name, List<PColumn> columns) {
+        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);
+        for (PColumn column : columns) {
+            columnByBytesBuilder.put(column.getName().getBytes(), column);
+            columnByStringBuilder.put(column.getName().getString(), column);
+        }
+        this.columnByBytes = columnByBytesBuilder.build();
+        this.columnByString = columnByStringBuilder.build();
+    }
+    
+    @Override
+    public PName getName() {
+        return name;
+    }
+
+    @Override
+    public List<PColumn> getColumns() {
+        return columns;
+    }
+
+    @Override
+    public PColumn getColumn(byte[] qualifier) throws ColumnNotFoundException  {
+        PColumn column = columnByBytes.get(qualifier);
+        if (column == null) {
+            throw new ColumnNotFoundException(Bytes.toString(qualifier));
+        }
+        return column;
+    }
+    
+    @Override
+    public PColumn getColumn(String name) throws ColumnNotFoundException  {
+        PColumn column = columnByString.get(name);
+        if (column == null) {
+            throw new ColumnNotFoundException(name);
+        }
+        return column;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..c20cc8c
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnImpl.java
@@ -0,0 +1,207 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.util.ByteUtil;
+
+import com.google.common.base.Preconditions;
+
+
+public class PColumnImpl implements PColumn {
+    private static final Integer NO_MAXLENGTH = Integer.MIN_VALUE;
+    private static final Integer NO_SCALE = Integer.MIN_VALUE;
+
+    private PName name;
+    private PName familyName;
+    private PDataType dataType;
+    private Integer maxLength;
+    private Integer scale;
+    private boolean nullable;
+    private int position;
+    private ColumnModifier columnModifier;
+    private Integer arraySize;
+
+    public PColumnImpl() {
+    }
+
+    public PColumnImpl(PName name,
+                       PName familyName,
+                       PDataType dataType,
+                       Integer maxLength,
+                       Integer scale,
+                       boolean nullable,
+                       int position,
+                       ColumnModifier sortOrder, Integer arrSize) {
+        init(name, familyName, dataType, maxLength, scale, nullable, position, sortOrder, arrSize);
+    }
+
+    public PColumnImpl(PColumn column, int position) {
+        this(column.getName(), column.getFamilyName(), column.getDataType(), column.getMaxLength(),
+                column.getScale(), column.isNullable(), position, column.getColumnModifier(), column.getArraySize());
+    }
+
+    private void init(PName name,
+            PName familyName,
+            PDataType dataType,
+            Integer maxLength,
+            Integer scale,
+            boolean nullable,
+            int position,
+            ColumnModifier columnModifier,
+            Integer arrSize) {
+        this.dataType = dataType;
+        if (familyName == null) {
+            // Allow nullable columns in PK, but only if they're variable length.
+            // Variable length types may be null, since we use a null-byte terminator
+            // (which is a disallowed character in variable length types). However,
+            // fixed width types do not have a way of representing null.
+            // TODO: we may be able to allow this for columns at the end of the PK
+            Preconditions.checkArgument(!nullable || !dataType.isFixedWidth(), 
+                    "PK columns may not be both fixed width and nullable: " + name.getString());
+        }
+        this.name = name;
+        this.familyName = familyName == null ? null : familyName;
+        this.maxLength = maxLength;
+        this.scale = scale;
+        this.nullable = nullable;
+        this.position = position;
+        this.columnModifier = columnModifier;
+        this.arraySize = arrSize;
+    }
+
+    @Override
+    public PName getName() {
+        return name;
+    }
+
+    @Override
+    public PName getFamilyName() {
+        return familyName;
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return dataType;
+    }
+
+    @Override
+    public Integer getMaxLength() {
+        return maxLength;
+    }
+
+    @Override
+    public Integer getScale() {
+        return scale;
+    }
+
+    @Override
+    public Integer getByteSize() {
+        Integer dataTypeMaxLength = dataType.getByteSize();
+        return dataTypeMaxLength == null ? dataType.estimateByteSizeFromLength(maxLength)
+                : dataTypeMaxLength;
+    }
+
+    @Override
+    public boolean isNullable() {
+        return nullable;
+    }
+
+    @Override
+    public int getPosition() {
+        return position;
+    }
+    
+    @Override
+    public ColumnModifier getColumnModifier() {
+    	return columnModifier;
+    }
+
+    @Override
+    public String toString() {
+        return (familyName == null ? "" : familyName.toString() + QueryConstants.NAME_SEPARATOR) + name.toString();
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        byte[] columnNameBytes = Bytes.readByteArray(input);
+        PName columnName = PNameFactory.newName(columnNameBytes);
+        byte[] familyNameBytes = Bytes.readByteArray(input);
+        PName familyName = familyNameBytes.length == 0 ? null : PNameFactory.newName(familyNameBytes);
+        // TODO: optimize the reading/writing of this b/c it could likely all fit in a single byte or two
+        PDataType dataType = PDataType.values()[WritableUtils.readVInt(input)];
+        int maxLength = WritableUtils.readVInt(input);
+        int scale = WritableUtils.readVInt(input);
+        boolean nullable = input.readBoolean();
+        int position = WritableUtils.readVInt(input);
+        ColumnModifier columnModifier = ColumnModifier.fromSystemValue(WritableUtils.readVInt(input));
+        int arrSize = WritableUtils.readVInt(input);
+        init(columnName, familyName, dataType, maxLength == NO_MAXLENGTH ? null : maxLength,
+                scale == NO_SCALE ? null : scale, nullable, position, columnModifier, arrSize == -1 ? null : arrSize);
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+        Bytes.writeByteArray(output, name.getBytes());
+        Bytes.writeByteArray(output, familyName == null ? ByteUtil.EMPTY_BYTE_ARRAY : familyName.getBytes());
+        WritableUtils.writeVInt(output, dataType.ordinal());
+        WritableUtils.writeVInt(output, maxLength == null ? NO_MAXLENGTH : maxLength);
+        WritableUtils.writeVInt(output, scale == null ? NO_SCALE : scale);
+        output.writeBoolean(nullable);
+        WritableUtils.writeVInt(output, position);
+        WritableUtils.writeVInt(output, ColumnModifier.toSystemValue(columnModifier));
+        WritableUtils.writeVInt(output, arraySize == null ? -1 : arraySize);
+    }
+    
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + ((familyName == null) ? 0 : familyName.hashCode());
+        result = prime * result + ((name == null) ? 0 : name.hashCode());
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) return true;
+        if (obj == null) return false;
+        if (getClass() != obj.getClass()) return false;
+        PColumnImpl other = (PColumnImpl)obj;
+        if (familyName == null) {
+            if (other.familyName != null) return false;
+        } else if (!familyName.equals(other.familyName)) return false;
+        if (name == null) {
+            if (other.name != null) return false;
+        } else if (!name.equals(other.name)) return false;
+        return true;
+    }
+
+    @Override
+    public Integer getArraySize() {
+        return arraySize;
+    }
+}
\ No newline at end of file


[25/51] [partial] Initial commit of master branch from github

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..3f56bb8
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexFailurePolicy.java
@@ -0,0 +1,102 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.index;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+
+import com.google.common.collect.Multimap;
+import org.apache.hadoop.hbase.index.table.HTableInterfaceReference;
+import org.apache.hadoop.hbase.index.write.KillServerOnFailurePolicy;
+import org.apache.phoenix.coprocessor.MetaDataProtocol;
+import org.apache.phoenix.coprocessor.MetaDataProtocol.MetaDataMutationResult;
+import org.apache.phoenix.coprocessor.MetaDataProtocol.MutationCode;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.schema.PIndexState;
+import org.apache.phoenix.util.SchemaUtil;
+
+/**
+ * 
+ * Handler called in the event that index updates cannot be written to their
+ * region server. First attempts to disable the index and failing that falls
+ * back to the default behavior of killing the region server.
+ *
+ * TODO: use delegate pattern instead
+ * 
+ * @author jtaylor
+ * @since 2.1
+ */
+public class PhoenixIndexFailurePolicy extends  KillServerOnFailurePolicy {
+    private static final Log LOG = LogFactory.getLog(PhoenixIndexFailurePolicy.class);
+    private RegionCoprocessorEnvironment env;
+
+    public PhoenixIndexFailurePolicy() {
+    }
+
+    @Override
+    public void setup(Stoppable parent, RegionCoprocessorEnvironment env) {
+      super.setup(parent, env);
+      this.env = env;
+    }
+
+    @Override
+    public void handleFailure(Multimap<HTableInterfaceReference, Mutation> attempted, Exception cause) throws IOException {
+        Set<HTableInterfaceReference> refs = attempted.asMap().keySet();
+        StringBuilder buf = new StringBuilder("Disabled index" + (refs.size() > 1 ? "es " : " "));
+        try {
+            for (HTableInterfaceReference ref : refs) {
+                // Disable the index by using the updateIndexState method of MetaDataProtocol end point coprocessor.
+                String indexTableName = ref.getTableName();
+                byte[] indexTableKey = SchemaUtil.getTableKeyFromFullName(indexTableName);
+                HTableInterface systemTable = env.getTable(PhoenixDatabaseMetaData.TYPE_TABLE_NAME_BYTES);
+                MetaDataProtocol mdProxy = systemTable.coprocessorProxy(MetaDataProtocol.class, indexTableKey);
+                // Mimic the Put that gets generated by the client on an update of the index state
+                Put put = new Put(indexTableKey);
+                put.add(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES, PhoenixDatabaseMetaData.INDEX_STATE_BYTES, PIndexState.DISABLE.getSerializedBytes());
+                List<Mutation> tableMetadata = Collections.<Mutation>singletonList(put);
+                MetaDataMutationResult result = mdProxy.updateIndexState(tableMetadata);
+                if (result.getMutationCode() != MutationCode.TABLE_ALREADY_EXISTS) {
+                    LOG.warn("Attempt to disable index " + indexTableName + " failed with code = " + result.getMutationCode() + ". Will use default failure policy instead.");
+                    super.handleFailure(attempted, cause);
+                }
+                LOG.info("Successfully disabled index " + indexTableName);
+                buf.append(indexTableName);
+                buf.append(',');
+            }
+            buf.setLength(buf.length()-1);
+            buf.append(" due to an exception while writing updates");
+        } catch (Throwable t) {
+            super.handleFailure(attempted, cause);
+        }
+        throw new DoNotRetryIOException(buf.toString(), cause);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/iterate/AggregatingResultIterator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/AggregatingResultIterator.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/AggregatingResultIterator.java
new file mode 100644
index 0000000..237a89a
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/AggregatingResultIterator.java
@@ -0,0 +1,40 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.iterate;
+
+import org.apache.phoenix.schema.tuple.Tuple;
+
+
+/**
+ * 
+ * Interface for scanners that either do aggregation
+ * or delegate to scanners that do aggregation.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public interface AggregatingResultIterator extends ResultIterator {
+    /**
+     * Provides a means of re-aggregating a result row. For
+     * scanners that need to look ahead (i.e. {@link org.apache.phoenix.iterate.OrderedAggregatingResultIterator}
+     * @param result the row to re-aggregate
+     */
+    void aggregate(Tuple result);
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterator.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterator.java
new file mode 100644
index 0000000..ed6c360
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterator.java
@@ -0,0 +1,43 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.iterate;
+
+import java.sql.SQLException;
+import java.util.List;
+
+/**
+ * 
+ * Abstract base class for ResultIterator implementations that
+ * do nothing on close and have no explain plan steps
+ *
+ * @author jtaylor
+ * @since 1.2
+ */
+public abstract class BaseResultIterator implements ResultIterator {
+    
+    @Override
+    public void close() throws SQLException {
+    }
+
+    @Override
+    public void explain(List<String> planSteps) {
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/iterate/ConcatResultIterator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/ConcatResultIterator.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/ConcatResultIterator.java
new file mode 100644
index 0000000..08b4639
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/ConcatResultIterator.java
@@ -0,0 +1,91 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.iterate;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.phoenix.schema.tuple.Tuple;
+
+
+/**
+ * 
+ * Result iterator that concatenates a list of other iterators.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class ConcatResultIterator implements PeekingResultIterator {
+    private final ResultIterators resultIterators;
+    private List<PeekingResultIterator> iterators;
+    private int index;
+    
+    public ConcatResultIterator(ResultIterators iterators) {
+        this.resultIterators = iterators;
+    }
+    
+    private List<PeekingResultIterator> getIterators() throws SQLException {
+        if (iterators == null) {
+            iterators = resultIterators.getIterators();
+        }
+        return iterators;
+    }
+    
+    @Override
+    public void close() throws SQLException {
+        if (iterators != null) {
+            for (;index < iterators.size(); index++) {
+                PeekingResultIterator iterator = iterators.get(index);
+                iterator.close();
+            }
+        }
+    }
+
+
+    @Override
+    public void explain(List<String> planSteps) {
+        resultIterators.explain(planSteps);
+    }
+
+    private PeekingResultIterator currentIterator() throws SQLException {
+        List<PeekingResultIterator> iterators = getIterators();
+        while (index < iterators.size()) {
+            PeekingResultIterator iterator = iterators.get(index);
+            Tuple r = iterator.peek();
+            if (r != null) {
+                return iterator;
+            }
+            iterator.close();
+            index++;
+        }
+        return EMPTY_ITERATOR;
+    }
+    
+    @Override
+    public Tuple peek() throws SQLException {
+        return currentIterator().peek();
+    }
+
+    @Override
+    public Tuple next() throws SQLException {
+        return currentIterator().next();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/iterate/DefaultParallelIteratorRegionSplitter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/DefaultParallelIteratorRegionSplitter.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/DefaultParallelIteratorRegionSplitter.java
new file mode 100644
index 0000000..ab194f6
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/DefaultParallelIteratorRegionSplitter.java
@@ -0,0 +1,229 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.iterate;
+
+import java.sql.SQLException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Predicate;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.ListMultimap;
+import com.google.common.collect.Lists;
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.parse.HintNode;
+import org.apache.phoenix.parse.HintNode.Hint;
+import org.apache.phoenix.query.KeyRange;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.query.StatsManager;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.util.ReadOnlyProps;
+
+
+/**
+ * Default strategy for splitting regions in ParallelIterator. Refactored from the
+ * original version.
+ * 
+ * @author jtaylor
+ * @author zhuang
+ */
+public class DefaultParallelIteratorRegionSplitter implements ParallelIteratorRegionSplitter {
+
+    protected final int targetConcurrency;
+    protected final int maxConcurrency;
+    protected final int maxIntraRegionParallelization;
+    protected final StatementContext context;
+    protected final TableRef tableRef;
+
+    public static DefaultParallelIteratorRegionSplitter getInstance(StatementContext context, TableRef table, HintNode hintNode) {
+        return new DefaultParallelIteratorRegionSplitter(context, table, hintNode);
+    }
+
+    protected DefaultParallelIteratorRegionSplitter(StatementContext context, TableRef table, HintNode hintNode) {
+        this.context = context;
+        this.tableRef = table;
+        ReadOnlyProps props = context.getConnection().getQueryServices().getProps();
+        this.targetConcurrency = props.getInt(QueryServices.TARGET_QUERY_CONCURRENCY_ATTRIB,
+                QueryServicesOptions.DEFAULT_TARGET_QUERY_CONCURRENCY);
+        this.maxConcurrency = props.getInt(QueryServices.MAX_QUERY_CONCURRENCY_ATTRIB,
+                QueryServicesOptions.DEFAULT_MAX_QUERY_CONCURRENCY);
+        Preconditions.checkArgument(targetConcurrency >= 1, "Invalid target concurrency: " + targetConcurrency);
+        Preconditions.checkArgument(maxConcurrency >= targetConcurrency , "Invalid max concurrency: " + maxConcurrency);
+        this.maxIntraRegionParallelization = hintNode.hasHint(Hint.NO_INTRA_REGION_PARALLELIZATION) ? 1 : props.getInt(QueryServices.MAX_INTRA_REGION_PARALLELIZATION_ATTRIB,
+                QueryServicesOptions.DEFAULT_MAX_INTRA_REGION_PARALLELIZATION);
+        Preconditions.checkArgument(maxIntraRegionParallelization >= 1 , "Invalid max intra region parallelization: " + maxIntraRegionParallelization);
+    }
+
+    // Get the mapping between key range and the regions that contains them.
+    protected List<HRegionLocation> getAllRegions() throws SQLException {
+        Scan scan = context.getScan();
+        PTable table = tableRef.getTable();
+        List<HRegionLocation> allTableRegions = context.getConnection().getQueryServices().getAllTableRegions(table.getPhysicalName().getBytes());
+        // If we're not salting, then we've already intersected the minMaxRange with the scan range
+        // so there's nothing to do here.
+        return filterRegions(allTableRegions, scan.getStartRow(), scan.getStopRow());
+    }
+
+    /**
+     * Filters out regions that intersect with key range specified by the startKey and stopKey
+     * @param allTableRegions all region infos for a given table
+     * @param startKey the lower bound of key range, inclusive
+     * @param stopKey the upper bound of key range, inclusive
+     * @return regions that intersect with the key range given by the startKey and stopKey
+     */
+    // exposed for tests
+    public static List<HRegionLocation> filterRegions(List<HRegionLocation> allTableRegions, byte[] startKey, byte[] stopKey) {
+        Iterable<HRegionLocation> regions;
+        final KeyRange keyRange = KeyRange.getKeyRange(startKey, true, stopKey, false);
+        if (keyRange == KeyRange.EVERYTHING_RANGE) {
+            return allTableRegions;
+        }
+        
+        regions = Iterables.filter(allTableRegions, new Predicate<HRegionLocation>() {
+            @Override
+            public boolean apply(HRegionLocation location) {
+                KeyRange regionKeyRange = KeyRange.getKeyRange(location.getRegionInfo().getStartKey(), location.getRegionInfo().getEndKey());
+                return keyRange.intersect(regionKeyRange) != KeyRange.EMPTY_RANGE;
+            }
+        });
+        return Lists.newArrayList(regions);
+    }
+
+    protected List<KeyRange> genKeyRanges(List<HRegionLocation> regions) {
+        if (regions.isEmpty()) {
+            return Collections.emptyList();
+        }
+        
+        StatsManager statsManager = context.getConnection().getQueryServices().getStatsManager();
+        // the splits are computed as follows:
+        //
+        // let's suppose:
+        // t = target concurrency
+        // m = max concurrency
+        // r = the number of regions we need to scan
+        //
+        // if r >= t:
+        //    scan using regional boundaries
+        // elif r > t/2:
+        //    split each region in s splits such that:
+        //    s = max(x) where s * x < m
+        // else:
+        //    split each region in s splits such that:
+        //    s = max(x) where s * x < t
+        //
+        // The idea is to align splits with region boundaries. If rows are not evenly
+        // distributed across regions, using this scheme compensates for regions that
+        // have more rows than others, by applying tighter splits and therefore spawning
+        // off more scans over the overloaded regions.
+        int splitsPerRegion = regions.size() >= targetConcurrency ? 1 : (regions.size() > targetConcurrency / 2 ? maxConcurrency : targetConcurrency) / regions.size();
+        splitsPerRegion = Math.min(splitsPerRegion, maxIntraRegionParallelization);
+        // Create a multi-map of ServerName to List<KeyRange> which we'll use to round robin from to ensure
+        // that we keep each region server busy for each query.
+        ListMultimap<HRegionLocation,KeyRange> keyRangesPerRegion = ArrayListMultimap.create(regions.size(),regions.size() * splitsPerRegion);;
+        if (splitsPerRegion == 1) {
+            for (HRegionLocation region : regions) {
+                keyRangesPerRegion.put(region, ParallelIterators.TO_KEY_RANGE.apply(region));
+            }
+        } else {
+            // Maintain bucket for each server and then returns KeyRanges in round-robin
+            // order to ensure all servers are utilized.
+            for (HRegionLocation region : regions) {
+                byte[] startKey = region.getRegionInfo().getStartKey();
+                byte[] stopKey = region.getRegionInfo().getEndKey();
+                boolean lowerUnbound = Bytes.compareTo(startKey, HConstants.EMPTY_START_ROW) == 0;
+                boolean upperUnbound = Bytes.compareTo(stopKey, HConstants.EMPTY_END_ROW) == 0;
+                /*
+                 * If lower/upper unbound, get the min/max key from the stats manager.
+                 * We use this as the boundary to split on, but we still use the empty
+                 * byte as the boundary in the actual scan (in case our stats are out
+                 * of date).
+                 */
+                if (lowerUnbound) {
+                    startKey = statsManager.getMinKey(tableRef);
+                    if (startKey == null) {
+                        keyRangesPerRegion.put(region,ParallelIterators.TO_KEY_RANGE.apply(region));
+                        continue;
+                    }
+                }
+                if (upperUnbound) {
+                    stopKey = statsManager.getMaxKey(tableRef);
+                    if (stopKey == null) {
+                        keyRangesPerRegion.put(region,ParallelIterators.TO_KEY_RANGE.apply(region));
+                        continue;
+                    }
+                }
+                
+                byte[][] boundaries = null;
+                // Both startKey and stopKey will be empty the first time
+                if (Bytes.compareTo(startKey, stopKey) >= 0 || (boundaries = Bytes.split(startKey, stopKey, splitsPerRegion - 1)) == null) {
+                    // Bytes.split may return null if the key space
+                    // between start and end key is too small
+                    keyRangesPerRegion.put(region,ParallelIterators.TO_KEY_RANGE.apply(region));
+                } else {
+                    keyRangesPerRegion.put(region,KeyRange.getKeyRange(lowerUnbound ? KeyRange.UNBOUND : boundaries[0], boundaries[1]));
+                    if (boundaries.length > 1) {
+                        for (int i = 1; i < boundaries.length-2; i++) {
+                            keyRangesPerRegion.put(region,KeyRange.getKeyRange(boundaries[i], true, boundaries[i+1], false));
+                        }
+                        keyRangesPerRegion.put(region,KeyRange.getKeyRange(boundaries[boundaries.length-2], true, upperUnbound ? KeyRange.UNBOUND : boundaries[boundaries.length-1], false));
+                    }
+                }
+            }
+        }
+        List<KeyRange> splits = Lists.newArrayListWithCapacity(regions.size() * splitsPerRegion);
+        // as documented for ListMultimap
+        Collection<Collection<KeyRange>> values = keyRangesPerRegion.asMap().values();
+        List<Collection<KeyRange>> keyRangesList = Lists.newArrayList(values);
+        // Randomize range order to ensure that we don't hit the region servers in the same order every time
+        // thus helping to distribute the load more evenly
+        Collections.shuffle(keyRangesList);
+        // Transpose values in map to get regions in round-robin server order. This ensures that
+        // all servers will be used to process the set of parallel threads available in our executor.
+        int i = 0;
+        boolean done;
+        do {
+            done = true;
+            for (int j = 0; j < keyRangesList.size(); j++) {
+                List<KeyRange> keyRanges = (List<KeyRange>)keyRangesList.get(j);
+                if (i < keyRanges.size()) {
+                    splits.add(keyRanges.get(i));
+                    done = false;
+                }
+            }
+            i++;
+        } while (!done);
+        return splits;
+    }
+
+    @Override
+    public List<KeyRange> getSplits() throws SQLException {
+        return genKeyRanges(getAllRegions());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/iterate/DelegateResultIterator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/DelegateResultIterator.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/DelegateResultIterator.java
new file mode 100644
index 0000000..25df5ad
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/DelegateResultIterator.java
@@ -0,0 +1,50 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.iterate;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.phoenix.schema.tuple.Tuple;
+
+
+public class DelegateResultIterator implements ResultIterator {
+    private final ResultIterator delegate;
+    
+    public DelegateResultIterator(ResultIterator delegate) {
+        this.delegate = delegate;
+    }
+    
+    @Override
+    public void close() throws SQLException {
+        delegate.close();
+    }
+
+    @Override
+    public Tuple next() throws SQLException {
+        return delegate.next();
+    }
+
+    @Override
+    public void explain(List<String> planSteps) {
+        delegate.explain(planSteps);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/iterate/DistinctAggregatingResultIterator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/DistinctAggregatingResultIterator.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/DistinctAggregatingResultIterator.java
new file mode 100644
index 0000000..f7af26c
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/DistinctAggregatingResultIterator.java
@@ -0,0 +1,163 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.iterate;
+
+import java.sql.SQLException;
+import java.util.*;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Sets;
+import org.apache.phoenix.compile.ColumnProjector;
+import org.apache.phoenix.compile.RowProjector;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+/**
+ * Result scanner that dedups the incoming tuples to make them distinct.
+ * <p>
+ * Note that the results are held in memory
+ *  
+ * @author jtaylor
+ * @since 1.2
+ */
+public class DistinctAggregatingResultIterator implements AggregatingResultIterator {
+    private final AggregatingResultIterator delegate;
+    private final RowProjector rowProjector;
+    private Iterator<ResultEntry> resultIterator;
+    private final ImmutableBytesWritable ptr1 = new ImmutableBytesWritable();
+    private final ImmutableBytesWritable ptr2 = new ImmutableBytesWritable();
+
+    private class ResultEntry {
+        private final int hashCode;
+        private final Tuple result;
+
+        ResultEntry(Tuple result) {
+            final int prime = 31;
+            this.result = result;
+            int hashCode = 0;
+            for (ColumnProjector column : rowProjector.getColumnProjectors()) {
+                Expression e = column.getExpression();
+                if (e.evaluate(this.result, ptr1)) {
+                    hashCode = prime * hashCode + ptr1.hashCode();
+                }
+            }
+            this.hashCode = hashCode;
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (o == this) {
+                return true;
+            }
+            if (o == null) {
+                return false;
+            }
+            if (o.getClass() != this.getClass()) {
+                return false;
+            }
+            ResultEntry that = (ResultEntry) o;
+            for (ColumnProjector column : rowProjector.getColumnProjectors()) {
+                Expression e = column.getExpression();
+                boolean isNull1 = !e.evaluate(this.result, ptr1);
+                boolean isNull2 = !e.evaluate(that.result, ptr2);
+                if (isNull1 && isNull2) {
+                    return true;
+                }
+                if (isNull1 || isNull2) {
+                    return false;
+                }
+                if (ptr1.compareTo(ptr2) != 0) {
+                    return false;
+                }
+            }
+            return true;
+        }
+        
+        @Override
+        public int hashCode() {
+            return hashCode;
+        }
+        
+        Tuple getResult() {
+            return result;
+        }
+    }
+    
+    protected ResultIterator getDelegate() {
+        return delegate;
+    }
+    
+    public DistinctAggregatingResultIterator(AggregatingResultIterator delegate,
+            RowProjector rowProjector) {
+        this.delegate = delegate;
+        this.rowProjector = rowProjector;
+    }
+
+    @Override
+    public Tuple next() throws SQLException {
+        Iterator<ResultEntry> iterator = getResultIterator();
+        if (iterator.hasNext()) {
+            ResultEntry entry = iterator.next();
+            Tuple tuple = entry.getResult();
+            aggregate(tuple);
+            return tuple;
+        }
+        resultIterator = Iterators.emptyIterator();
+        return null;
+    }
+    
+    private Iterator<ResultEntry> getResultIterator() throws SQLException {
+        if (resultIterator != null) {
+            return resultIterator;
+        }
+        
+        Set<ResultEntry> entries = Sets.<ResultEntry>newHashSet(); // TODO: size?
+        try {
+            for (Tuple result = delegate.next(); result != null; result = delegate.next()) {
+                ResultEntry entry = new ResultEntry(result);
+                entries.add(entry);
+            }
+        } finally {
+            delegate.close();
+        }
+        
+        resultIterator = entries.iterator();
+        return resultIterator;
+    }
+
+    @Override
+    public void close()  {
+        resultIterator = Iterators.emptyIterator();
+    }
+
+
+    @Override
+    public void explain(List<String> planSteps) {
+        delegate.explain(planSteps);
+        planSteps.add("CLIENT DISTINCT ON " + rowProjector.toString());
+    }
+
+    @Override
+    public void aggregate(Tuple result) {
+        delegate.aggregate(result);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/iterate/ExplainTable.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/ExplainTable.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/ExplainTable.java
new file mode 100644
index 0000000..f32b377
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/ExplainTable.java
@@ -0,0 +1,271 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.iterate;
+
+import java.text.Format;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
+import org.apache.hadoop.hbase.filter.PageFilter;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import com.google.common.collect.Iterators;
+import org.apache.phoenix.compile.GroupByCompiler.GroupBy;
+import org.apache.phoenix.compile.ScanRanges;
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.query.KeyRange;
+import org.apache.phoenix.query.KeyRange.Bound;
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.RowKeySchema;
+import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.util.StringUtil;
+
+
+public abstract class ExplainTable {
+    private static final List<KeyRange> EVERYTHING = Collections.singletonList(KeyRange.EVERYTHING_RANGE);
+    protected final StatementContext context;
+    protected final TableRef tableRef;
+    protected final GroupBy groupBy;
+   
+    public ExplainTable(StatementContext context, TableRef table) {
+        this(context,table,GroupBy.EMPTY_GROUP_BY);
+    }
+
+    public ExplainTable(StatementContext context, TableRef table, GroupBy groupBy) {
+        this.context = context;
+        this.tableRef = table;
+        this.groupBy = groupBy;
+    }
+
+    private boolean explainSkipScan(StringBuilder buf) {
+        ScanRanges scanRanges = context.getScanRanges();
+        if (scanRanges.useSkipScanFilter()) {
+            buf.append("SKIP SCAN ");
+            int count = 1;
+            boolean hasRanges = false;
+            for (List<KeyRange> ranges : scanRanges.getRanges()) {
+                count *= ranges.size();
+                for (KeyRange range : ranges) {
+                    hasRanges |= !range.isSingleKey();
+                }
+            }
+            buf.append("ON ");
+            buf.append(count);
+            buf.append(hasRanges ? " RANGE" : " KEY");
+            buf.append(count > 1 ? "S " : " ");
+            return true;
+        } else {
+            buf.append("RANGE SCAN ");
+        }
+        return false;
+    }
+    
+    protected void explain(String prefix, List<String> planSteps) {
+        StringBuilder buf = new StringBuilder(prefix);
+        ScanRanges scanRanges = context.getScanRanges();
+        boolean hasSkipScanFilter = false;
+        if (scanRanges.isEverything()) {
+            buf.append("FULL SCAN ");
+        } else {
+            hasSkipScanFilter = explainSkipScan(buf);
+        }
+        buf.append("OVER " + tableRef.getTable().getName().getString());
+        appendKeyRanges(buf);
+        planSteps.add(buf.toString());
+        
+        Scan scan = context.getScan();
+        Filter filter = scan.getFilter();
+        PageFilter pageFilter = null;
+        if (filter != null) {
+            int offset = 0;
+            boolean hasFirstKeyOnlyFilter = false;
+            String filterDesc = "";
+            if (hasSkipScanFilter) {
+                if (filter instanceof FilterList) {
+                    List<Filter> filterList = ((FilterList) filter).getFilters();
+                    if (filterList.get(0) instanceof FirstKeyOnlyFilter) {
+                        hasFirstKeyOnlyFilter = true;
+                        offset = 1;
+                    }
+                    if (filterList.size() > offset+1) {
+                        filterDesc = filterList.get(offset+1).toString();
+                        if (filterList.size() > offset+2) {
+                            pageFilter = (PageFilter) filterList.get(offset+2);
+                        }
+                    }
+                }
+            } else if (filter instanceof FilterList) {
+                List<Filter> filterList = ((FilterList) filter).getFilters();
+                if (filterList.get(0) instanceof FirstKeyOnlyFilter) {
+                    hasFirstKeyOnlyFilter = true;
+                    offset = 1;
+                }
+                if (filterList.size() > offset) {
+                    filterDesc = filterList.get(offset).toString();
+                    if (filterList.size() > offset+1) {
+                        pageFilter = (PageFilter) filterList.get(offset+1);
+                    }
+                }
+            } else {
+                if (filter instanceof FirstKeyOnlyFilter) {
+                    hasFirstKeyOnlyFilter = true;
+                } else {
+                    filterDesc = filter.toString();
+                }
+            }
+            if (filterDesc.length() > 0) {
+                planSteps.add("    SERVER FILTER BY " + (hasFirstKeyOnlyFilter ? "FIRST KEY ONLY AND " : "") + filterDesc);
+            } else if (hasFirstKeyOnlyFilter) {
+                planSteps.add("    SERVER FILTER BY FIRST KEY ONLY");
+            }
+            if (pageFilter != null) {
+                planSteps.add("    SERVER " + pageFilter.getPageSize() + " ROW LIMIT");
+            }
+        }
+        groupBy.explain(planSteps);
+    }
+
+    private void appendPKColumnValue(StringBuilder buf, byte[] range, Boolean isNull, int slotIndex) {
+        if (Boolean.TRUE.equals(isNull)) {
+            buf.append("null");
+            return;
+        }
+        if (Boolean.FALSE.equals(isNull)) {
+            buf.append("not null");
+            return;
+        }
+        if (range.length == 0) {
+            buf.append('*');
+            return;
+        }
+        ScanRanges scanRanges = context.getScanRanges();
+        PDataType type = scanRanges.getSchema().getField(slotIndex).getDataType();
+        ColumnModifier modifier = tableRef.getTable().getPKColumns().get(slotIndex).getColumnModifier();
+        if (modifier != null) {
+            buf.append('~');
+            range = modifier.apply(range, 0, new byte[range.length], 0, range.length);
+        }
+        Format formatter = context.getConnection().getFormatter(type);
+        buf.append(type.toStringLiteral(range, formatter));
+    }
+    
+    private static class RowKeyValueIterator implements Iterator<byte[]> {
+        private final RowKeySchema schema;
+        private ImmutableBytesWritable ptr = new ImmutableBytesWritable();
+        private int position = 0;
+        private final int maxOffset;
+        private byte[] nextValue;
+       
+        public RowKeyValueIterator(RowKeySchema schema, byte[] rowKey) {
+            this.schema = schema;
+            this.maxOffset = schema.iterator(rowKey, ptr);
+            iterate();
+        }
+        
+        private void iterate() {
+            if (schema.next(ptr, position++, maxOffset) == null) {
+                nextValue = null;
+            } else {
+                nextValue = ptr.copyBytes();
+            }
+        }
+        
+        @Override
+        public boolean hasNext() {
+            return nextValue != null;
+        }
+
+        @Override
+        public byte[] next() {
+            if (nextValue == null) {
+                throw new NoSuchElementException();
+            }
+            byte[] value = nextValue;
+            iterate();
+            return value;
+        }
+
+        @Override
+        public void remove() {
+            throw new UnsupportedOperationException();
+        }
+        
+    }
+    
+    private void appendScanRow(StringBuilder buf, Bound bound) {
+        ScanRanges scanRanges = context.getScanRanges();
+        KeyRange minMaxRange = context.getMinMaxRange();
+        Iterator<byte[]> minMaxIterator = Iterators.emptyIterator();
+        if (minMaxRange != null) {
+            RowKeySchema schema = tableRef.getTable().getRowKeySchema();
+            if (!minMaxRange.isUnbound(bound)) {
+                minMaxIterator = new RowKeyValueIterator(schema, minMaxRange.getRange(bound));
+            }
+        }
+        int nRanges = scanRanges.getRanges().size();
+        for (int i = 0, minPos = 0; minPos < nRanges || minMaxIterator.hasNext(); i++) {
+            List<KeyRange> ranges = minPos >= nRanges ? EVERYTHING :  scanRanges.getRanges().get(minPos++);
+            KeyRange range = bound == Bound.LOWER ? ranges.get(0) : ranges.get(ranges.size()-1);
+            byte[] b = range.getRange(bound);
+            Boolean isNull = KeyRange.IS_NULL_RANGE == range ? Boolean.TRUE : KeyRange.IS_NOT_NULL_RANGE == range ? Boolean.FALSE : null;
+            if (minMaxIterator.hasNext()) {
+                byte[] bMinMax = minMaxIterator.next();
+                int cmp = Bytes.compareTo(bMinMax, b) * (bound == Bound.LOWER ? 1 : -1);
+                if (cmp > 0) {
+                    minPos = nRanges;
+                    b = bMinMax;
+                    isNull = null;
+                } else if (cmp < 0) {
+                    minMaxIterator = Iterators.emptyIterator();
+                }
+            }
+            appendPKColumnValue(buf, b, isNull, i);
+            buf.append(',');
+        }
+    }
+    
+    private void appendKeyRanges(StringBuilder buf) {
+        ScanRanges scanRanges = context.getScanRanges();
+        KeyRange minMaxRange = context.getMinMaxRange();
+        if (minMaxRange == null && (scanRanges == ScanRanges.EVERYTHING || scanRanges == ScanRanges.NOTHING)) {
+            return;
+        }
+        buf.append(" [");
+        StringBuilder buf1 = new StringBuilder();
+        appendScanRow(buf1, Bound.LOWER);
+        buf.append(buf1);
+        buf.setCharAt(buf.length()-1, ']');
+        StringBuilder buf2 = new StringBuilder();
+        appendScanRow(buf2, Bound.UPPER);
+        if (!StringUtil.equals(buf1, buf2)) {
+            buf.append( " - [");
+            buf.append(buf2);
+        }
+        buf.setCharAt(buf.length()-1, ']');
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/iterate/FilterAggregatingResultIterator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/FilterAggregatingResultIterator.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/FilterAggregatingResultIterator.java
new file mode 100644
index 0000000..d687480
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/FilterAggregatingResultIterator.java
@@ -0,0 +1,80 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.iterate;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+
+/**
+ * 
+ * Post aggregation filter for HAVING clause. Due to the way we cache aggregation values
+ * we cannot have a look ahead for this Iterator, because the expressions in the SELECT
+ * clause would return values for the peeked row instead of the current row. If we only
+ * use the Result argument in {@link org.apache.phoenix.expression.Expression}
+ * instead of our cached value in Aggregators, we could have a look ahead.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class FilterAggregatingResultIterator  implements AggregatingResultIterator {
+    private final AggregatingResultIterator delegate;
+    private final Expression expression;
+    private final ImmutableBytesWritable ptr = new ImmutableBytesWritable();
+    
+    public FilterAggregatingResultIterator(AggregatingResultIterator delegate, Expression expression) {
+        this.delegate = delegate;
+        this.expression = expression;
+        if (expression.getDataType() != PDataType.BOOLEAN) {
+            throw new IllegalArgumentException("FilterResultIterator requires a boolean expression, but got " + expression);
+        }
+    }
+
+    @Override
+    public Tuple next() throws SQLException {
+        Tuple next;
+        do {
+            next = delegate.next();
+        } while (next != null && expression.evaluate(next, ptr) && Boolean.FALSE.equals(expression.getDataType().toObject(ptr)));
+        return next;
+    }
+
+    @Override
+    public void close() throws SQLException {
+        delegate.close();
+    }
+
+    @Override
+    public void aggregate(Tuple result) {
+        delegate.aggregate(result);
+    }
+
+    @Override
+    public void explain(List<String> planSteps) {
+        delegate.explain(planSteps);
+        planSteps.add("CLIENT FILTER BY " + expression.toString());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/iterate/FilterResultIterator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/FilterResultIterator.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/FilterResultIterator.java
new file mode 100644
index 0000000..5518643
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/FilterResultIterator.java
@@ -0,0 +1,78 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.iterate;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+
+/**
+ * 
+ * Result scanner that filters out rows based on the results of a boolean
+ * expression (i.e. filters out if {@link org.apache.phoenix.expression.Expression#evaluate(Tuple, ImmutableBytesWritable)}
+ * returns false or the ptr contains a FALSE value}). May not be used where
+ * the delegate provided is an {@link org.apache.phoenix.iterate.AggregatingResultIterator}.
+ * For these, the {@link org.apache.phoenix.iterate.FilterAggregatingResultIterator} should be used.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class FilterResultIterator  extends LookAheadResultIterator {
+    private final ResultIterator delegate;
+    private final Expression expression;
+    private final ImmutableBytesWritable ptr = new ImmutableBytesWritable();
+    
+    public FilterResultIterator(ResultIterator delegate, Expression expression) {
+        if (delegate instanceof AggregatingResultIterator) {
+            throw new IllegalArgumentException("FilterResultScanner may not be used with an aggregate delegate. Use phoenix.iterate.FilterAggregateResultScanner instead");
+        }
+        this.delegate = delegate;
+        this.expression = expression;
+        if (expression.getDataType() != PDataType.BOOLEAN) {
+            throw new IllegalArgumentException("FilterResultIterator requires a boolean expression, but got " + expression);
+        }
+    }
+
+    @Override
+    protected Tuple advance() throws SQLException {
+        Tuple next;
+        do {
+            next = delegate.next();
+        } while (next != null && expression.evaluate(next, ptr) && Boolean.FALSE.equals(expression.getDataType().toObject(ptr)));
+        return next;
+    }
+    
+    @Override
+    public void close() throws SQLException {
+        delegate.close();
+    }
+
+    @Override
+    public void explain(List<String> planSteps) {
+        delegate.explain(planSteps);
+        planSteps.add("CLIENT FILTER BY " + expression.toString());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/iterate/GroupedAggregatingResultIterator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/GroupedAggregatingResultIterator.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/GroupedAggregatingResultIterator.java
new file mode 100644
index 0000000..7985699
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/GroupedAggregatingResultIterator.java
@@ -0,0 +1,106 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.iterate;
+
+import static org.apache.phoenix.query.QueryConstants.*;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.expression.aggregator.Aggregator;
+import org.apache.phoenix.expression.aggregator.Aggregators;
+import org.apache.phoenix.schema.tuple.SingleKeyValueTuple;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.KeyValueUtil;
+import org.apache.phoenix.util.TupleUtil;
+
+
+
+/**
+ * 
+ * Result scanner that aggregates the row count value for rows with duplicate keys.
+ * The rows from the backing result iterator must be in key sorted order.  For example,
+ * given the following input:
+ *   a  1
+ *   a  2
+ *   b  1
+ *   b  3
+ *   c  1
+ * the following will be output:
+ *   a  3
+ *   b  4
+ *   c  1
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class GroupedAggregatingResultIterator implements AggregatingResultIterator {
+    private final ImmutableBytesWritable tempPtr = new ImmutableBytesWritable();
+    private final PeekingResultIterator resultIterator;
+    protected final Aggregators aggregators;
+    
+    public GroupedAggregatingResultIterator( PeekingResultIterator resultIterator, Aggregators aggregators) {
+        if (resultIterator == null) throw new NullPointerException();
+        if (aggregators == null) throw new NullPointerException();
+        this.resultIterator = resultIterator;
+        this.aggregators = aggregators;
+    }
+    
+    @Override
+    public Tuple next() throws SQLException {
+        Tuple result = resultIterator.next();
+        if (result == null) {
+            return null;
+        }
+        Aggregator[] rowAggregators = aggregators.getAggregators();
+        aggregators.reset(rowAggregators);
+        while (true) {
+            aggregators.aggregate(rowAggregators, result);
+            Tuple nextResult = resultIterator.peek();
+            if (nextResult == null || !TupleUtil.equals(result, nextResult, tempPtr)) {
+                break;
+            }
+            result = resultIterator.next();
+        }
+        
+        byte[] value = aggregators.toBytes(rowAggregators);
+        result.getKey(tempPtr);
+        return new SingleKeyValueTuple(KeyValueUtil.newKeyValue(tempPtr, SINGLE_COLUMN_FAMILY, SINGLE_COLUMN, AGG_TIMESTAMP, value, 0, value.length));
+    }
+    
+    @Override
+    public void close() throws SQLException {
+        resultIterator.close();
+    }
+    
+    @Override
+    public void aggregate(Tuple result) {
+        Aggregator[] rowAggregators = aggregators.getAggregators();
+        aggregators.reset(rowAggregators);
+        aggregators.aggregate(rowAggregators, result);
+    }
+
+    @Override
+    public void explain(List<String> planSteps) {
+        resultIterator.explain(planSteps);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/iterate/LimitingResultIterator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/LimitingResultIterator.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/LimitingResultIterator.java
new file mode 100644
index 0000000..ecea92d
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/LimitingResultIterator.java
@@ -0,0 +1,57 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.iterate;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.phoenix.schema.tuple.Tuple;
+
+/**
+ * 
+ * Iterates through tuples up to a limit
+ *
+ * @author jtaylor
+ * @since 1.2
+ */
+public class LimitingResultIterator extends DelegateResultIterator {
+    private int rowCount;
+    private final int limit;
+    
+    public LimitingResultIterator(ResultIterator delegate, int limit) {
+        super(delegate);
+        this.limit = limit;
+    }
+
+    @Override
+    public Tuple next() throws SQLException {
+        if (rowCount++ >= limit) {
+            return null;
+        }
+        return super.next();
+    }
+
+    @Override
+    public void explain(List<String> planSteps) {
+        super.explain(planSteps);
+        planSteps.add("CLIENT " + limit + " ROW LIMIT");
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..0e4b78b
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/LookAheadResultIterator.java
@@ -0,0 +1,53 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.iterate;
+
+import java.sql.SQLException;
+
+import org.apache.phoenix.schema.tuple.ResultTuple;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+
+abstract public class LookAheadResultIterator implements PeekingResultIterator {
+    private final static Tuple UNINITIALIZED = new ResultTuple();
+    private Tuple next = UNINITIALIZED;
+    
+    abstract protected Tuple advance() throws SQLException;
+    
+    private void init() throws SQLException {
+        if (next == UNINITIALIZED) {
+            next = advance();
+        }
+    }
+    
+    @Override
+    public Tuple next() throws SQLException {
+        init();
+        Tuple next = this.next;
+        this.next = advance();
+        return next;
+    }
+    
+    @Override
+    public Tuple peek() throws SQLException {
+        init();
+        return next;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/iterate/MappedByteBufferSortedQueue.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/MappedByteBufferSortedQueue.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/MappedByteBufferSortedQueue.java
new file mode 100644
index 0000000..1606ae6
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/MappedByteBufferSortedQueue.java
@@ -0,0 +1,389 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.iterate;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.MappedByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.channels.FileChannel.MapMode;
+import java.util.AbstractQueue;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.UUID;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import com.google.common.collect.MinMaxPriorityQueue;
+import org.apache.phoenix.iterate.OrderedResultIterator.ResultEntry;
+import org.apache.phoenix.schema.tuple.ResultTuple;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+public class MappedByteBufferSortedQueue extends AbstractQueue<ResultEntry> {
+    private Comparator<ResultEntry> comparator;
+    private final int limit;
+    private final int thresholdBytes;
+    private List<MappedByteBufferPriorityQueue> queues = new ArrayList<MappedByteBufferPriorityQueue>();
+    private MappedByteBufferPriorityQueue currentQueue = null;
+    private int currentIndex = 0;
+    MinMaxPriorityQueue<IndexedResultEntry> mergedQueue = null;
+
+    public MappedByteBufferSortedQueue(Comparator<ResultEntry> comparator,
+            Integer limit, int thresholdBytes) throws IOException {
+        this.comparator = comparator;
+        this.limit = limit == null ? -1 : limit;
+        this.thresholdBytes = thresholdBytes;
+        this.currentQueue = new MappedByteBufferPriorityQueue(0,
+                this.limit, thresholdBytes, comparator);
+        this.queues.add(currentQueue);
+    }
+
+    @Override
+    public boolean offer(ResultEntry e) {
+        try {
+            boolean isFlush = this.currentQueue.writeResult(e);
+            if (isFlush) {
+                currentIndex++;
+                currentQueue = new MappedByteBufferPriorityQueue(currentIndex,
+                        limit, thresholdBytes, comparator);
+                queues.add(currentQueue);
+            }
+        } catch (IOException ioe) {
+            throw new RuntimeException(ioe);
+        }
+
+        return true;
+    }
+
+    @Override
+    public ResultEntry poll() {
+        if (mergedQueue == null) {
+            mergedQueue = MinMaxPriorityQueue.<ResultEntry> orderedBy(
+                    comparator).maximumSize(queues.size()).create();
+            for (MappedByteBufferPriorityQueue queue : queues) {
+                try {
+                    IndexedResultEntry next = queue.getNextResult();
+                    if (next != null) {
+                        mergedQueue.add(next);
+                    }
+                } catch (IOException e) {
+                    throw new RuntimeException(e);
+                }
+            }
+        }
+        if (!mergedQueue.isEmpty()) {
+            IndexedResultEntry re = mergedQueue.pollFirst();
+            if (re != null) {
+                IndexedResultEntry next = null;
+                try {
+                    next = queues.get(re.getIndex()).getNextResult();
+                } catch (IOException e) {
+                    throw new RuntimeException(e);
+                }
+                if (next != null) {
+                    mergedQueue.add(next);
+                }
+                return re;
+            }
+        }
+        return null;
+    }
+
+    @Override
+    public ResultEntry peek() {
+        if (mergedQueue == null) {
+            mergedQueue = MinMaxPriorityQueue.<ResultEntry> orderedBy(
+                    comparator).maximumSize(queues.size()).create();
+            for (MappedByteBufferPriorityQueue queue : queues) {
+                try {
+                    IndexedResultEntry next = queue.getNextResult();
+                    if (next != null) {
+                        mergedQueue.add(next);
+                    }
+                } catch (IOException e) {
+                    throw new RuntimeException(e);
+                }
+            }
+        }
+        if (!mergedQueue.isEmpty()) {
+            IndexedResultEntry re = mergedQueue.peekFirst();
+            if (re != null) {
+                return re;
+            }
+        }
+        return null;
+    }
+
+    @Override
+    public Iterator<ResultEntry> iterator() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public int size() {
+        int size = 0;
+        for (MappedByteBufferPriorityQueue queue : queues) {
+            size += queue.size();
+        }
+        return size;
+    }
+    
+    public long getByteSize() {
+        return currentQueue.getInMemByteSize();
+    }
+
+    public void close() {
+        if (queues != null) {
+            for (MappedByteBufferPriorityQueue queue : queues) {
+                queue.close();
+            }
+        }
+    }
+
+    private static class IndexedResultEntry extends ResultEntry {
+        private int index;
+
+        public IndexedResultEntry(int index, ResultEntry resultEntry) {
+            super(resultEntry.sortKeys, resultEntry.result);
+            this.index = index;
+        }
+
+        public int getIndex() {
+            return this.index;
+        }
+    }
+
+    private static class MappedByteBufferPriorityQueue {
+        private static final long DEFAULT_MAPPING_SIZE = 1024;
+        
+        private final int limit;
+        private final int thresholdBytes;
+        private long totalResultSize = 0;
+        private int maxResultSize = 0;
+        private long mappingSize = 0;
+        private long writeIndex = 0;
+        private long readIndex = 0;
+        private MappedByteBuffer writeBuffer;
+        private MappedByteBuffer readBuffer;
+        private FileChannel fc;
+        private RandomAccessFile af;
+        private File file;
+        private boolean isClosed = false;
+        MinMaxPriorityQueue<ResultEntry> results = null;
+        private boolean flushBuffer = false;
+        private int index;
+        private int flushedCount;
+
+        public MappedByteBufferPriorityQueue(int index, int limit, int thresholdBytes,
+                Comparator<ResultEntry> comparator) throws IOException {
+            this.index = index;
+            this.limit = limit;
+            this.thresholdBytes = thresholdBytes;
+            results = limit < 0 ? 
+                    MinMaxPriorityQueue.<ResultEntry> orderedBy(comparator).create()
+                  : MinMaxPriorityQueue.<ResultEntry> orderedBy(comparator).maximumSize(limit).create();
+        }
+        
+        public int size() {
+            if (flushBuffer)
+                return flushedCount;
+            return results.size();
+        }
+        
+        public long getInMemByteSize() {
+            if (flushBuffer)
+                return 0;
+            return totalResultSize;
+        }
+
+        private List<KeyValue> toKeyValues(ResultEntry entry) {
+            Tuple result = entry.getResult();
+            int size = result.size();
+            List<KeyValue> kvs = new ArrayList<KeyValue>(size);
+            for (int i = 0; i < size; i++) {
+                kvs.add(result.getValue(i));
+            }
+            return kvs;
+        }
+
+        private int sizeof(List<KeyValue> kvs) {
+            int size = Bytes.SIZEOF_INT; // totalLen
+
+            for (KeyValue kv : kvs) {
+                size += kv.getLength();
+                size += Bytes.SIZEOF_INT; // kv.getLength
+            }
+
+            return size;
+        }
+
+        private int sizeof(ImmutableBytesWritable[] sortKeys) {
+            int size = Bytes.SIZEOF_INT;
+            if (sortKeys != null) {
+                for (ImmutableBytesWritable sortKey : sortKeys) {
+                    if (sortKey != null) {
+                        size += sortKey.getLength();
+                    }
+                    size += Bytes.SIZEOF_INT;
+                }
+            }
+            return size;
+        }
+
+        public boolean writeResult(ResultEntry entry) throws IOException {
+            if (flushBuffer)
+                throw new IOException("Results already flushed");
+            
+            int sortKeySize = sizeof(entry.sortKeys);
+            int resultSize = sizeof(toKeyValues(entry)) + sortKeySize;
+            boolean added = results.add(entry);
+            if (added) {
+                maxResultSize = Math.max(maxResultSize, resultSize);
+                totalResultSize = limit < 0 ? (totalResultSize + resultSize) : maxResultSize * results.size();
+                if (totalResultSize >= thresholdBytes) {
+                    this.file = File.createTempFile(UUID.randomUUID().toString(), null);
+                    this.af = new RandomAccessFile(file, "rw");
+                    this.fc = af.getChannel();
+                    mappingSize = Math.min(Math.max(maxResultSize, DEFAULT_MAPPING_SIZE), totalResultSize);
+                    writeBuffer = fc.map(MapMode.READ_WRITE, writeIndex, mappingSize);
+                
+                    int resSize = results.size();
+                    for (int i = 0; i < resSize; i++) {                
+                        int totalLen = 0;
+                        ResultEntry re = results.pollFirst();
+                        List<KeyValue> keyValues = toKeyValues(re);
+                        for (KeyValue kv : keyValues) {
+                            totalLen += (kv.getLength() + Bytes.SIZEOF_INT);
+                        }
+                        writeBuffer.putInt(totalLen);
+                        for (KeyValue kv : keyValues) {
+                            writeBuffer.putInt(kv.getLength());
+                            writeBuffer.put(kv.getBuffer(), kv.getOffset(), kv
+                                    .getLength());
+                        }
+                        ImmutableBytesWritable[] sortKeys = re.sortKeys;
+                        writeBuffer.putInt(sortKeys.length);
+                        for (ImmutableBytesWritable sortKey : sortKeys) {
+                            if (sortKey != null) {
+                                writeBuffer.putInt(sortKey.getLength());
+                                writeBuffer.put(sortKey.get(), sortKey.getOffset(),
+                                        sortKey.getLength());
+                            } else {
+                                writeBuffer.putInt(0);
+                            }
+                        }
+                        // buffer close to exhausted, re-map.
+                        if (mappingSize - writeBuffer.position() < maxResultSize) {
+                            writeIndex += writeBuffer.position();
+                            writeBuffer = fc.map(MapMode.READ_WRITE, writeIndex, mappingSize);
+                        }
+                    }
+                    writeBuffer.putInt(-1); // end
+                    flushedCount = results.size();
+                    results.clear();
+                    flushBuffer = true;
+                }
+            }
+            return flushBuffer;
+        }
+
+        public IndexedResultEntry getNextResult() throws IOException {
+            if (isClosed)
+                return null;
+            
+            if (!flushBuffer) {
+                ResultEntry re = results.poll();
+                if (re == null) {
+                    reachedEnd();
+                    return null;
+                }
+                return new IndexedResultEntry(index, re);
+            }
+            
+            if (readBuffer == null) {
+                readBuffer = this.fc.map(MapMode.READ_ONLY, readIndex, mappingSize);
+            }
+            
+            int length = readBuffer.getInt();
+            if (length < 0) {
+                reachedEnd();
+                return null;
+            }
+            
+            byte[] rb = new byte[length];
+            readBuffer.get(rb);
+            Result result = new Result(new ImmutableBytesWritable(rb));
+            ResultTuple rt = new ResultTuple(result);
+            int sortKeySize = readBuffer.getInt();
+            ImmutableBytesWritable[] sortKeys = new ImmutableBytesWritable[sortKeySize];
+            for (int i = 0; i < sortKeySize; i++) {
+                int contentLength = readBuffer.getInt();
+                if (contentLength > 0) {
+                    byte[] sortKeyContent = new byte[contentLength];
+                    readBuffer.get(sortKeyContent);
+                    sortKeys[i] = new ImmutableBytesWritable(sortKeyContent);
+                } else {
+                    sortKeys[i] = null;
+                }
+            }
+            // buffer close to exhausted, re-map.
+            if (mappingSize - readBuffer.position() < maxResultSize) {
+                readIndex += readBuffer.position();
+                readBuffer = fc.map(MapMode.READ_ONLY, readIndex, mappingSize);
+            }
+            
+            return new IndexedResultEntry(index, new ResultEntry(sortKeys, rt));
+        }
+
+        private void reachedEnd() {
+            this.isClosed = true;
+            if (this.fc != null) {
+                try {
+                    this.fc.close();
+                } catch (IOException ignored) {
+                }
+                this.fc = null;
+            }
+            if (this.af != null) {
+                try {
+                    this.af.close();
+                } catch (IOException ignored) {
+                }
+                this.af = null;
+            }
+            if (this.file != null) {
+                file.delete();
+                file = null;
+            }
+        }
+
+        public void close() {
+            if (!isClosed) {
+                this.reachedEnd();
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/iterate/MaterializedResultIterator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/MaterializedResultIterator.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/MaterializedResultIterator.java
new file mode 100644
index 0000000..c509b2c
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/MaterializedResultIterator.java
@@ -0,0 +1,112 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.iterate;
+
+import java.sql.SQLException;
+import java.util.*;
+
+import org.apache.phoenix.schema.tuple.Tuple;
+
+
+/**
+ * 
+ * Fully materialized result iterator backed by the result list provided.
+ * No copy is made of the backing results collection.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class MaterializedResultIterator implements PeekingResultIterator {
+    private final PeekingCollectionIterator iterator;
+    
+    public MaterializedResultIterator(Collection<Tuple> results) {
+        iterator = new PeekingCollectionIterator(results);
+    }
+    
+    @Override
+    public void close() {
+    }
+
+    @Override
+    public Tuple next() throws SQLException {
+        return iterator.nextOrNull();
+    }
+
+    @Override
+    public Tuple peek() throws SQLException {
+        return iterator.peek();
+    }
+
+    private static class PeekingCollectionIterator implements Iterator<Tuple> {
+        private final Iterator<Tuple> iterator;
+        private Tuple current;            
+        
+        private PeekingCollectionIterator(Collection<Tuple> results) {
+            iterator = results.iterator();
+            advance();
+        }
+        
+        private Tuple advance() {
+            if (iterator.hasNext()) {
+                current = iterator.next();
+            } else {
+                current = null;
+            }
+            return current;
+        }
+        
+        @Override
+        public boolean hasNext() {
+            return current != null;
+        }
+
+        @Override
+        public Tuple next() {
+            Tuple next = nextOrNull();
+            if (next == null) {
+                throw new NoSuchElementException();
+            }
+            return next;
+        }
+
+        public Tuple nextOrNull() {
+            if (current == null) {
+                return null;
+            }
+            Tuple next = current;
+            advance();
+            return next;
+        }
+
+        @Override
+        public void remove() {
+            throw new UnsupportedOperationException();
+        }
+        
+        public Tuple peek() {
+            return current;
+        }
+
+    }
+
+    @Override
+    public void explain(List<String> planSteps) {
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/iterate/MergeSortResultIterator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/MergeSortResultIterator.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/MergeSortResultIterator.java
new file mode 100644
index 0000000..58b7e40
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/MergeSortResultIterator.java
@@ -0,0 +1,95 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.iterate;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.SQLCloseables;
+
+
+/**
+ * 
+ * Base class for a ResultIterator that does a merge sort on the list of iterators
+ * provided.
+ *
+ * @author jtaylor
+ * @since 1.2
+ */
+public abstract class MergeSortResultIterator implements PeekingResultIterator {
+    protected final ResultIterators resultIterators;
+    protected final ImmutableBytesWritable tempPtr = new ImmutableBytesWritable();
+    private List<PeekingResultIterator> iterators;
+    
+    public MergeSortResultIterator(ResultIterators iterators) {
+        this.resultIterators = iterators;
+    }
+    
+    private List<PeekingResultIterator> getIterators() throws SQLException {
+        if (iterators == null) {
+            iterators = resultIterators.getIterators();
+        }
+        return iterators;
+    }
+    
+    @Override
+    public void close() throws SQLException {
+        if (iterators != null) {
+            SQLCloseables.closeAll(iterators);
+        }
+    }
+
+    abstract protected int compare(Tuple t1, Tuple t2);
+    
+    private PeekingResultIterator minIterator() throws SQLException {
+        List<PeekingResultIterator> iterators = getIterators();
+        Tuple minResult = null;
+        PeekingResultIterator minIterator = EMPTY_ITERATOR;
+        for (int i = iterators.size()-1; i >= 0; i--) {
+            PeekingResultIterator iterator = iterators.get(i);
+            Tuple r = iterator.peek();
+            if (r != null) {
+                if (minResult == null || compare(r, minResult) < 0) {
+                    minResult = r;
+                    minIterator = iterator;
+                }
+                continue;
+            }
+            iterator.close();
+            iterators.remove(i);
+        }
+        return minIterator;
+    }
+    
+    @Override
+    public Tuple peek() throws SQLException {
+        PeekingResultIterator iterator = minIterator();
+        return iterator.peek();
+    }
+
+    @Override
+    public Tuple next() throws SQLException {
+        PeekingResultIterator iterator = minIterator();
+        return iterator.next();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/iterate/MergeSortRowKeyResultIterator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/MergeSortRowKeyResultIterator.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/MergeSortRowKeyResultIterator.java
new file mode 100644
index 0000000..a411bab
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/MergeSortRowKeyResultIterator.java
@@ -0,0 +1,61 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.iterate;
+
+import java.util.List;
+
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.TupleUtil;
+
+
+/**
+ * 
+ * ResultIterator that does a merge sort on the list of iterators provided,
+ * returning the rows in row key ascending order. The iterators provided
+ * must be in row key ascending order.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class MergeSortRowKeyResultIterator extends MergeSortResultIterator {
+    private final int keyOffset;
+    private final int factor;
+    
+    public MergeSortRowKeyResultIterator(ResultIterators iterators) {
+        this(iterators, 0, false);
+    }
+    
+    public MergeSortRowKeyResultIterator(ResultIterators iterators, int keyOffset, boolean isReverse) {
+        super(iterators);
+        this.keyOffset = keyOffset;
+        this.factor = isReverse ? -1 : 1;
+    }
+   
+    @Override
+    protected int compare(Tuple t1, Tuple t2) {
+        return factor * TupleUtil.compare(t1, t2, tempPtr, keyOffset);
+    }
+
+    @Override
+    public void explain(List<String> planSteps) {
+        resultIterators.explain(planSteps);
+        planSteps.add("CLIENT MERGE SORT");
+    }
+}
\ No newline at end of file


[51/51] [partial] git commit: Initial commit of master branch from github

Posted by ja...@apache.org.
Initial commit of master branch from github


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

Branch: refs/heads/master
Commit: 50d523f6e7e3ba530128a86f0ad4b68b151213c8
Parents: 64da9e2
Author: James Taylor <ja...@apache.org>
Authored: Mon Jan 27 14:14:21 2014 -0800
Committer: James Taylor <ja...@apache.org>
Committed: Mon Jan 27 14:14:21 2014 -0800

----------------------------------------------------------------------
 .gitignore                                      |    1 +
 bin/csv-bulk-loader.sh                          |    2 +-
 bin/performance.sh                              |    5 +-
 bin/psql.sh                                     |    2 +-
 bin/sqlline.sh                                  |    4 +-
 bin/upgradeTo2.sh                               |    3 +-
 build.txt                                       |   37 +-
 config/apache-access-logs.properties            |   35 +
 config/csv-bulk-load-config.properties          |    5 +
 dev/eclipse_prefs_phoenix.epf                   |    2 +-
 phoenix-assembly/pom.xml                        |  115 +
 phoenix-assembly/src/build/all.xml              |  139 +
 phoenix-assembly/src/build/client-minimal.xml   |   16 +
 .../src/build/client-without-hbase.xml          |   18 +
 phoenix-assembly/src/build/client.xml           |   41 +
 .../src/build/components-major-client.xml       |   32 +
 .../src/build/components-minimal.xml            |   47 +
 phoenix-core/pom.xml                            |  319 +
 phoenix-core/src/main/antlr3/PhoenixSQL.g       | 1136 +++
 .../hadoop/hbase/index/CapturingAbortable.java  |   68 +
 .../hbase/index/IndexLogRollSynchronizer.java   |  125 +
 .../org/apache/hadoop/hbase/index/Indexer.java  |  706 ++
 .../apache/hadoop/hbase/index/ValueGetter.java  |   38 +
 .../hbase/index/builder/BaseIndexBuilder.java   |   99 +
 .../hbase/index/builder/IndexBuildManager.java  |  216 +
 .../hbase/index/builder/IndexBuilder.java       |  144 +
 .../builder/IndexBuildingFailureException.java  |   50 +
 .../hadoop/hbase/index/covered/Batch.java       |   62 +
 .../hbase/index/covered/CoveredColumns.java     |   51 +
 .../covered/CoveredColumnsIndexBuilder.java     |  490 ++
 .../hadoop/hbase/index/covered/IndexCodec.java  |  112 +
 .../hadoop/hbase/index/covered/IndexUpdate.java |   79 +
 .../hbase/index/covered/KeyValueStore.java      |   35 +
 .../hbase/index/covered/LocalTableState.java    |  244 +
 .../hadoop/hbase/index/covered/TableState.java  |   98 +
 .../hbase/index/covered/data/IndexMemStore.java |  333 +
 .../index/covered/data/LazyValueGetter.java     |   89 +
 .../index/covered/data/LocalHBaseState.java     |   49 +
 .../hbase/index/covered/data/LocalTable.java    |   74 +
 .../index/covered/example/ColumnGroup.java      |   95 +
 .../index/covered/example/CoveredColumn.java    |   89 +
 .../example/CoveredColumnIndexCodec.java        |  367 +
 .../CoveredColumnIndexSpecifierBuilder.java     |  166 +
 .../covered/example/CoveredColumnIndexer.java   |  146 +
 .../filter/ApplyAndFilterDeletesFilter.java     |  310 +
 ...olumnTrackingNextLargestTimestampFilter.java |   74 +
 .../index/covered/filter/FamilyOnlyFilter.java  |   80 +
 .../covered/filter/MaxTimestampFilter.java      |   76 +
 .../covered/filter/NewerTimestampFilter.java    |   37 +
 .../index/covered/update/ColumnReference.java   |  168 +
 .../index/covered/update/ColumnTracker.java     |  117 +
 .../covered/update/IndexUpdateManager.java      |  240 +
 .../covered/update/IndexedColumnGroup.java      |   30 +
 .../index/covered/update/SortedCollection.java  |  130 +
 .../index/exception/IndexWriteException.java    |   45 +
 .../MultiIndexWriteFailureException.java        |   46 +
 .../SingleIndexWriteFailureException.java       |   63 +
 .../hbase/index/parallel/BaseTaskRunner.java    |  131 +
 .../hbase/index/parallel/EarlyExitFailure.java  |   36 +
 .../index/parallel/QuickFailingTaskRunner.java  |   51 +
 .../hadoop/hbase/index/parallel/Task.java       |   42 +
 .../hadoop/hbase/index/parallel/TaskBatch.java  |   76 +
 .../hadoop/hbase/index/parallel/TaskRunner.java |   62 +
 .../hbase/index/parallel/ThreadPoolBuilder.java |   92 +
 .../hbase/index/parallel/ThreadPoolManager.java |  148 +
 .../parallel/WaitForCompletionTaskRunner.java   |   53 +
 .../hbase/index/scanner/EmptyScanner.java       |   32 +
 .../index/scanner/FilteredKeyValueScanner.java  |  144 +
 .../hadoop/hbase/index/scanner/Scanner.java     |   37 +
 .../hbase/index/scanner/ScannerBuilder.java     |  147 +
 .../hbase/index/table/CachingHTableFactory.java |  112 +
 .../index/table/CoprocessorHTableFactory.java   |   50 +
 .../hadoop/hbase/index/table/HTableFactory.java |   14 +
 .../index/table/HTableInterfaceReference.java   |   46 +
 .../hbase/index/util/ImmutableBytesPtr.java     |  113 +
 .../hbase/index/util/IndexManagementUtil.java   |  246 +
 .../hadoop/hbase/index/wal/IndexedKeyValue.java |  155 +
 .../hadoop/hbase/index/wal/KeyValueCodec.java   |   79 +
 .../hbase/index/write/IndexCommitter.java       |   39 +
 .../hbase/index/write/IndexFailurePolicy.java   |   46 +
 .../hadoop/hbase/index/write/IndexWriter.java   |  226 +
 .../hbase/index/write/IndexWriterUtils.java     |   74 +
 .../index/write/KillServerOnFailurePolicy.java  |   82 +
 .../write/ParallelWriterIndexCommitter.java     |  212 +
 .../recovery/PerRegionIndexWriteCache.java      |   65 +
 .../recovery/StoreFailuresInCachePolicy.java    |   84 +
 .../TrackingParallelWriterIndexCommitter.java   |  228 +
 .../regionserver/IndexKeyValueSkipListSet.java  |   78 +
 .../regionserver/wal/IndexedHLogReader.java     |  152 +
 .../hbase/regionserver/wal/IndexedWALEdit.java  |   91 +
 .../regionserver/wal/IndexedWALEditCodec.java   |  195 +
 .../org/apache/phoenix/cache/GlobalCache.java   |  114 +
 .../org/apache/phoenix/cache/HashCache.java     |   40 +
 .../phoenix/cache/IndexMetaDataCache.java       |   10 +
 .../apache/phoenix/cache/ServerCacheClient.java |  278 +
 .../org/apache/phoenix/cache/TenantCache.java   |   44 +
 .../apache/phoenix/cache/TenantCacheImpl.java   |   96 +
 .../phoenix/cache/aggcache/SpillFile.java       |  142 +
 .../phoenix/cache/aggcache/SpillManager.java    |  323 +
 .../apache/phoenix/cache/aggcache/SpillMap.java |  494 ++
 .../cache/aggcache/SpillableGroupByCache.java   |  361 +
 .../apache/phoenix/client/ClientKeyValue.java   |  503 ++
 .../phoenix/client/ClientKeyValueBuilder.java   |   67 +
 .../phoenix/client/GenericKeyValueBuilder.java  |   71 +
 .../apache/phoenix/client/KeyValueBuilder.java  |  108 +
 .../phoenix/compile/AggregationManager.java     |   53 +
 .../org/apache/phoenix/compile/BindManager.java |   82 +
 .../apache/phoenix/compile/ColumnProjector.java |   70 +
 .../apache/phoenix/compile/ColumnResolver.java  |   55 +
 .../phoenix/compile/CreateIndexCompiler.java    |   88 +
 .../phoenix/compile/CreateSequenceCompiler.java |  213 +
 .../phoenix/compile/CreateTableCompiler.java    |  245 +
 .../apache/phoenix/compile/DeleteCompiler.java  |  360 +
 .../phoenix/compile/DropSequenceCompiler.java   |   69 +
 .../org/apache/phoenix/compile/ExplainPlan.java |   39 +
 .../phoenix/compile/ExpressionCompiler.java     | 1391 ++++
 .../phoenix/compile/ExpressionManager.java      |   70 +
 .../phoenix/compile/ExpressionProjector.java    |   93 +
 .../apache/phoenix/compile/FromCompiler.java    |  472 ++
 .../apache/phoenix/compile/GroupByCompiler.java |  282 +
 .../apache/phoenix/compile/HavingCompiler.java  |  241 +
 .../phoenix/compile/IndexStatementRewriter.java |  103 +
 .../apache/phoenix/compile/JoinCompiler.java    | 1133 +++
 .../org/apache/phoenix/compile/KeyPart.java     |   79 +
 .../apache/phoenix/compile/LimitCompiler.java   |  110 +
 .../MutatingParallelIteratorFactory.java        |  115 +
 .../apache/phoenix/compile/MutationPlan.java    |   31 +
 .../apache/phoenix/compile/OrderByCompiler.java |  138 +
 .../apache/phoenix/compile/PostDDLCompiler.java |  225 +
 .../phoenix/compile/PostIndexDDLCompiler.java   |  119 +
 .../phoenix/compile/ProjectionCompiler.java     |  377 +
 .../apache/phoenix/compile/QueryCompiler.java   |  289 +
 .../org/apache/phoenix/compile/QueryPlan.java   |   69 +
 .../apache/phoenix/compile/RowProjector.java    |  124 +
 .../org/apache/phoenix/compile/ScanRanges.java  |  185 +
 .../apache/phoenix/compile/SequenceManager.java |  194 +
 .../phoenix/compile/StatementContext.java       |  227 +
 .../phoenix/compile/StatementNormalizer.java    |  102 +
 .../apache/phoenix/compile/StatementPlan.java   |   47 +
 .../TrackOrderPreservingExpressionCompiler.java |  211 +
 .../apache/phoenix/compile/UpsertCompiler.java  |  837 +++
 .../apache/phoenix/compile/WhereCompiler.java   |  214 +
 .../apache/phoenix/compile/WhereOptimizer.java  | 1135 +++
 .../phoenix/coprocessor/BaseRegionScanner.java  |   69 +
 .../coprocessor/BaseScannerRegionObserver.java  |   57 +
 .../phoenix/coprocessor/GroupByCache.java       |   41 +
 .../GroupedAggregateRegionObserver.java         |  507 ++
 .../coprocessor/HashJoinRegionScanner.java      |  284 +
 .../coprocessor/MetaDataEndpointImpl.java       | 1220 +++
 .../phoenix/coprocessor/MetaDataProtocol.java   |  269 +
 .../coprocessor/MetaDataRegionObserver.java     |   40 +
 .../phoenix/coprocessor/ScanProjector.java      |  202 +
 .../phoenix/coprocessor/ScanRegionObserver.java |  313 +
 .../coprocessor/SequenceRegionObserver.java     |  258 +
 .../coprocessor/ServerCachingEndpointImpl.java  |   57 +
 .../coprocessor/ServerCachingProtocol.java      |   63 +
 .../UngroupedAggregateRegionObserver.java       |  423 ++
 .../phoenix/exception/PhoenixIOException.java   |   33 +
 .../exception/PhoenixParserException.java       |  112 +
 .../phoenix/exception/SQLExceptionCode.java     |  340 +
 .../phoenix/exception/SQLExceptionInfo.java     |  166 +
 .../exception/UnknownFunctionException.java     |   37 +
 .../ValueTypeIncompatibleException.java         |   38 +
 .../apache/phoenix/execute/AggregatePlan.java   |  171 +
 .../apache/phoenix/execute/BasicQueryPlan.java  |  197 +
 .../apache/phoenix/execute/CommitException.java |   43 +
 .../phoenix/execute/DegenerateQueryPlan.java    |   52 +
 .../apache/phoenix/execute/HashJoinPlan.java    |  186 +
 .../apache/phoenix/execute/MutationState.java   |  444 ++
 .../org/apache/phoenix/execute/ScanPlan.java    |  116 +
 .../phoenix/expression/AddExpression.java       |   56 +
 .../phoenix/expression/AndExpression.java       |   79 +
 .../phoenix/expression/AndOrExpression.java     |   99 +
 .../expression/ArithmeticExpression.java        |   45 +
 .../expression/ArrayConstructorExpression.java  |  101 +
 .../expression/BaseAddSubtractExpression.java   |   53 +
 .../expression/BaseCompoundExpression.java      |  145 +
 .../BaseDecimalAddSubtractExpression.java       |    5 +
 .../phoenix/expression/BaseExpression.java      |  108 +
 .../expression/BaseSingleExpression.java        |   91 +
 .../expression/BaseTerminalExpression.java      |   47 +
 .../phoenix/expression/CaseExpression.java      |  234 +
 .../phoenix/expression/CoerceExpression.java    |  153 +
 .../phoenix/expression/ColumnExpression.java    |  154 +
 .../expression/ComparisonExpression.java        |  159 +
 .../expression/CurrentDateTimeFunction.java     |   44 +
 .../phoenix/expression/DateAddExpression.java   |   80 +
 .../expression/DateSubtractExpression.java      |   80 +
 .../expression/DecimalAddExpression.java        |  101 +
 .../expression/DecimalDivideExpression.java     |   79 +
 .../expression/DecimalMultiplyExpression.java   |   79 +
 .../expression/DecimalSubtractExpression.java   |  118 +
 .../phoenix/expression/DivideExpression.java    |   98 +
 .../phoenix/expression/DoubleAddExpression.java |   70 +
 .../expression/DoubleDivideExpression.java      |   74 +
 .../expression/DoubleMultiplyExpression.java    |   70 +
 .../expression/DoubleSubtractExpression.java    |   74 +
 .../apache/phoenix/expression/Expression.java   |   82 +
 .../phoenix/expression/ExpressionType.java      |  192 +
 .../phoenix/expression/InListExpression.java    |  291 +
 .../IndexKeyValueColumnExpression.java          |   26 +
 .../phoenix/expression/IsNullExpression.java    |  106 +
 .../expression/KeyValueColumnExpression.java    |  113 +
 .../phoenix/expression/LikeExpression.java      |  302 +
 .../phoenix/expression/LiteralExpression.java   |  316 +
 .../phoenix/expression/LongAddExpression.java   |   62 +
 .../expression/LongDivideExpression.java        |   66 +
 .../expression/LongMultiplyExpression.java      |   65 +
 .../expression/LongSubtractExpression.java      |   83 +
 .../phoenix/expression/MultiplyExpression.java  |   98 +
 .../phoenix/expression/NotExpression.java       |   86 +
 .../apache/phoenix/expression/OrExpression.java |   67 +
 .../phoenix/expression/OrderByExpression.java   |   87 +
 .../expression/ProjectedColumnExpression.java   |  144 +
 .../expression/RowKeyColumnExpression.java      |  150 +
 .../phoenix/expression/RowKeyExpression.java    |   25 +
 .../RowValueConstructorExpression.java          |  358 +
 .../expression/StringConcatExpression.java      |   95 +
 .../phoenix/expression/SubtractExpression.java  |   60 +
 .../expression/TimestampAddExpression.java      |   89 +
 .../expression/TimestampSubtractExpression.java |   89 +
 .../expression/aggregator/Aggregator.java       |   48 +
 .../expression/aggregator/Aggregators.java      |  133 +
 .../expression/aggregator/BaseAggregator.java   |   59 +
 .../aggregator/BaseDecimalStddevAggregator.java |  119 +
 .../aggregator/BaseStddevAggregator.java        |   96 +
 .../aggregator/ClientAggregators.java           |   80 +
 .../expression/aggregator/CountAggregator.java  |   92 +
 .../aggregator/DecimalStddevPopAggregator.java  |   43 +
 .../aggregator/DecimalStddevSampAggregator.java |   43 +
 .../aggregator/DecimalSumAggregator.java        |  103 +
 .../DistinctCountClientAggregator.java          |   55 +
 .../DistinctValueWithCountClientAggregator.java |  132 +
 .../DistinctValueWithCountServerAggregator.java |  181 +
 .../aggregator/DoubleSumAggregator.java         |   94 +
 .../expression/aggregator/IntSumAggregator.java |   43 +
 .../aggregator/LongSumAggregator.java           |   42 +
 .../expression/aggregator/MaxAggregator.java    |   50 +
 .../expression/aggregator/MinAggregator.java    |   98 +
 .../aggregator/NumberSumAggregator.java         |  114 +
 .../aggregator/PercentRankClientAggregator.java |   92 +
 .../aggregator/PercentileClientAggregator.java  |  110 +
 .../PercentileDiscClientAggregator.java         |  106 +
 .../aggregator/ServerAggregators.java           |  145 +
 .../aggregator/StddevPopAggregator.java         |   43 +
 .../aggregator/StddevSampAggregator.java        |   43 +
 .../aggregator/UnsignedIntSumAggregator.java    |   43 +
 .../aggregator/UnsignedLongSumAggregator.java   |   44 +
 .../expression/function/AggregateFunction.java  |   54 +
 .../expression/function/ArrayIndexFunction.java |   88 +
 .../function/ArrayLengthFunction.java           |   75 +
 .../function/AvgAggregateFunction.java          |  113 +
 .../expression/function/CeilDateExpression.java |   77 +
 .../function/CeilDecimalExpression.java         |   77 +
 .../expression/function/CeilFunction.java       |   57 +
 .../function/CeilTimestampExpression.java       |  111 +
 .../expression/function/CoalesceFunction.java   |  100 +
 .../function/CompositeAggregateFunction.java    |   47 +
 .../function/CountAggregateFunction.java        |  116 +
 .../function/CurrentDateFunction.java           |   69 +
 .../function/CurrentTimeFunction.java           |   69 +
 ...elegateConstantToCountAggregateFunction.java |   75 +
 .../DistinctCountAggregateFunction.java         |  124 +
 ...DistinctValueWithCountAggregateFunction.java |   50 +
 .../function/FloorDateExpression.java           |   88 +
 .../function/FloorDecimalExpression.java        |   77 +
 .../expression/function/FloorFunction.java      |   56 +
 .../function/FunctionArgumentType.java          |   35 +
 .../expression/function/FunctionExpression.java |   71 +
 .../function/IndexStateNameFunction.java        |   81 +
 .../expression/function/InvertFunction.java     |  129 +
 .../expression/function/LTrimFunction.java      |  115 +
 .../expression/function/LengthFunction.java     |   96 +
 .../function/MaxAggregateFunction.java          |   75 +
 .../function/MinAggregateFunction.java          |   89 +
 .../function/PercentRankAggregateFunction.java  |   74 +
 .../PercentileContAggregateFunction.java        |   75 +
 .../PercentileDiscAggregateFunction.java        |   72 +
 .../expression/function/PrefixFunction.java     |   83 +
 .../expression/function/RTrimFunction.java      |  161 +
 .../function/RegexpReplaceFunction.java         |  130 +
 .../function/RegexpSubstrFunction.java          |  175 +
 .../expression/function/ReverseFunction.java    |   70 +
 .../function/RoundDateExpression.java           |  274 +
 .../function/RoundDecimalExpression.java        |  136 +
 .../expression/function/RoundFunction.java      |   57 +
 .../function/RoundTimestampExpression.java      |  106 +
 .../function/SQLTableTypeFunction.java          |   80 +
 .../function/SQLViewTypeFunction.java           |   80 +
 .../expression/function/ScalarFunction.java     |   72 +
 .../function/SingleAggregateFunction.java       |  173 +
 .../function/SqlTypeNameFunction.java           |   86 +
 .../expression/function/StddevPopFunction.java  |   78 +
 .../expression/function/StddevSampFunction.java |   78 +
 .../expression/function/SubstrFunction.java     |  217 +
 .../function/SumAggregateFunction.java          |  146 +
 .../phoenix/expression/function/TimeUnit.java   |   50 +
 .../expression/function/ToCharFunction.java     |  135 +
 .../expression/function/ToDateFunction.java     |  133 +
 .../expression/function/ToNumberFunction.java   |  176 +
 .../expression/function/TrimFunction.java       |  108 +
 .../expression/function/TruncFunction.java      |   65 +
 .../visitor/BaseExpressionVisitor.java          |  264 +
 .../expression/visitor/ExpressionVisitor.java   |  142 +
 .../visitor/KeyValueExpressionVisitor.java      |   38 +
 .../visitor/SingleAggregateFunctionVisitor.java |   40 +
 .../visitor/TraverseAllExpressionVisitor.java   |   57 +
 .../visitor/TraverseNoExpressionVisitor.java    |   35 +
 .../phoenix/filter/BooleanExpressionFilter.java |  125 +
 .../filter/EvaluateOnCompletionVisitor.java     |   83 +
 .../MultiCFCQKeyValueComparisonFilter.java      |  134 +
 .../filter/MultiCQKeyValueComparisonFilter.java |   70 +
 .../filter/MultiKeyValueComparisonFilter.java   |  247 +
 .../phoenix/filter/RowKeyComparisonFilter.java  |  156 +
 .../SingleCFCQKeyValueComparisonFilter.java     |   51 +
 .../SingleCQKeyValueComparisonFilter.java       |   49 +
 .../filter/SingleKeyValueComparisonFilter.java  |  147 +
 .../apache/phoenix/filter/SkipScanFilter.java   |  523 ++
 .../apache/phoenix/index/BaseIndexCodec.java    |   61 +
 .../apache/phoenix/index/IndexMaintainer.java   |  810 ++
 .../phoenix/index/IndexMetaDataCacheClient.java |   98 +
 .../index/IndexMetaDataCacheFactory.java        |   66 +
 .../phoenix/index/PhoenixIndexBuilder.java      |   96 +
 .../apache/phoenix/index/PhoenixIndexCodec.java |  182 +
 .../index/PhoenixIndexFailurePolicy.java        |  102 +
 .../iterate/AggregatingResultIterator.java      |   40 +
 .../phoenix/iterate/BaseResultIterator.java     |   43 +
 .../phoenix/iterate/ConcatResultIterator.java   |   91 +
 .../DefaultParallelIteratorRegionSplitter.java  |  229 +
 .../phoenix/iterate/DelegateResultIterator.java |   50 +
 .../DistinctAggregatingResultIterator.java      |  163 +
 .../apache/phoenix/iterate/ExplainTable.java    |  271 +
 .../FilterAggregatingResultIterator.java        |   80 +
 .../phoenix/iterate/FilterResultIterator.java   |   78 +
 .../GroupedAggregatingResultIterator.java       |  106 +
 .../phoenix/iterate/LimitingResultIterator.java |   57 +
 .../iterate/LookAheadResultIterator.java        |   53 +
 .../iterate/MappedByteBufferSortedQueue.java    |  389 +
 .../iterate/MaterializedResultIterator.java     |  112 +
 .../iterate/MergeSortResultIterator.java        |   95 +
 .../iterate/MergeSortRowKeyResultIterator.java  |   61 +
 .../iterate/MergeSortTopNResultIterator.java    |  100 +
 .../OrderedAggregatingResultIterator.java       |   63 +
 .../phoenix/iterate/OrderedResultIterator.java  |  254 +
 .../iterate/ParallelIteratorRegionSplitter.java |   36 +
 .../ParallelIteratorRegionSplitterFactory.java  |   40 +
 .../phoenix/iterate/ParallelIterators.java      |  235 +
 .../phoenix/iterate/PeekingResultIterator.java  |   63 +
 .../iterate/RegionScannerResultIterator.java    |   65 +
 .../apache/phoenix/iterate/ResultIterator.java  |   54 +
 .../apache/phoenix/iterate/ResultIterators.java |   29 +
 .../phoenix/iterate/ScanningResultIterator.java |   61 +
 .../phoenix/iterate/SequenceResultIterator.java |   59 +
 ...SkipRangeParallelIteratorRegionSplitter.java |   85 +
 .../iterate/SpoolTooBigToDiskException.java     |   17 +
 .../phoenix/iterate/SpoolingResultIterator.java |  320 +
 .../phoenix/iterate/TableResultIterator.java    |   86 +
 .../UngroupedAggregatingResultIterator.java     |   56 +
 .../java/org/apache/phoenix/jdbc/Jdbc7Shim.java |   66 +
 .../apache/phoenix/jdbc/PhoenixConnection.java  |  671 ++
 .../phoenix/jdbc/PhoenixDatabaseMetaData.java   | 1456 ++++
 .../org/apache/phoenix/jdbc/PhoenixDriver.java  |   96 +
 .../phoenix/jdbc/PhoenixEmbeddedDriver.java     |  325 +
 .../phoenix/jdbc/PhoenixParameterMetaData.java  |  172 +
 .../phoenix/jdbc/PhoenixPreparedStatement.java  |  469 ++
 .../apache/phoenix/jdbc/PhoenixResultSet.java   | 1240 ++++
 .../phoenix/jdbc/PhoenixResultSetMetaData.java  |  213 +
 .../apache/phoenix/jdbc/PhoenixStatement.java   | 1258 ++++
 .../phoenix/jdbc/PhoenixStatementFactory.java   |   24 +
 .../phoenix/job/AbstractRoundRobinQueue.java    |  314 +
 .../java/org/apache/phoenix/job/JobManager.java |  115 +
 .../apache/phoenix/join/HashCacheClient.java    |  133 +
 .../apache/phoenix/join/HashCacheFactory.java   |  127 +
 .../org/apache/phoenix/join/HashJoinInfo.java   |  210 +
 .../MaxServerCacheSizeExceededException.java    |   49 +
 .../org/apache/phoenix/join/ScanProjector.java  |  242 +
 .../phoenix/map/reduce/CSVBulkLoader.java       |  401 +
 .../apache/phoenix/map/reduce/MapReduceJob.java |  204 +
 .../phoenix/map/reduce/util/ConfigReader.java   |  114 +
 .../phoenix/memory/ChildMemoryManager.java      |  118 +
 .../phoenix/memory/DelegatingMemoryManager.java |   61 +
 .../phoenix/memory/GlobalMemoryManager.java     |  178 +
 .../memory/InsufficientMemoryException.java     |   47 +
 .../apache/phoenix/memory/MemoryManager.java    |   97 +
 .../apache/phoenix/optimize/QueryOptimizer.java |  239 +
 .../phoenix/parse/AddColumnStatement.java       |   51 +
 .../org/apache/phoenix/parse/AddParseNode.java  |   49 +
 .../parse/AggregateFunctionParseNode.java       |   39 +
 .../org/apache/phoenix/parse/AliasedNode.java   |   53 +
 .../phoenix/parse/AlterIndexStatement.java      |   53 +
 .../phoenix/parse/AlterTableStatement.java      |   35 +
 .../org/apache/phoenix/parse/AndParseNode.java  |   51 +
 .../phoenix/parse/ArithmeticParseNode.java      |   30 +
 .../phoenix/parse/ArrayConstructorNode.java     |   44 +
 .../phoenix/parse/AvgAggregateParseNode.java    |   48 +
 .../phoenix/parse/BaseParseNodeVisitor.java     |   70 +
 .../apache/phoenix/parse/BetweenParseNode.java  |   54 +
 .../apache/phoenix/parse/BinaryParseNode.java   |   44 +
 .../org/apache/phoenix/parse/BindParseNode.java |   61 +
 .../org/apache/phoenix/parse/BindTableNode.java |   44 +
 .../apache/phoenix/parse/BindableStatement.java |   25 +
 .../org/apache/phoenix/parse/CaseParseNode.java |   49 +
 .../org/apache/phoenix/parse/CastParseNode.java |   80 +
 .../org/apache/phoenix/parse/CeilParseNode.java |   82 +
 .../org/apache/phoenix/parse/ColumnDef.java     |  178 +
 .../apache/phoenix/parse/ColumnFamilyDef.java   |   57 +
 .../org/apache/phoenix/parse/ColumnName.java    |  102 +
 .../apache/phoenix/parse/ColumnParseNode.java   |   90 +
 .../phoenix/parse/ComparisonParseNode.java      |   59 +
 .../apache/phoenix/parse/CompoundParseNode.java |   75 +
 .../apache/phoenix/parse/ConcreteTableNode.java |   43 +
 .../phoenix/parse/CreateIndexStatement.java     |   74 +
 .../phoenix/parse/CreateSequenceStatement.java  |   64 +
 .../phoenix/parse/CreateTableStatement.java     |  100 +
 .../phoenix/parse/CurrentDateParseNode.java     |   41 +
 .../phoenix/parse/CurrentTimeParseNode.java     |   41 +
 .../parse/DelegateConstantToCountParseNode.java |   48 +
 .../apache/phoenix/parse/DeleteStatement.java   |   69 +
 .../apache/phoenix/parse/DerivedTableNode.java  |   51 +
 .../phoenix/parse/DistinctCountParseNode.java   |   46 +
 .../apache/phoenix/parse/DivideParseNode.java   |   49 +
 .../phoenix/parse/DropColumnStatement.java      |   43 +
 .../phoenix/parse/DropIndexStatement.java       |   50 +
 .../phoenix/parse/DropSequenceStatement.java    |   46 +
 .../phoenix/parse/DropTableStatement.java       |   51 +
 .../apache/phoenix/parse/EqualParseNode.java    |   47 +
 .../apache/phoenix/parse/ExistsParseNode.java   |   55 +
 .../apache/phoenix/parse/ExplainStatement.java  |   37 +
 .../phoenix/parse/FamilyWildcardParseNode.java  |   55 +
 .../phoenix/parse/FilterableStatement.java      |   31 +
 .../apache/phoenix/parse/FloorParseNode.java    |   81 +
 .../apache/phoenix/parse/FunctionParseNode.java |  422 ++
 .../parse/GreaterThanOrEqualParseNode.java      |   49 +
 .../phoenix/parse/GreaterThanParseNode.java     |   48 +
 .../java/org/apache/phoenix/parse/HintNode.java |  160 +
 .../apache/phoenix/parse/InListParseNode.java   |   66 +
 .../org/apache/phoenix/parse/InParseNode.java   |   55 +
 .../phoenix/parse/IndexKeyConstraint.java       |   39 +
 .../apache/phoenix/parse/IsNullParseNode.java   |   55 +
 .../org/apache/phoenix/parse/JoinTableNode.java |   64 +
 .../phoenix/parse/LessThanOrEqualParseNode.java |   48 +
 .../apache/phoenix/parse/LessThanParseNode.java |   48 +
 .../org/apache/phoenix/parse/LikeParseNode.java |   55 +
 .../org/apache/phoenix/parse/LimitNode.java     |   40 +
 .../apache/phoenix/parse/LiteralParseNode.java  |   77 +
 .../phoenix/parse/MaxAggregateParseNode.java    |   41 +
 .../phoenix/parse/MinAggregateParseNode.java    |   41 +
 .../apache/phoenix/parse/MultiplyParseNode.java |   49 +
 .../org/apache/phoenix/parse/NamedNode.java     |   45 +
 .../apache/phoenix/parse/NamedParseNode.java    |   53 +
 .../apache/phoenix/parse/NamedTableNode.java    |   64 +
 .../apache/phoenix/parse/NotEqualParseNode.java |   48 +
 .../org/apache/phoenix/parse/NotParseNode.java  |   49 +
 .../org/apache/phoenix/parse/OrParseNode.java   |   49 +
 .../org/apache/phoenix/parse/OrderByNode.java   |   57 +
 .../phoenix/parse/OuterJoinParseNode.java       |   49 +
 .../org/apache/phoenix/parse/ParseContext.java  |   60 +
 .../apache/phoenix/parse/ParseException.java    |   47 +
 .../org/apache/phoenix/parse/ParseNode.java     |   52 +
 .../apache/phoenix/parse/ParseNodeFactory.java  |  564 ++
 .../apache/phoenix/parse/ParseNodeRewriter.java |  553 ++
 .../apache/phoenix/parse/ParseNodeVisitor.java  |  108 +
 .../phoenix/parse/PrimaryKeyConstraint.java     |   69 +
 .../org/apache/phoenix/parse/PropertyName.java  |   25 +
 .../apache/phoenix/parse/RoundParseNode.java    |   83 +
 .../parse/RowValueConstructorParseNode.java     |   48 +
 .../org/apache/phoenix/parse/SQLParser.java     |  198 +
 .../apache/phoenix/parse/SelectStatement.java   |  179 +
 .../phoenix/parse/SelectStatementRewriter.java  |  188 +
 .../phoenix/parse/SequenceValueParseNode.java   |   33 +
 .../phoenix/parse/SingleTableSQLStatement.java  |   39 +
 .../StatelessTraverseAllParseNodeVisitor.java   |   91 +
 .../phoenix/parse/StringConcatParseNode.java    |   51 +
 .../apache/phoenix/parse/SubqueryParseNode.java |   49 +
 .../apache/phoenix/parse/SubtractParseNode.java |   48 +
 .../phoenix/parse/SumAggregateParseNode.java    |   41 +
 .../org/apache/phoenix/parse/TableName.java     |   89 +
 .../org/apache/phoenix/parse/TableNode.java     |   45 +
 .../apache/phoenix/parse/TableNodeVisitor.java  |   37 +
 .../apache/phoenix/parse/TerminalParseNode.java |   37 +
 .../apache/phoenix/parse/ToCharParseNode.java   |   67 +
 .../apache/phoenix/parse/ToDateParseNode.java   |   51 +
 .../apache/phoenix/parse/ToNumberParseNode.java |   49 +
 .../parse/TraverseAllParseNodeVisitor.java      |  158 +
 .../parse/TraverseNoParseNodeVisitor.java       |  242 +
 .../apache/phoenix/parse/UnaryParseNode.java    |   35 +
 .../parse/UnsupportedAllParseNodeVisitor.java   |  228 +
 .../apache/phoenix/parse/UpsertStatement.java   |   54 +
 .../apache/phoenix/parse/WildcardParseNode.java |   58 +
 .../phoenix/query/BaseQueryServicesImpl.java    |   80 +
 .../phoenix/query/ChildQueryServices.java       |   47 +
 .../phoenix/query/ConfigurationFactory.java     |   46 +
 .../phoenix/query/ConnectionQueryServices.java  |  102 +
 .../query/ConnectionQueryServicesImpl.java      | 1428 ++++
 .../query/ConnectionlessQueryServicesImpl.java  |  360 +
 .../query/DelegateConnectionQueryServices.java  |  227 +
 .../phoenix/query/DelegateQueryServices.java    |   74 +
 .../phoenix/query/HBaseFactoryProvider.java     |   57 +
 .../phoenix/query/HConnectionFactory.java       |   51 +
 .../org/apache/phoenix/query/HTableFactory.java |   56 +
 .../java/org/apache/phoenix/query/KeyRange.java |  623 ++
 .../apache/phoenix/query/MetaDataMutated.java   |   42 +
 .../apache/phoenix/query/QueryConstants.java    |  215 +
 .../org/apache/phoenix/query/QueryServices.java |  119 +
 .../apache/phoenix/query/QueryServicesImpl.java |   38 +
 .../phoenix/query/QueryServicesOptions.java     |  410 ++
 .../org/apache/phoenix/query/StatsManager.java  |   59 +
 .../apache/phoenix/query/StatsManagerImpl.java  |  221 +
 .../schema/AmbiguousColumnException.java        |   52 +
 .../phoenix/schema/AmbiguousTableException.java |   52 +
 .../schema/ArgumentTypeMismatchException.java   |   44 +
 .../schema/ColumnAlreadyExistsException.java    |   62 +
 .../schema/ColumnFamilyNotFoundException.java   |   46 +
 .../apache/phoenix/schema/ColumnModifier.java   |  149 +
 .../phoenix/schema/ColumnNotFoundException.java |   65 +
 .../org/apache/phoenix/schema/ColumnRef.java    |  135 +
 .../ConcurrentTableMutationException.java       |   48 +
 .../schema/ConstraintViolationException.java    |   48 +
 .../apache/phoenix/schema/DelegateColumn.java   |   71 +
 .../apache/phoenix/schema/DelegateDatum.java    |   62 +
 .../schema/EmptySequenceCacheException.java     |   21 +
 .../ExecuteQueryNotApplicableException.java     |   39 +
 .../ExecuteUpdateNotApplicableException.java    |   38 +
 .../phoenix/schema/IllegalDataException.java    |   47 +
 .../apache/phoenix/schema/KeyValueSchema.java   |  218 +
 .../apache/phoenix/schema/MetaDataClient.java   | 1739 +++++
 .../schema/MetaDataEntityNotFoundException.java |   30 +
 .../phoenix/schema/MetaDataSplitPolicy.java     |   51 +
 .../NewerTableAlreadyExistsException.java       |   29 +
 .../apache/phoenix/schema/PArrayDataType.java   |  386 +
 .../org/apache/phoenix/schema/PBaseColumn.java  |   52 +
 .../java/org/apache/phoenix/schema/PColumn.java |   51 +
 .../apache/phoenix/schema/PColumnFamily.java    |   54 +
 .../phoenix/schema/PColumnFamilyImpl.java       |   75 +
 .../org/apache/phoenix/schema/PColumnImpl.java  |  207 +
 .../org/apache/phoenix/schema/PDataType.java    | 6926 ++++++++++++++++++
 .../org/apache/phoenix/schema/PDateColumn.java  |   27 +
 .../java/org/apache/phoenix/schema/PDatum.java  |   54 +
 .../org/apache/phoenix/schema/PIndexState.java  |   92 +
 .../apache/phoenix/schema/PIntegerColumn.java   |   35 +
 .../org/apache/phoenix/schema/PLongColumn.java  |   35 +
 .../org/apache/phoenix/schema/PMetaData.java    |   30 +
 .../apache/phoenix/schema/PMetaDataImpl.java    |  200 +
 .../java/org/apache/phoenix/schema/PName.java   |   96 +
 .../org/apache/phoenix/schema/PNameFactory.java |   23 +
 .../org/apache/phoenix/schema/PNameImpl.java    |  101 +
 .../apache/phoenix/schema/PNormalizedName.java  |   53 +
 .../java/org/apache/phoenix/schema/PRow.java    |   75 +
 .../apache/phoenix/schema/PStringColumn.java    |   36 +
 .../java/org/apache/phoenix/schema/PTable.java  |  280 +
 .../org/apache/phoenix/schema/PTableImpl.java   |  863 +++
 .../org/apache/phoenix/schema/PTableType.java   |  111 +
 .../org/apache/phoenix/schema/PhoenixArray.java |  470 ++
 .../phoenix/schema/ReadOnlyTableException.java  |   57 +
 .../org/apache/phoenix/schema/RowKeySchema.java |  243 +
 .../phoenix/schema/RowKeyValueAccessor.java     |  195 +
 .../org/apache/phoenix/schema/SaltingUtil.java  |  172 +
 .../org/apache/phoenix/schema/Sequence.java     |  391 +
 .../schema/SequenceAlreadyExistsException.java  |   48 +
 .../org/apache/phoenix/schema/SequenceKey.java  |   76 +
 .../schema/SequenceNotFoundException.java       |   50 +
 .../schema/TableAlreadyExistsException.java     |   55 +
 .../phoenix/schema/TableNotFoundException.java  |   73 +
 .../org/apache/phoenix/schema/TableRef.java     |   82 +
 .../phoenix/schema/TypeMismatchException.java   |   60 +
 .../org/apache/phoenix/schema/ValueBitSet.java  |  200 +
 .../phoenix/schema/ValueRangeExcpetion.java     |   40 +
 .../org/apache/phoenix/schema/ValueSchema.java  |  337 +
 .../apache/phoenix/schema/stat/PTableStats.java |   47 +
 .../phoenix/schema/stat/PTableStatsImpl.java    |   70 +
 .../schema/tuple/MultiKeyValueTuple.java        |   85 +
 .../phoenix/schema/tuple/ResultTuple.java       |  104 +
 .../schema/tuple/SingleKeyValueTuple.java       |  111 +
 .../org/apache/phoenix/schema/tuple/Tuple.java  |   84 +
 .../org/apache/phoenix/util/BigDecimalUtil.java |   70 +
 .../java/org/apache/phoenix/util/BitSet.java    |  106 +
 .../java/org/apache/phoenix/util/ByteUtil.java  |  561 ++
 .../java/org/apache/phoenix/util/CSVLoader.java |  250 +
 .../org/apache/phoenix/util/Closeables.java     |  124 +
 .../org/apache/phoenix/util/ColumnInfo.java     |   22 +
 .../java/org/apache/phoenix/util/DateUtil.java  |  154 +
 .../java/org/apache/phoenix/util/IndexUtil.java |  204 +
 .../apache/phoenix/util/InstanceResolver.java   |   66 +
 .../java/org/apache/phoenix/util/JDBCUtil.java  |   83 +
 .../org/apache/phoenix/util/KeyValueUtil.java   |  170 +
 .../org/apache/phoenix/util/MetaDataUtil.java   |  233 +
 .../org/apache/phoenix/util/NumberUtil.java     |   54 +
 .../org/apache/phoenix/util/PhoenixRuntime.java |  308 +
 .../java/org/apache/phoenix/util/QueryUtil.java |   75 +
 .../org/apache/phoenix/util/ReadOnlyProps.java  |  239 +
 .../org/apache/phoenix/util/ResultUtil.java     |  177 +
 .../org/apache/phoenix/util/SQLCloseable.java   |   34 +
 .../org/apache/phoenix/util/SQLCloseables.java  |  133 +
 .../java/org/apache/phoenix/util/ScanUtil.java  |  447 ++
 .../org/apache/phoenix/util/SchemaUtil.java     |  507 ++
 .../org/apache/phoenix/util/ServerUtil.java     |  123 +
 .../java/org/apache/phoenix/util/SizedUtil.java |   63 +
 .../org/apache/phoenix/util/StringUtil.java     |  246 +
 .../util/TrustedByteArrayOutputStream.java      |   56 +
 .../java/org/apache/phoenix/util/TupleUtil.java |  155 +
 phoenix-core/src/main/java/overview.html        |   25 +
 phoenix-core/src/main/resources/java.sql.Driver |    1 +
 phoenix-core/src/site/bin/merge.sh              |   10 +
 .../language_reference_source/datatypes.html    |  493 ++
 .../language_reference_source/functions.html    |  740 ++
 .../site/language_reference_source/index.html   |  947 +++
 .../markdown/Phoenix-in-15-minutes-or-less.md   |   80 +
 phoenix-core/src/site/markdown/building.md      |   20 +
 phoenix-core/src/site/markdown/download.md      |   84 +
 .../src/site/markdown/dynamic_columns.md        |   17 +
 phoenix-core/src/site/markdown/faq.md           |  279 +
 phoenix-core/src/site/markdown/flume.md         |   42 +
 phoenix-core/src/site/markdown/index.md         |   69 +
 phoenix-core/src/site/markdown/issues.md        |    9 +
 phoenix-core/src/site/markdown/mailing_list.md  |   14 +
 phoenix-core/src/site/markdown/mr_dataload.md   |   63 +
 phoenix-core/src/site/markdown/paged.md         |   31 +
 phoenix-core/src/site/markdown/performance.md   |   86 +
 .../src/site/markdown/phoenix_on_emr.md         |   43 +
 phoenix-core/src/site/markdown/recent.md        |   18 +
 phoenix-core/src/site/markdown/roadmap.md       |   35 +
 phoenix-core/src/site/markdown/salted.md        |   25 +
 .../src/site/markdown/secondary_indexing.md     |  152 +
 phoenix-core/src/site/markdown/sequences.md     |   47 +
 phoenix-core/src/site/markdown/skip_scan.md     |   22 +
 phoenix-core/src/site/markdown/source.md        |    9 +
 phoenix-core/src/site/markdown/tuning.md        |  128 +
 phoenix-core/src/site/resources/css/site.css    |   65 +
 phoenix-core/src/site/resources/favicon.ico     |  Bin 0 -> 318 bytes
 .../src/site/resources/images/PhoenixVsHive.png |  Bin 0 -> 30363 bytes
 .../site/resources/images/PhoenixVsImpala.png   |  Bin 0 -> 19219 bytes
 .../site/resources/images/PhoenixVsOpenTSDB.png |  Bin 0 -> 10039 bytes
 phoenix-core/src/site/resources/images/logo.png |  Bin 0 -> 17729 bytes
 .../src/site/resources/images/perf-esscf.png    |  Bin 0 -> 8081 bytes
 .../site/resources/images/perf-salted-read.png  |  Bin 0 -> 8086 bytes
 .../site/resources/images/perf-salted-write.png |  Bin 0 -> 8379 bytes
 .../src/site/resources/images/perf-skipscan.png |  Bin 0 -> 7526 bytes
 .../src/site/resources/images/perf-topn.png     |  Bin 0 -> 7807 bytes
 phoenix-core/src/site/resources/images/psql.png |  Bin 0 -> 25997 bytes
 .../src/site/resources/images/sqlline.png       |  Bin 0 -> 57490 bytes
 .../src/site/resources/images/squirrel.png      |  Bin 0 -> 64649 bytes
 .../site/resources/images/topbar-logo-small.png |  Bin 0 -> 4931 bytes
 .../src/site/resources/images/topbar-logo.png   |  Bin 0 -> 7939 bytes
 .../site/resources/language/images/div-d.png    |  Bin 0 -> 8289 bytes
 .../site/resources/language/images/div-ke.png   |  Bin 0 -> 763 bytes
 .../site/resources/language/images/div-ks.png   |  Bin 0 -> 759 bytes
 .../site/resources/language/images/div-le.png   |  Bin 0 -> 387 bytes
 .../site/resources/language/images/div-ls.png   |  Bin 0 -> 396 bytes
 .../site/resources/language/images/div-te.png   |  Bin 0 -> 739 bytes
 .../site/resources/language/images/div-ts.png   |  Bin 0 -> 727 bytes
 .../src/site/resources/language/stylesheet.css  |  139 +
 phoenix-core/src/site/site.xml                  |   96 +
 .../src/site/xhtml/language/datatypes.xhtml     |    5 +
 .../src/site/xhtml/language/functions.xhtml     |    5 +
 .../src/site/xhtml/language/index.xhtml         |    5 +
 phoenix-core/src/site/xhtml/team.xhtml          |  115 +
 .../hadoop/hbase/index/IndexTestingUtils.java   |   96 +
 .../hadoop/hbase/index/StubAbortable.java       |   43 +
 .../apache/hadoop/hbase/index/TableName.java    |   47 +
 .../TestFailForUnsupportedHBaseVersions.java    |  157 +
 .../covered/CoveredIndexCodecForTesting.java    |   73 +
 .../hbase/index/covered/TestCoveredColumns.java |   47 +
 .../TestEndToEndCoveredColumnsIndexBuilder.java |  341 +
 .../index/covered/TestLocalTableState.java      |  198 +
 .../index/covered/data/TestIndexMemStore.java   |   95 +
 .../covered/example/TestColumnTracker.java      |   63 +
 .../example/TestCoveredColumnIndexCodec.java    |  250 +
 .../TestCoveredIndexSpecifierBuilder.java       |   74 +
 .../example/TestEndToEndCoveredIndexing.java    |  879 +++
 .../TestEndtoEndIndexingWithCompression.java    |   51 +
 .../covered/example/TestFailWithoutRetries.java |  147 +
 .../filter/TestApplyAndFilterDeletesFilter.java |  212 +
 .../covered/filter/TestFamilyOnlyFilter.java    |  107 +
 .../filter/TestNewerTimestampFilter.java        |   49 +
 .../covered/update/TestIndexUpdateManager.java  |  142 +
 .../index/parallel/TestThreadPoolBuilder.java   |   65 +
 .../index/parallel/TestThreadPoolManager.java   |   95 +
 .../index/util/TestIndexManagementUtil.java     |   68 +
 .../hbase/index/write/FakeTableFactory.java     |   33 +
 .../index/write/TestCachingHTableFactory.java   |   60 +
 .../hbase/index/write/TestIndexWriter.java      |  286 +
 .../index/write/TestParalleIndexWriter.java     |  119 +
 .../write/TestParalleWriterIndexCommitter.java  |  119 +
 .../index/write/TestWALRecoveryCaching.java     |  370 +
 .../recovery/TestPerRegionIndexWriteCache.java  |  170 +
 .../wal/TestReadWriteKeyValuesWithCodec.java    |  155 +
 ...ALReplayWithIndexWritesAndCompressedWAL.java |  275 +
 ...exWritesAndUncompressedWALInHBase_094_9.java |   24 +
 .../arithmetic/ArithmeticOperationTest.java     |  298 +
 .../phoenix/client/TestClientKeyValue.java      |  184 +
 .../phoenix/client/TestClientKeyValueLocal.java |  270 +
 .../phoenix/compile/HavingClauseTest.java       |  194 +
 .../phoenix/compile/JoinQueryCompileTest.java   |   65 +
 .../apache/phoenix/compile/LimitClauseTest.java |  157 +
 .../phoenix/compile/QueryCompileTest.java       | 1273 ++++
 .../phoenix/compile/QueryMetaDataTest.java      |  446 ++
 .../phoenix/compile/QueryOptimizerTest.java     |  281 +
 .../apache/phoenix/compile/ScanRangesTest.java  |  546 ++
 .../compile/SelectStatementRewriterTest.java    |  119 +
 .../compile/StatementHintsCompilationTest.java  |  124 +
 .../apache/phoenix/compile/ViewCompileTest.java |  119 +
 .../phoenix/compile/WhereClauseCompileTest.java |  839 +++
 .../compile/WhereClauseOptimizerTest.java       | 1620 ++++
 .../apache/phoenix/end2end/AlterTableTest.java  |  793 ++
 .../phoenix/end2end/ArithmeticQueryTest.java    |  594 ++
 .../org/apache/phoenix/end2end/ArrayTest.java   |  810 ++
 .../apache/phoenix/end2end/AutoCommitTest.java  |   83 +
 .../end2end/BaseClientManagedTimeTest.java      |   43 +
 .../phoenix/end2end/BaseConnectedQueryTest.java |  739 ++
 .../end2end/BaseHBaseManagedTimeTest.java       |   43 +
 .../end2end/BaseTenantSpecificTablesTest.java   |   77 +
 .../phoenix/end2end/BinaryRowKeyTest.java       |  149 +
 .../apache/phoenix/end2end/CSVLoaderTest.java   |  284 +
 .../phoenix/end2end/CoalesceFunctionTest.java   |   59 +
 .../end2end/CompareDecimalToLongTest.java       |  240 +
 .../ConnectionQueryServicesTestImpl.java        |  102 +
 .../apache/phoenix/end2end/CreateTableTest.java |  104 +
 .../phoenix/end2end/CustomEntityDataTest.java   |  216 +
 ...aultParallelIteratorsRegionSplitterTest.java |  358 +
 .../org/apache/phoenix/end2end/DeleteTest.java  |  385 +
 .../end2end/DescColumnSortOrderTest.java        |  555 ++
 .../phoenix/end2end/DistinctCountTest.java      |  380 +
 .../phoenix/end2end/DynamicColumnTest.java      |  220 +
 .../phoenix/end2end/DynamicFamilyTest.java      |  362 +
 .../phoenix/end2end/DynamicUpsertTest.java      |  227 +
 .../phoenix/end2end/ExecuteStatementsTest.java  |  303 +
 .../phoenix/end2end/ExtendedQueryExecTest.java  |  190 +
 .../apache/phoenix/end2end/FunkyNamesTest.java  |  145 +
 .../apache/phoenix/end2end/GroupByCaseTest.java |  196 +
 .../apache/phoenix/end2end/HashJoinTest.java    | 1439 ++++
 .../phoenix/end2end/InMemoryOrderByTest.java    |   42 +
 .../org/apache/phoenix/end2end/IsNullTest.java  |   67 +
 .../org/apache/phoenix/end2end/KeyOnlyTest.java |  148 +
 .../phoenix/end2end/MultiCfQueryExecTest.java   |  231 +
 .../phoenix/end2end/NativeHBaseTypesTest.java   |  307 +
 .../org/apache/phoenix/end2end/OrderByTest.java |  155 +
 .../apache/phoenix/end2end/PercentileTest.java  |  621 ++
 .../phoenix/end2end/ProductMetricsTest.java     | 2027 +++++
 .../end2end/QueryDatabaseMetaDataTest.java      |  975 +++
 .../end2end/QueryExecWithoutSCNTest.java        |   51 +
 .../apache/phoenix/end2end/QueryPlanTest.java   |  203 +
 .../org/apache/phoenix/end2end/QueryTest.java   | 2812 +++++++
 .../phoenix/end2end/ReadIsolationLevelTest.java |  152 +
 .../phoenix/end2end/ReverseFunctionTest.java    |  109 +
 .../RoundFloorCeilFunctionsEnd2EndTest.java     |  346 +
 .../end2end/RowValueConstructorTest.java        |  873 +++
 .../apache/phoenix/end2end/SequenceTest.java    |  530 ++
 .../phoenix/end2end/ServerExceptionTest.java    |   69 +
 ...RangeParallelIteratorRegionSplitterTest.java |  369 +
 .../phoenix/end2end/SkipScanQueryTest.java      |  211 +
 .../phoenix/end2end/SpillableGroupByTest.java   |  136 +
 .../phoenix/end2end/SpooledOrderByTest.java     |   40 +
 .../phoenix/end2end/StatementHintsTest.java     |  163 +
 .../org/apache/phoenix/end2end/StddevTest.java  |  137 +
 .../end2end/TenantSpecificTablesDDLTest.java    |  444 ++
 .../end2end/TenantSpecificTablesDMLTest.java    |  349 +
 .../phoenix/end2end/ToCharFunctionTest.java     |  242 +
 .../phoenix/end2end/ToNumberFunctionTest.java   |  286 +
 .../org/apache/phoenix/end2end/TopNTest.java    |  160 +
 .../phoenix/end2end/UpsertBigValuesTest.java    |  371 +
 .../end2end/UpsertSelectAutoCommitTest.java     |   88 +
 .../phoenix/end2end/UpsertSelectTest.java       |  689 ++
 .../phoenix/end2end/UpsertValuesTest.java       |  366 +
 .../phoenix/end2end/VariableLengthPKTest.java   | 1881 +++++
 .../org/apache/phoenix/end2end/ViewTest.java    |  179 +
 .../end2end/index/BaseMutableIndexTest.java     |  144 +
 .../phoenix/end2end/index/DropViewTest.java     |  101 +
 .../end2end/index/ImmutableIndexTest.java       |  525 ++
 .../end2end/index/IndexMetadataTest.java        |  380 +
 .../phoenix/end2end/index/IndexTestUtil.java    |  168 +
 .../end2end/index/MutableIndexFailureTest.java  |  126 +
 .../phoenix/end2end/index/MutableIndexTest.java |  677 ++
 .../end2end/index/MutableSaltedIndexTest.java   |  189 +
 .../phoenix/end2end/salted/SaltedTableTest.java |  528 ++
 .../salted/SaltedTableUpsertSelectTest.java     |  203 +
 .../salted/SaltedTableVarLengthRowKeyTest.java  |   68 +
 .../expression/CoerceExpressionTest.java        |  132 +
 .../expression/ColumnExpressionTest.java        |  116 +
 .../DescColumnSortOrderExpressionTest.java      |  311 +
 .../phoenix/expression/LikeExpressionTest.java  |   44 +
 .../RoundFloorCeilExpressionsUnitTests.java     |  233 +
 .../filter/SkipScanFilterIntersectTest.java     |  514 ++
 .../phoenix/filter/SkipScanFilterTest.java      |  395 +
 .../phoenix/index/IndexMaintainerTest.java      |  239 +
 .../iterate/AggregateResultScannerTest.java     |  147 +
 .../iterate/ConcatResultIteratorTest.java       |  127 +
 .../iterate/MergeSortResultIteratorTest.java    |   83 +
 .../iterate/SpoolingResultIteratorTest.java     |   73 +
 .../phoenix/jdbc/PhoenixEmbeddedDriverTest.java |  103 +
 .../apache/phoenix/jdbc/PhoenixTestDriver.java  |   82 +
 .../apache/phoenix/logging/PerformanceLog.java  |  119 +
 .../phoenix/memory/MemoryManagerTest.java       |  269 +
 .../apache/phoenix/parse/QueryParserTest.java   |  670 ++
 .../query/BaseConnectionlessQueryTest.java      |  105 +
 .../java/org/apache/phoenix/query/BaseTest.java |  485 ++
 .../phoenix/query/ConnectionlessUpsertTest.java |  184 +
 .../phoenix/query/HBaseFactoryProviderTest.java |   44 +
 .../phoenix/query/KeyRangeCoalesceTests.java    |  143 +
 .../phoenix/query/KeyRangeIntersectTests.java   |   79 +
 .../phoenix/query/KeyRangeUnionTests.java       |   79 +
 .../phoenix/query/QueryServicesTestImpl.java    |   84 +
 .../phoenix/schema/PDataTypeForArraysTest.java  |  522 ++
 .../apache/phoenix/schema/PDataTypeTest.java    | 1601 ++++
 .../apache/phoenix/schema/RowKeySchemaTest.java |  131 +
 .../phoenix/schema/RowKeyValueAccessorTest.java |   69 +
 .../apache/phoenix/schema/SchemaUtilTest.java   |   59 +
 .../apache/phoenix/schema/ValueBitSetTest.java  |  137 +
 .../org/apache/phoenix/util/AssertResults.java  |  132 +
 .../org/apache/phoenix/util/ByteUtilTest.java   |   70 +
 .../org/apache/phoenix/util/DateUtilTest.java   |   64 +
 .../phoenix/util/GeneratePerformanceData.java   |   65 +
 .../apache/phoenix/util/LikeExpressionTest.java |   42 +
 .../apache/phoenix/util/MetaDataUtilTest.java   |   46 +
 .../org/apache/phoenix/util/ScanUtilTest.java   |  343 +
 .../java/org/apache/phoenix/util/TestUtil.java  |  331 +
 .../src/test/resources/hbase-default.xml        |   36 +
 .../src/test/resources/log4j.properties         |   63 +
 phoenix-flume/pom.xml                           |  104 +
 .../phoenix/flume/DefaultKeyGenerator.java      |   71 +
 .../apache/phoenix/flume/FlumeConstants.java    |   81 +
 .../org/apache/phoenix/flume/KeyGenerator.java  |   26 +
 .../org/apache/phoenix/flume/SchemaHandler.java |   49 +
 .../flume/serializer/BaseEventSerializer.java   |  237 +
 .../flume/serializer/EventSerializer.java       |   44 +
 .../flume/serializer/EventSerializers.java      |   38 +
 .../flume/serializer/RegexEventSerializer.java  |  149 +
 .../apache/phoenix/flume/sink/PhoenixSink.java  |  207 +
 .../apache/phoenix/flume/TestPhoenixSink.java   |  198 +
 .../phoenix/flume/TestRegexEventSerializer.java |  418 ++
 phoenix-hadoop-compat/pom.xml                   |   40 +
 phoenix-hadoop1-compat/pom.xml                  |   18 +
 phoenix-hadoop2-compat/pom.xml                  |   10 +
 phoenix-pig/pom.xml                             |   22 +
 .../apache/phoenix/pig/PhoenixHBaseStorage.java |  189 +
 .../phoenix/pig/PhoenixPigConfiguration.java    |  165 +
 .../java/org/apache/phoenix/pig/TypeUtil.java   |  191 +
 .../pig/hadoop/PhoenixOutputCommitter.java      |  113 +
 .../phoenix/pig/hadoop/PhoenixOutputFormat.java |   96 +
 .../phoenix/pig/hadoop/PhoenixRecord.java       |   91 +
 .../phoenix/pig/hadoop/PhoenixRecordWriter.java |   85 +
 pom.xml                                         |  820 +--
 src/build/all.xml                               |  118 -
 src/build/client.xml                            |   85 -
 src/main/antlr3/PhoenixSQL.g                    | 1102 ---
 src/main/config/csv-bulk-load-config.properties |    5 -
 .../regionserver/IndexKeyValueSkipListSet.java  |   78 -
 .../regionserver/wal/IndexedHLogReader.java     |  152 -
 .../hbase/regionserver/wal/IndexedWALEdit.java  |   91 -
 .../regionserver/wal/IndexedWALEditCodec.java   |  195 -
 .../apache/hbase/index/CapturingAbortable.java  |   68 -
 .../hbase/index/IndexLogRollSynchronizer.java   |  125 -
 .../java/org/apache/hbase/index/Indexer.java    |  704 --
 .../org/apache/hbase/index/ValueGetter.java     |   38 -
 .../hbase/index/builder/BaseIndexBuilder.java   |   99 -
 .../hbase/index/builder/IndexBuildManager.java  |  216 -
 .../hbase/index/builder/IndexBuilder.java       |  144 -
 .../builder/IndexBuildingFailureException.java  |   50 -
 .../org/apache/hbase/index/covered/Batch.java   |   62 -
 .../hbase/index/covered/CoveredColumns.java     |   51 -
 .../covered/CoveredColumnsIndexBuilder.java     |  490 --
 .../apache/hbase/index/covered/IndexCodec.java  |  112 -
 .../apache/hbase/index/covered/IndexUpdate.java |   79 -
 .../hbase/index/covered/KeyValueStore.java      |   35 -
 .../hbase/index/covered/LocalTableState.java    |  244 -
 .../apache/hbase/index/covered/TableState.java  |   98 -
 .../hbase/index/covered/data/IndexMemStore.java |  317 -
 .../index/covered/data/LazyValueGetter.java     |   89 -
 .../index/covered/data/LocalHBaseState.java     |   49 -
 .../hbase/index/covered/data/LocalTable.java    |   74 -
 .../index/covered/example/ColumnGroup.java      |   95 -
 .../index/covered/example/CoveredColumn.java    |   89 -
 .../example/CoveredColumnIndexCodec.java        |  367 -
 .../CoveredColumnIndexSpecifierBuilder.java     |  166 -
 .../covered/example/CoveredColumnIndexer.java   |  146 -
 .../filter/ApplyAndFilterDeletesFilter.java     |  310 -
 ...olumnTrackingNextLargestTimestampFilter.java |   74 -
 .../index/covered/filter/FamilyOnlyFilter.java  |   80 -
 .../covered/filter/MaxTimestampFilter.java      |   76 -
 .../covered/filter/NewerTimestampFilter.java    |   37 -
 .../index/covered/update/ColumnReference.java   |  141 -
 .../index/covered/update/ColumnTracker.java     |  117 -
 .../covered/update/IndexUpdateManager.java      |  240 -
 .../covered/update/IndexedColumnGroup.java      |   30 -
 .../index/covered/update/SortedCollection.java  |  130 -
 .../index/exception/IndexWriteException.java    |   45 -
 .../MultiIndexWriteFailureException.java        |   46 -
 .../SingleIndexWriteFailureException.java       |   63 -
 .../hbase/index/parallel/BaseTaskRunner.java    |  131 -
 .../hbase/index/parallel/EarlyExitFailure.java  |   36 -
 .../index/parallel/QuickFailingTaskRunner.java  |   51 -
 .../org/apache/hbase/index/parallel/Task.java   |   42 -
 .../apache/hbase/index/parallel/TaskBatch.java  |   76 -
 .../apache/hbase/index/parallel/TaskRunner.java |   62 -
 .../hbase/index/parallel/ThreadPoolBuilder.java |   92 -
 .../hbase/index/parallel/ThreadPoolManager.java |  148 -
 .../parallel/WaitForCompletionTaskRunner.java   |   53 -
 .../hbase/index/scanner/EmptyScanner.java       |   32 -
 .../index/scanner/FilteredKeyValueScanner.java  |  134 -
 .../org/apache/hbase/index/scanner/Scanner.java |   37 -
 .../hbase/index/scanner/ScannerBuilder.java     |  147 -
 .../hbase/index/table/CachingHTableFactory.java |  112 -
 .../index/table/CoprocessorHTableFactory.java   |   50 -
 .../apache/hbase/index/table/HTableFactory.java |   14 -
 .../index/table/HTableInterfaceReference.java   |   46 -
 .../hbase/index/util/ImmutableBytesPtr.java     |   96 -
 .../hbase/index/util/IndexManagementUtil.java   |  246 -
 .../apache/hbase/index/wal/IndexedKeyValue.java |  155 -
 .../apache/hbase/index/wal/KeyValueCodec.java   |   79 -
 .../hbase/index/write/IndexCommitter.java       |   39 -
 .../hbase/index/write/IndexFailurePolicy.java   |   46 -
 .../apache/hbase/index/write/IndexWriter.java   |  226 -
 .../hbase/index/write/IndexWriterUtils.java     |   74 -
 .../index/write/KillServerOnFailurePolicy.java  |   82 -
 .../write/ParallelWriterIndexCommitter.java     |  212 -
 .../recovery/PerRegionIndexWriteCache.java      |   65 -
 .../recovery/StoreFailuresInCachePolicy.java    |   84 -
 .../TrackingParallelWriterIndexCommitter.java   |  228 -
 .../org/apache/phoenix/cache/GlobalCache.java   |  114 -
 .../org/apache/phoenix/cache/HashCache.java     |   40 -
 .../phoenix/cache/IndexMetaDataCache.java       |   10 -
 .../apache/phoenix/cache/ServerCacheClient.java |  278 -
 .../org/apache/phoenix/cache/TenantCache.java   |   44 -
 .../apache/phoenix/cache/TenantCacheImpl.java   |   96 -
 .../phoenix/compile/AggregationManager.java     |   53 -
 .../org/apache/phoenix/compile/BindManager.java |   82 -
 .../apache/phoenix/compile/ColumnProjector.java |   69 -
 .../apache/phoenix/compile/ColumnResolver.java  |   55 -
 .../phoenix/compile/CreateIndexCompiler.java    |   88 -
 .../phoenix/compile/CreateTableCompiler.java    |   90 -
 .../apache/phoenix/compile/DeleteCompiler.java  |  352 -
 .../org/apache/phoenix/compile/ExplainPlan.java |   39 -
 .../phoenix/compile/ExpressionCompiler.java     | 1250 ----
 .../phoenix/compile/ExpressionManager.java      |   70 -
 .../phoenix/compile/ExpressionProjector.java    |   84 -
 .../apache/phoenix/compile/FromCompiler.java    |  390 -
 .../apache/phoenix/compile/GroupByCompiler.java |  278 -
 .../apache/phoenix/compile/HavingCompiler.java  |  240 -
 .../phoenix/compile/IndexStatementRewriter.java |   67 -
 .../org/apache/phoenix/compile/KeyPart.java     |   79 -
 .../apache/phoenix/compile/LimitCompiler.java   |  109 -
 .../MutatingParallelIteratorFactory.java        |  115 -
 .../apache/phoenix/compile/MutationPlan.java    |   31 -
 .../apache/phoenix/compile/OrderByCompiler.java |  126 -
 .../apache/phoenix/compile/PostDDLCompiler.java |  220 -
 .../phoenix/compile/PostIndexDDLCompiler.java   |  119 -
 .../phoenix/compile/ProjectionCompiler.java     |  354 -
 .../apache/phoenix/compile/QueryCompiler.java   |  136 -
 .../org/apache/phoenix/compile/QueryPlan.java   |   67 -
 .../apache/phoenix/compile/RowProjector.java    |  124 -
 .../org/apache/phoenix/compile/ScanRanges.java  |  185 -
 .../phoenix/compile/StatementContext.java       |  199 -
 .../phoenix/compile/StatementNormalizer.java    |   83 -
 .../apache/phoenix/compile/StatementPlan.java   |   47 -
 .../TrackOrderPreservingExpressionCompiler.java |  194 -
 .../apache/phoenix/compile/UpsertCompiler.java  |  630 --
 .../apache/phoenix/compile/WhereCompiler.java   |  214 -
 .../apache/phoenix/compile/WhereOptimizer.java  | 1112 ---
 .../phoenix/coprocessor/BaseRegionScanner.java  |   69 -
 .../coprocessor/BaseScannerRegionObserver.java  |   57 -
 .../GroupedAggregateRegionObserver.java         |  385 -
 .../coprocessor/HashJoinRegionScanner.java      |  235 -
 .../coprocessor/MetaDataEndpointImpl.java       | 1058 ---
 .../phoenix/coprocessor/MetaDataProtocol.java   |  255 -
 .../coprocessor/MetaDataRegionObserver.java     |   40 -
 .../phoenix/coprocessor/ScanProjector.java      |  202 -
 .../phoenix/coprocessor/ScanRegionObserver.java |  312 -
 .../coprocessor/ServerCachingEndpointImpl.java  |   57 -
 .../coprocessor/ServerCachingProtocol.java      |   63 -
 .../UngroupedAggregateRegionObserver.java       |  437 --
 .../phoenix/exception/PhoenixIOException.java   |   33 -
 .../exception/PhoenixParserException.java       |  107 -
 .../phoenix/exception/SQLExceptionCode.java     |  195 -
 .../phoenix/exception/SQLExceptionInfo.java     |  142 -
 .../exception/UnknownFunctionException.java     |   37 -
 .../ValueTypeIncompatibleException.java         |   38 -
 .../apache/phoenix/execute/AggregatePlan.java   |  178 -
 .../apache/phoenix/execute/BasicQueryPlan.java  |  188 -
 .../apache/phoenix/execute/CommitException.java |   43 -
 .../phoenix/execute/DegenerateQueryPlan.java    |   51 -
 .../apache/phoenix/execute/MutationState.java   |  435 --
 .../org/apache/phoenix/execute/ScanPlan.java    |  110 -
 .../phoenix/expression/AddExpression.java       |   56 -
 .../phoenix/expression/AndExpression.java       |   67 -
 .../phoenix/expression/AndOrExpression.java     |   99 -
 .../expression/ArithmeticExpression.java        |   45 -
 .../expression/BaseAddSubtractExpression.java   |   53 -
 .../expression/BaseCompoundExpression.java      |  123 -
 .../BaseDecimalAddSubtractExpression.java       |    5 -
 .../phoenix/expression/BaseExpression.java      |  103 -
 .../expression/BaseSingleExpression.java        |   91 -
 .../expression/BaseTerminalExpression.java      |   47 -
 .../phoenix/expression/CaseExpression.java      |  232 -
 .../expression/CeilingDecimalExpression.java    |   87 -
 .../expression/CeilingTimestampExpression.java  |   83 -
 .../phoenix/expression/CoerceExpression.java    |  150 -
 .../phoenix/expression/ColumnExpression.java    |  154 -
 .../expression/ComparisonExpression.java        |  155 -
 .../phoenix/expression/DateAddExpression.java   |   80 -
 .../expression/DateSubtractExpression.java      |   80 -
 .../expression/DecimalAddExpression.java        |  101 -
 .../expression/DecimalDivideExpression.java     |   79 -
 .../expression/DecimalMultiplyExpression.java   |   79 -
 .../expression/DecimalSubtractExpression.java   |  118 -
 .../phoenix/expression/DivideExpression.java    |   98 -
 .../phoenix/expression/DoubleAddExpression.java |   70 -
 .../expression/DoubleDivideExpression.java      |   74 -
 .../expression/DoubleMultiplyExpression.java    |   70 -
 .../expression/DoubleSubtractExpression.java    |   74 -
 .../apache/phoenix/expression/Expression.java   |   77 -
 .../phoenix/expression/ExpressionType.java      |  175 -
 .../expression/FloorDecimalExpression.java      |   51 -
 .../expression/FloorTimestampExpression.java    |   49 -
 .../phoenix/expression/InListExpression.java    |  291 -
 .../IndexKeyValueColumnExpression.java          |   26 -
 .../phoenix/expression/IsNullExpression.java    |  106 -
 .../expression/KeyValueColumnExpression.java    |  119 -
 .../phoenix/expression/LikeExpression.java      |  302 -
 .../phoenix/expression/LiteralExpression.java   |  260 -
 .../phoenix/expression/LongAddExpression.java   |   62 -
 .../expression/LongDivideExpression.java        |   66 -
 .../expression/LongMultiplyExpression.java      |   65 -
 .../expression/LongSubtractExpression.java      |   83 -
 .../phoenix/expression/MultiplyExpression.java  |   98 -
 .../phoenix/expression/NotExpression.java       |   86 -
 .../apache/phoenix/expression/OrExpression.java |   67 -
 .../phoenix/expression/OrderByExpression.java   |   87 -
 .../expression/RowKeyColumnExpression.java      |  150 -
 .../phoenix/expression/RowKeyExpression.java    |   25 -
 .../RowValueConstructorExpression.java          |  355 -
 .../expression/StringConcatExpression.java      |   95 -
 .../phoenix/expression/SubtractExpression.java  |   60 -
 .../expression/TimestampAddExpression.java      |   89 -
 .../expression/TimestampSubtractExpression.java |   89 -
 .../expression/aggregator/Aggregator.java       |   48 -
 .../expression/aggregator/Aggregators.java      |  125 -
 .../expression/aggregator/BaseAggregator.java   |   50 -
 .../aggregator/BaseDecimalStddevAggregator.java |  119 -
 .../aggregator/BaseStddevAggregator.java        |   96 -
 .../aggregator/ClientAggregators.java           |   80 -
 .../expression/aggregator/CountAggregator.java  |   84 -
 .../aggregator/DecimalStddevPopAggregator.java  |   43 -
 .../aggregator/DecimalStddevSampAggregator.java |   43 -
 .../aggregator/DecimalSumAggregator.java        |   91 -
 .../DistinctCountClientAggregator.java          |   55 -
 .../DistinctValueWithCountClientAggregator.java |  132 -
 .../DistinctValueWithCountServerAggregator.java |  174 -
 .../aggregator/DoubleSumAggregator.java         |   90 -
 .../expression/aggregator/IntSumAggregator.java |   43 -
 .../aggregator/LongSumAggregator.java           |   42 -
 .../expression/aggregator/MaxAggregator.java    |   50 -
 .../expression/aggregator/MinAggregator.java    |   98 -
 .../aggregator/NumberSumAggregator.java         |   99 -
 .../aggregator/PercentRankClientAggregator.java |   92 -
 .../aggregator/PercentileClientAggregator.java  |  110 -
 .../PercentileDiscClientAggregator.java         |  106 -
 .../aggregator/ServerAggregators.java           |  145 -
 .../aggregator/StddevPopAggregator.java         |   43 -
 .../aggregator/StddevSampAggregator.java        |   43 -
 .../aggregator/UnsignedIntSumAggregator.java    |   43 -
 .../aggregator/UnsignedLongSumAggregator.java   |   44 -
 .../expression/function/AggregateFunction.java  |   44 -
 .../function/AvgAggregateFunction.java          |  113 -
 .../expression/function/CoalesceFunction.java   |  100 -
 .../function/CompositeAggregateFunction.java    |   47 -
 .../function/CountAggregateFunction.java        |  106 -
 .../function/CurrentDateFunction.java           |   68 -
 .../function/CurrentTimeFunction.java           |   68 -
 ...elegateConstantToCountAggregateFunction.java |   75 -
 .../DistinctCountAggregateFunction.java         |  115 -
 .../function/FunctionArgumentType.java          |   35 -
 .../expression/function/FunctionExpression.java |   71 -
 .../function/IndexStateNameFunction.java        |   81 -
 .../expression/function/InvertFunction.java     |  129 -
 .../expression/function/LTrimFunction.java      |  115 -
 .../expression/function/LengthFunction.java     |   96 -
 .../expression/function/LowerFunction.java      |   60 -
 .../expression/function/MD5Function.java        |  100 -
 .../function/MaxAggregateFunction.java          |   75 -
 .../function/MinAggregateFunction.java          |   89 -
 .../function/PercentRankAggregateFunction.java  |   71 -
 .../PercentileContAggregateFunction.java        |   72 -
 .../PercentileDiscAggregateFunction.java        |   69 -
 .../expression/function/PrefixFunction.java     |   83 -
 .../expression/function/RTrimFunction.java      |  161 -
 .../function/RegexpReplaceFunction.java         |  130 -
 .../function/RegexpSubstrFunction.java          |  175 -
 .../expression/function/ReverseFunction.java    |   70 -
 .../expression/function/RoundFunction.java      |  238 -
 .../expression/function/ScalarFunction.java     |   72 -
 .../function/SingleAggregateFunction.java       |  166 -
 .../expression/function/SqlTableType.java       |   80 -
 .../function/SqlTypeNameFunction.java           |   86 -
 .../expression/function/StddevPopFunction.java  |   74 -
 .../expression/function/StddevSampFunction.java |   74 -
 .../expression/function/SubstrFunction.java     |  217 -
 .../function/SumAggregateFunction.java          |  146 -
 .../phoenix/expression/function/TimeUnit.java   |   22 -
 .../expression/function/ToCharFunction.java     |  135 -
 .../expression/function/ToDateFunction.java     |  133 -
 .../expression/function/ToNumberFunction.java   |  176 -
 .../expression/function/TrimFunction.java       |  108 -
 .../expression/function/TruncFunction.java      |   58 -
 .../expression/function/UpperFunction.java      |   58 -
 .../visitor/BaseExpressionVisitor.java          |  247 -
 .../expression/visitor/ExpressionVisitor.java   |  136 -
 .../visitor/KeyValueExpressionVisitor.java      |   38 -
 .../visitor/SingleAggregateFunctionVisitor.java |   40 -
 .../visitor/TraverseAllExpressionVisitor.java   |   57 -
 .../visitor/TraverseNoExpressionVisitor.java    |   35 -
 .../phoenix/filter/BooleanExpressionFilter.java |  125 -
 .../filter/EvaluateOnCompletionVisitor.java     |   76 -
 .../MultiCFCQKeyValueComparisonFilter.java      |  134 -
 .../filter/MultiCQKeyValueComparisonFilter.java |   70 -
 .../filter/MultiKeyValueComparisonFilter.java   |  237 -
 .../phoenix/filter/RowKeyComparisonFilter.java  |  150 -
 .../SingleCFCQKeyValueComparisonFilter.java     |   51 -
 .../SingleCQKeyValueComparisonFilter.java       |   49 -
 .../filter/SingleKeyValueComparisonFilter.java  |  147 -
 .../apache/phoenix/filter/SkipScanFilter.java   |  523 --
 .../apache/phoenix/index/BaseIndexCodec.java    |   61 -
 .../apache/phoenix/index/IndexMaintainer.java   |  780 --
 .../phoenix/index/IndexMetaDataCacheClient.java |   96 -
 .../index/IndexMetaDataCacheFactory.java        |   63 -
 .../phoenix/index/PhoenixIndexBuilder.java      |   96 -
 .../apache/phoenix/index/PhoenixIndexCodec.java |  172 -
 .../index/PhoenixIndexFailurePolicy.java        |  102 -
 .../iterate/AggregatingResultIterator.java      |   40 -
 .../phoenix/iterate/BaseResultIterator.java     |   43 -
 .../phoenix/iterate/ConcatResultIterator.java   |   91 -
 .../DefaultParallelIteratorRegionSplitter.java  |  229 -
 .../phoenix/iterate/DelegateResultIterator.java |   50 -
 .../DistinctAggregatingResultIterator.java      |  163 -
 .../apache/phoenix/iterate/ExplainTable.java    |  271 -
 .../FilterAggregatingResultIterator.java        |   80 -
 .../phoenix/iterate/FilterResultIterator.java   |   78 -
 .../GroupedAggregatingResultIterator.java       |  106 -
 .../phoenix/iterate/LimitingResultIterator.java |   57 -
 .../iterate/LookAheadResultIterator.java        |   53 -
 .../iterate/MappedByteBufferSortedQueue.java    |  381 -
 .../iterate/MaterializedResultIterator.java     |  112 -
 .../iterate/MergeSortResultIterator.java        |   95 -
 .../iterate/MergeSortRowKeyResultIterator.java  |   59 -
 .../iterate/MergeSortTopNResultIterator.java    |  100 -
 .../OrderedAggregatingResultIterator.java       |   63 -
 .../phoenix/iterate/OrderedResultIterator.java  |  254 -
 .../iterate/ParallelIteratorRegionSplitter.java |   36 -
 .../ParallelIteratorRegionSplitterFactory.java  |   40 -
 .../phoenix/iterate/ParallelIterators.java      |  232 -
 .../phoenix/iterate/PeekingResultIterator.java  |   63 -
 .../iterate/RegionScannerResultIterator.java    |   65 -
 .../apache/phoenix/iterate/ResultIterator.java  |   54 -
 .../apache/phoenix/iterate/ResultIterators.java |   29 -
 .../phoenix/iterate/ScanningResultIterator.java |   61 -
 ...SkipRangeParallelIteratorRegionSplitter.java |   85 -
 .../iterate/SpoolTooBigToDiskException.java     |   17 -
 .../phoenix/iterate/SpoolingResultIterator.java |  320 -
 .../phoenix/iterate/TableResultIterator.java    |   86 -
 .../UngroupedAggregatingResultIterator.java     |   56 -
 .../java/org/apache/phoenix/jdbc/Jdbc7Shim.java |   66 -
 .../apache/phoenix/jdbc/PhoenixConnection.java  |  639 --
 .../phoenix/jdbc/PhoenixDatabaseMetaData.java   | 1382 ----
 .../org/apache/phoenix/jdbc/PhoenixDriver.java  |   96 -
 .../phoenix/jdbc/PhoenixEmbeddedDriver.java     |  321 -
 .../phoenix/jdbc/PhoenixParameterMetaData.java  |  172 -
 .../phoenix/jdbc/PhoenixPreparedStatement.java  |  430 --
 .../apache/phoenix/jdbc/PhoenixResultSet.java   | 1231 ----
 .../phoenix/jdbc/PhoenixResultSetMetaData.java  |  213 -
 .../apache/phoenix/jdbc/PhoenixStatement.java   | 1239 ----
 .../phoenix/jdbc/PhoenixStatementFactory.java   |   24 -
 .../phoenix/job/AbstractRoundRobinQueue.java    |  314 -
 .../java/org/apache/phoenix/job/JobManager.java |  115 -
 .../apache/phoenix/join/HashCacheClient.java    |  136 -
 .../apache/phoenix/join/HashCacheFactory.java   |  127 -
 .../org/apache/phoenix/join/HashJoinInfo.java   |  126 -
 .../phoenix/join/HashJoiningRegionObserver.java |   57 -
 .../MaxServerCacheSizeExceededException.java    |   49 -
 .../phoenix/map/reduce/CSVBulkLoader.java       |  401 -
 .../apache/phoenix/map/reduce/MapReduceJob.java |  204 -
 .../phoenix/map/reduce/util/ConfigReader.java   |  114 -
 .../phoenix/memory/ChildMemoryManager.java      |  118 -
 .../phoenix/memory/DelegatingMemoryManager.java |   61 -
 .../phoenix/memory/GlobalMemoryManager.java     |  178 -
 .../memory/InsufficientMemoryException.java     |   47 -
 .../apache/phoenix/memory/MemoryManager.java    |   97 -
 .../apache/phoenix/optimize/QueryOptimizer.java |  239 -
 .../phoenix/parse/AddColumnStatement.java       |   49 -
 .../org/apache/phoenix/parse/AddParseNode.java  |   49 -
 .../parse/AggregateFunctionParseNode.java       |   39 -
 .../org/apache/phoenix/parse/AliasedNode.java   |   53 -
 .../phoenix/parse/AlterIndexStatement.java      |   53 -
 .../phoenix/parse/AlterTableStatement.java      |   28 -
 .../org/apache/phoenix/parse/AndParseNode.java  |   51 -
 .../phoenix/parse/ArithmeticParseNode.java      |   30 -
 .../phoenix/parse/AvgAggregateParseNode.java    |   48 -
 .../phoenix/parse/BaseParseNodeVisitor.java     |   70 -
 .../apache/phoenix/parse/BetweenParseNode.java  |   54 -
 .../apache/phoenix/parse/BinaryParseNode.java   |   44 -
 .../org/apache/phoenix/parse/BindParseNode.java |   61 -
 .../org/apache/phoenix/parse/BindTableNode.java |   44 -
 .../apache/phoenix/parse/BindableStatement.java |   25 -
 .../org/apache/phoenix/parse/CaseParseNode.java |   49 -
 .../org/apache/phoenix/parse/CastParseNode.java |   63 -
 .../org/apache/phoenix/parse/ColumnDef.java     |  147 -
 .../apache/phoenix/parse/ColumnFamilyDef.java   |   57 -
 .../org/apache/phoenix/parse/ColumnName.java    |  102 -
 .../apache/phoenix/parse/ColumnParseNode.java   |   90 -
 .../phoenix/parse/ComparisonParseNode.java      |   59 -
 .../apache/phoenix/parse/CompoundParseNode.java |   72 -
 .../apache/phoenix/parse/ConcreteTableNode.java |   43 -
 .../phoenix/parse/CreateIndexStatement.java     |   74 -
 .../phoenix/parse/CreateTableStatement.java     |   84 -
 .../phoenix/parse/CurrentDateParseNode.java     |   41 -
 .../phoenix/parse/CurrentTimeParseNode.java     |   41 -
 .../parse/DelegateConstantToCountParseNode.java |   48 -
 .../apache/phoenix/parse/DeleteStatement.java   |   69 -
 .../apache/phoenix/parse/DerivedTableNode.java  |   51 -
 .../phoenix/parse/DistinctCountParseNode.java   |   46 -
 .../apache/phoenix/parse/DivideParseNode.java   |   49 -
 .../phoenix/parse/DropColumnStatement.java      |   41 -
 .../phoenix/parse/DropIndexStatement.java       |   50 -
 .../phoenix/parse/DropTableStatement.java       |   51 -
 .../apache/phoenix/parse/EqualParseNode.java    |   47 -
 .../apache/phoenix/parse/ExistsParseNode.java   |   55 -
 .../apache/phoenix/parse/ExplainStatement.java  |   37 -
 .../phoenix/parse/FamilyWildcardParseNode.java  |   55 -
 .../phoenix/parse/FilterableStatement.java      |   31 -
 .../apache/phoenix/parse/FunctionParseNode.java |  427 --
 .../parse/GreaterThanOrEqualParseNode.java      |   49 -
 .../phoenix/parse/GreaterThanParseNode.java     |   48 -
 .../java/org/apache/phoenix/parse/HintNode.java |  160 -
 .../apache/phoenix/parse/InListParseNode.java   |   66 -
 .../org/apache/phoenix/parse/InParseNode.java   |   55 -
 .../phoenix/parse/IndexKeyConstraint.java       |   39 -
 .../apache/phoenix/parse/IsNullParseNode.java   |   55 -
 .../org/apache/phoenix/parse/JoinTableNode.java |   58 -
 .../phoenix/parse/LessThanOrEqualParseNode.java |   48 -
 .../apache/phoenix/parse/LessThanParseNode.java |   48 -
 .../org/apache/phoenix/parse/LikeParseNode.java |   55 -
 .../org/apache/phoenix/parse/LimitNode.java     |   40 -
 .../apache/phoenix/parse/LiteralParseNode.java  |   73 -
 .../phoenix/parse/MaxAggregateParseNode.java    |   41 -
 .../phoenix/parse/MinAggregateParseNode.java    |   41 -
 .../apache/phoenix/parse/MultiplyParseNode.java |   49 -
 .../org/apache/phoenix/parse/NamedNode.java     |   45 -
 .../apache/phoenix/parse/NamedParseNode.java    |   53 -
 .../apache/phoenix/parse/NamedTableNode.java    |   64 -
 .../apache/phoenix/parse/NotEqualParseNode.java |   48 -
 .../org/apache/phoenix/parse/NotParseNode.java  |   49 -
 .../org/apache/phoenix/parse/OrParseNode.java   |   49 -
 .../org/apache/phoenix/parse/OrderByNode.java   |   57 -
 .../phoenix/parse/OuterJoinParseNode.java       |   49 -
 .../org/apache/phoenix/parse/ParseContext.java  |   60 -
 .../apache/phoenix/parse/ParseException.java    |   47 -
 .../org/apache/phoenix/parse/ParseNode.java     |   52 -
 .../apache/phoenix/parse/ParseNodeFactory.java  |  534 --
 .../apache/phoenix/parse/ParseNodeRewriter.java |  504 --
 .../apache/phoenix/parse/ParseNodeVisitor.java  |  104 -
 .../phoenix/parse/PrimaryKeyConstraint.java     |   69 -
 .../org/apache/phoenix/parse/PropertyName.java  |   25 -
 .../parse/RowValueConstructorParseNode.java     |   48 -
 .../org/apache/phoenix/parse/SQLParser.java     |  173 -
 .../apache/phoenix/parse/SelectStatement.java   |  184 -
 .../phoenix/parse/SelectStatementRewriter.java  |  188 -
 .../phoenix/parse/ShowTablesStatement.java      |    8 -
 .../phoenix/parse/SingleTableSQLStatement.java  |   39 -
 .../StatelessTraverseAllParseNodeVisitor.java   |   86 -
 .../phoenix/parse/StringConcatParseNode.java    |   51 -
 .../apache/phoenix/parse/SubqueryParseNode.java |   49 -
 .../apache/phoenix/parse/SubtractParseNode.java |   48 -
 .../phoenix/parse/SumAggregateParseNode.java    |   41 -
 .../org/apache/phoenix/parse/TableName.java     |   56 -
 .../org/apache/phoenix/parse/TableNode.java     |   45 -
 .../apache/phoenix/parse/TableNodeVisitor.java  |   37 -
 .../apache/phoenix/parse/TerminalParseNode.java |   37 -
 .../apache/phoenix/parse/ToCharParseNode.java   |   65 -
 .../apache/phoenix/parse/ToDateParseNode.java   |   51 -
 .../apache/phoenix/parse/ToNumberParseNode.java |   47 -
 .../parse/TraverseAllParseNodeVisitor.java      |  148 -
 .../parse/TraverseNoParseNodeVisitor.java       |  227 -
 .../apache/phoenix/parse/UnaryParseNode.java    |   35 -
 .../parse/UnsupportedAllParseNodeVisitor.java   |  212 -
 .../apache/phoenix/parse/UpsertStatement.java   |   54 -
 .../apache/phoenix/parse/WildcardParseNode.java |   56 -
 .../apache/phoenix/pig/PhoenixHBaseStorage.java |  189 -
 .../phoenix/pig/PhoenixPigConfiguration.java    |  165 -
 .../java/org/apache/phoenix/pig/TypeUtil.java   |  175 -
 .../pig/hadoop/PhoenixOutputCommitter.java      |  113 -
 .../phoenix/pig/hadoop/PhoenixOutputFormat.java |   96 -
 .../phoenix/pig/hadoop/PhoenixRecord.java       |   91 -
 .../phoenix/pig/hadoop/PhoenixRecordWriter.java |   85 -
 .../phoenix/query/BaseQueryServicesImpl.java    |   80 -
 .../phoenix/query/ChildQueryServices.java       |   47 -
 .../phoenix/query/ConfigurationFactory.java     |   46 -
 .../phoenix/query/ConnectionQueryServices.java  |   84 -
 .../query/ConnectionQueryServicesImpl.java      | 1296 ----
 .../query/ConnectionlessQueryServicesImpl.java  |  236 -
 .../apache/phoenix/query/DegenerateScanner.java |   60 -
 .../query/DelegateConnectionQueryServices.java  |  171 -
 .../phoenix/query/DelegateQueryServices.java    |   74 -
 .../apache/phoenix/query/DelegateScanner.java   |   56 -
 .../phoenix/query/HBaseFactoryProvider.java     |   50 -
 .../org/apache/phoenix/query/HTableFactory.java |   58 -
 .../java/org/apache/phoenix/query/KeyRange.java |  623 --
 .../apache/phoenix/query/MetaDataMutated.java   |   40 -
 .../apache/phoenix/query/QueryConstants.java    |  173 -
 .../org/apache/phoenix/query/QueryServices.java |  115 -
 .../apache/phoenix/query/QueryServicesImpl.java |   38 -
 .../phoenix/query/QueryServicesOptions.java     |  363 -
 .../java/org/apache/phoenix/query/Scanner.java  |   35 -
 .../org/apache/phoenix/query/StatsManager.java  |   59 -
 .../apache/phoenix/query/StatsManagerImpl.java  |  221 -
 .../apache/phoenix/query/WrappedScanner.java    |   70 -
 .../schema/AmbiguousColumnException.java        |   52 -
 .../phoenix/schema/AmbiguousTableException.java |   52 -
 .../schema/ArgumentTypeMismatchException.java   |   44 -
 .../schema/ColumnAlreadyExistsException.java    |   62 -
 .../schema/ColumnFamilyNotFoundException.java   |   46 -
 .../apache/phoenix/schema/ColumnModifier.java   |  149 -
 .../phoenix/schema/ColumnNotFoundException.java |   65 -
 .../org/apache/phoenix/schema/ColumnRef.java    |  132 -
 .../ConcurrentTableMutationException.java       |   48 -
 .../schema/ConstraintViolationException.java    |   48 -
 .../apache/phoenix/schema/DelegateColumn.java   |   64 -
 .../apache/phoenix/schema/DelegateDatum.java    |   62 -
 .../ExecuteQueryNotApplicableException.java     |   39 -
 .../ExecuteUpdateNotApplicableException.java    |   38 -
 .../phoenix/schema/IllegalDataException.java    |   47 -
 .../apache/phoenix/schema/KeyValueSchema.java   |  207 -
 .../apache/phoenix/schema/MetaDataClient.java   | 1440 ----
 .../schema/MetaDataEntityNotFoundException.java |   30 -
 .../phoenix/schema/MetaDataSplitPolicy.java     |   51 -
 .../NewerTableAlreadyExistsException.java       |   29 -
 .../org/apache/phoenix/schema/PBaseColumn.java  |   52 -
 .../java/org/apache/phoenix/schema/PColumn.java |   46 -
 .../apache/phoenix/schema/PColumnFamily.java    |   54 -
 .../phoenix/schema/PColumnFamilyImpl.java       |   75 -
 .../org/apache/phoenix/schema/PColumnImpl.java  |  197 -
 .../org/apache/phoenix/schema/PDataType.java    | 4769 ------------
 .../org/apache/phoenix/schema/PDateColumn.java  |   27 -
 .../java/org/apache/phoenix/schema/PDatum.java  |   54 -
 .../org/apache/phoenix/schema/PIndexState.java  |   92 -
 .../apache/phoenix/schema/PIntegerColumn.java   |   35 -
 .../org/apache/phoenix/schema/PLongColumn.java  |   35 -
 .../org/apache/phoenix/schema/PMetaData.java    |   30 -
 .../apache/phoenix/schema/PMetaDataImpl.java    |  144 -
 .../java/org/apache/phoenix/schema/PName.java   |   80 -
 .../org/apache/phoenix/schema/PNameFactory.java |   75 -
 .../apache/phoenix/schema/PNormalizedName.java  |   72 -
 .../java/org/apache/phoenix/schema/PRow.java    |   70 -
 .../apache/phoenix/schema/PStringColumn.java    |   36 -
 .../java/org/apache/phoenix/schema/PTable.java  |  179 -
 .../org/apache/phoenix/schema/PTableImpl.java   |  740 --
 .../org/apache/phoenix/schema/PTableType.java   |  110 -
 .../phoenix/schema/ReadOnlyTableException.java  |   56 -
 .../org/apache/phoenix/schema/RowKeySchema.java |  243 -
 .../phoenix/schema/RowKeyValueAccessor.java     |  195 -
 .../org/apache/phoenix/schema/SaltingUtil.java  |  172 -
 .../schema/TableAlreadyExistsException.java     |   61 -
 .../phoenix/schema/TableNotFoundException.java  |   57 -
 .../org/apache/phoenix/schema/TableRef.java     |   81 -
 .../phoenix/schema/TypeMismatchException.java   |   52 -
 .../org/apache/phoenix/schema/ValueBitSet.java  |  200 -
 .../phoenix/schema/ValueRangeExcpetion.java     |   40 -
 .../org/apache/phoenix/schema/ValueSchema.java  |  311 -
 .../apache/phoenix/schema/stat/PTableStats.java |   47 -
 .../phoenix/schema/stat/PTableStatsImpl.java    |   70 -
 .../schema/tuple/MultiKeyValueTuple.java        |   75 -
 .../phoenix/schema/tuple/ResultTuple.java       |   94 -
 .../schema/tuple/SingleKeyValueTuple.java       |  102 -
 .../org/apache/phoenix/schema/tuple/Tuple.java  |   72 -
 .../org/apache/phoenix/util/BigDecimalUtil.java |   70 -
 .../java/org/apache/phoenix/util/BitSet.java    |  106 -
 .../java/org/apache/phoenix/util/ByteUtil.java  |  559 --
 .../java/org/apache/phoenix/util/CSVLoader.java |  250 -
 .../org/apache/phoenix/util/Closeables.java     |  124 -
 .../org/apache/phoenix/util/ColumnInfo.java     |   22 -
 .../java/org/apache/phoenix/util/DateUtil.java  |  154 -
 .../java/org/apache/phoenix/util/IndexUtil.java |  199 -
 .../apache/phoenix/util/InstanceResolver.java   |   66 -
 .../java/org/apache/phoenix/util/JDBCUtil.java  |   81 -
 .../org/apache/phoenix/util/KeyValueUtil.java   |  134 -
 .../org/apache/phoenix/util/MetaDataUtil.java   |  197 -
 .../org/apache/phoenix/util/NumberUtil.java     |   54 -
 .../org/apache/phoenix/util/PhoenixRuntime.java |  320 -
 .../java/org/apache/phoenix/util/QueryUtil.java |   68 -
 .../org/apache/phoenix/util/ReadOnlyProps.java  |  239 -
 .../org/apache/phoenix/util/ResultUtil.java     |  172 -
 .../org/apache/phoenix/util/SQLCloseable.java   |   34 -
 .../org/apache/phoenix/util/SQLCloseables.java  |  133 -
 .../java/org/apache/phoenix/util/ScanUtil.java  |  425 --
 .../org/apache/phoenix/util/SchemaUtil.java     |  823 ---
 .../org/apache/phoenix/util/ServerUtil.java     |  123 -
 .../java/org/apache/phoenix/util/SizedUtil.java |   63 -
 .../org/apache/phoenix/util/StringUtil.java     |  245 -
 .../util/TrustedByteArrayOutputStream.java      |   56 -
 .../java/org/apache/phoenix/util/TupleUtil.java |  155 -
 src/main/java/overview.html                     |   25 -
 src/main/resources/java.sql.Driver              |    1 -
 src/site/bin/merge.sh                           |   10 -
 .../language_reference_source/datatypes.html    |  493 --
 .../language_reference_source/functions.html    |  740 --
 src/site/language_reference_source/index.html   |  947 ---
 .../markdown/Phoenix-in-15-minutes-or-less.md   |   80 -
 src/site/markdown/building.md                   |   20 -
 src/site/markdown/download.md                   |   84 -
 src/site/markdown/dynamic_columns.md            |   17 -
 src/site/markdown/faq.md                        |  279 -
 src/site/markdown/flume.md                      |   42 -
 src/site/markdown/index.md                      |   69 -
 src/site/markdown/issues.md                     |    9 -
 src/site/markdown/mailing_list.md               |   14 -
 src/site/markdown/mr_dataload.md                |   63 -
 src/site/markdown/paged.md                      |   31 -
 src/site/markdown/performance.md                |   86 -
 src/site/markdown/phoenix_on_emr.md             |   43 -
 src/site/markdown/recent.md                     |   18 -
 src/site/markdown/roadmap.md                    |   35 -
 src/site/markdown/salted.md                     |   25 -
 src/site/markdown/secondary_indexing.md         |  152 -
 src/site/markdown/sequences.md                  |   47 -
 src/site/markdown/skip_scan.md                  |   22 -
 src/site/markdown/source.md                     |    9 -
 src/site/markdown/tuning.md                     |  128 -
 src/site/resources/css/site.css                 |   65 -
 src/site/resources/favicon.ico                  |  Bin 318 -> 0 bytes
 src/site/resources/images/PhoenixVsHive.png     |  Bin 30363 -> 0 bytes
 src/site/resources/images/PhoenixVsImpala.png   |  Bin 19219 -> 0 bytes
 src/site/resources/images/PhoenixVsOpenTSDB.png |  Bin 10039 -> 0 bytes
 src/site/resources/images/logo.png              |  Bin 17729 -> 0 bytes
 src/site/resources/images/perf-esscf.png        |  Bin 8081 -> 0 bytes
 src/site/resources/images/perf-salted-read.png  |  Bin 8086 -> 0 bytes
 src/site/resources/images/perf-salted-write.png |  Bin 8379 -> 0 bytes
 src/site/resources/images/perf-skipscan.png     |  Bin 7526 -> 0 bytes
 src/site/resources/images/perf-topn.png         |  Bin 7807 -> 0 bytes
 src/site/resources/images/psql.png              |  Bin 25997 -> 0 bytes
 src/site/resources/images/sqlline.png           |  Bin 57490 -> 0 bytes
 src/site/resources/images/squirrel.png          |  Bin 64649 -> 0 bytes
 src/site/resources/images/topbar-logo-small.png |  Bin 4931 -> 0 bytes
 src/site/resources/images/topbar-logo.png       |  Bin 7939 -> 0 bytes
 src/site/resources/language/images/div-d.png    |  Bin 8289 -> 0 bytes
 src/site/resources/language/images/div-ke.png   |  Bin 763 -> 0 bytes
 src/site/resources/language/images/div-ks.png   |  Bin 759 -> 0 bytes
 src/site/resources/language/images/div-le.png   |  Bin 387 -> 0 bytes
 src/site/resources/language/images/div-ls.png   |  Bin 396 -> 0 bytes
 src/site/resources/language/images/div-te.png   |  Bin 739 -> 0 bytes
 src/site/resources/language/images/div-ts.png   |  Bin 727 -> 0 bytes
 src/site/resources/language/stylesheet.css      |  139 -
 src/site/site.xml                               |   96 -
 src/site/xhtml/language/datatypes.xhtml         |    5 -
 src/site/xhtml/language/functions.xhtml         |    5 -
 src/site/xhtml/language/index.xhtml             |    5 -
 src/site/xhtml/team.xhtml                       |  115 -
 .../wal/TestReadWriteKeyValuesWithCodec.java    |  155 -
 ...ALReplayWithIndexWritesAndCompressedWAL.java |  275 -
 ...exWritesAndUncompressedWALInHBase_094_9.java |   24 -
 .../apache/hbase/index/IndexTestingUtils.java   |   96 -
 .../org/apache/hbase/index/StubAbortable.java   |   43 -
 .../java/org/apache/hbase/index/TableName.java  |   47 -
 .../TestFailForUnsupportedHBaseVersions.java    |  157 -
 .../covered/CoveredIndexCodecForTesting.java    |   73 -
 .../hbase/index/covered/TestCoveredColumns.java |   47 -
 .../TestEndToEndCoveredColumnsIndexBuilder.java |  341 -
 .../index/covered/TestLocalTableState.java      |  198 -
 .../index/covered/data/TestIndexMemStore.java   |   95 -
 .../covered/example/TestColumnTracker.java      |   63 -
 .../example/TestCoveredColumnIndexCodec.java    |  250 -
 .../TestCoveredIndexSpecifierBuilder.java       |   74 -
 .../example/TestEndToEndCoveredIndexing.java    |  879 ---
 .../TestEndtoEndIndexingWithCompression.java    |   51 -
 .../covered/example/TestFailWithoutRetries.java |  147 -
 .../filter/TestApplyAndFilterDeletesFilter.java |  212 -
 .../covered/filter/TestFamilyOnlyFilter.java    |  107 -
 .../filter/TestNewerTimestampFilter.java        |   49 -
 .../covered/update/TestIndexUpdateManager.java  |  142 -
 .../index/parallel/TestThreadPoolBuilder.java   |   65 -
 .../index/parallel/TestThreadPoolManager.java   |   95 -
 .../index/util/TestIndexManagementUtil.java     |   68 -
 .../hbase/index/write/FakeTableFactory.java     |   33 -
 .../index/write/TestCachingHTableFactory.java   |   60 -
 .../hbase/index/write/TestIndexWriter.java      |  286 -
 .../index/write/TestParalleIndexWriter.java     |  119 -
 .../write/TestParalleWriterIndexCommitter.java  |  119 -
 .../index/write/TestWALRecoveryCaching.java     |  370 -
 .../recovery/TestPerRegionIndexWriteCache.java  |  170 -
 .../arithmetic/ArithmeticOperationTest.java     |  298 -
 .../phoenix/compile/HavingClauseTest.java       |  193 -
 .../apache/phoenix/compile/LimitClauseTest.java |  156 -
 .../phoenix/compile/QueryCompileTest.java       | 1118 ---
 .../phoenix/compile/QueryMetaDataTest.java      |  447 --
 .../phoenix/compile/QueryOptimizerTest.java     |  262 -
 .../apache/phoenix/compile/ScanRangesTest.java  |  546 --
 .../compile/SelectStatementRewriterTest.java    |  118 -
 .../compile/StatementHintsCompilationTest.java  |  123 -
 .../phoenix/compile/WhereClauseFilterTest.java  |  911 ---
 .../phoenix/compile/WhereClauseScanKeyTest.java | 1618 ----
 .../apache/phoenix/end2end/AlterTableTest.java  |  699 --
 .../phoenix/end2end/ArithmeticQueryTest.java    |  594 --
 .../apache/phoenix/end2end/AutoCommitTest.java  |   83 -
 .../end2end/BaseClientMangedTimeTest.java       |   43 -
 .../phoenix/end2end/BaseConnectedQueryTest.java |  611 --
 .../end2end/BaseHBaseManagedTimeTest.java       |   43 -
 .../phoenix/end2end/BinaryRowKeyTest.java       |  149 -
 .../apache/phoenix/end2end/CSVLoaderTest.java   |  284 -
 .../phoenix/end2end/CoalesceFunctionTest.java   |   59 -
 .../end2end/CompareDecimalToLongTest.java       |  240 -
 .../ConnectionQueryServicesTestImpl.java        |  102 -
 .../apache/phoenix/end2end/CreateTableTest.java |   93 -
 .../phoenix/end2end/CustomEntityDataTest.java   |  216 -
 ...aultParallelIteratorsRegionSplitterTest.java |  358 -
 .../org/apache/phoenix/end2end/DeleteTest.java  |  385 -
 .../end2end/DescColumnSortOrderTest.java        |  555 --
 .../phoenix/end2end/DistinctCountTest.java      |  380 -
 .../phoenix/end2end/DynamicColumnTest.java      |  220 -
 .../phoenix/end2end/DynamicFamilyTest.java      |  362 -
 .../phoenix/end2end/DynamicUpsertTest.java      |  227 -
 .../phoenix/end2end/ExecuteStatementsTest.java  |  303 -
 .../phoenix/end2end/ExtendedQueryExecTest.java  |  190 -
 .../apache/phoenix/end2end/FunkyNamesTest.java  |  145 -
 .../apache/phoenix/end2end/GroupByCaseTest.java |  196 -
 .../phoenix/end2end/InMemoryOrderByTest.java    |   42 -
 .../org/apache/phoenix/end2end/IsNullTest.java  |   67 -
 .../org/apache/phoenix/end2end/KeyOnlyTest.java |  148 -
 .../apache/phoenix/end2end/MD5FunctionTest.java |   85 -
 .../phoenix/end2end/MultiCfQueryExecTest.java   |  231 -
 .../phoenix/end2end/NativeHBaseTypesTest.java   |  307 -
 .../org/apache/phoenix/end2end/OrderByTest.java |  125 -
 .../apache/phoenix/end2end/PercentileTest.java  |  621 --
 .../phoenix/end2end/ProductMetricsTest.java     | 2027 -----
 .../end2end/QueryDatabaseMetaDataTest.java      |  976 ---
 .../end2end/QueryExecWithoutSCNTest.java        |   51 -
 .../apache/phoenix/end2end/QueryPlanTest.java   |  203 -
 .../org/apache/phoenix/end2end/QueryTest.java   | 2712 -------
 .../phoenix/end2end/ReadIsolationLevelTest.java |  152 -
 .../phoenix/end2end/ReverseFunctionTest.java    |  109 -
 .../end2end/RowValueConstructorTest.java        |  812 --
 .../phoenix/end2end/ServerExceptionTest.java    |   69 -
 ...RangeParallelIteratorRegionSplitterTest.java |  369 -
 .../phoenix/end2end/SkipScanQueryTest.java      |  211 -
 .../phoenix/end2end/SpooledOrderByTest.java     |   40 -
 .../phoenix/end2end/StatementHintsTest.java     |  163 -
 .../org/apache/phoenix/end2end/StddevTest.java  |  137 -
 .../phoenix/end2end/ToCharFunctionTest.java     |  242 -
 .../phoenix/end2end/ToNumberFunctionTest.java   |  286 -
 .../org/apache/phoenix/end2end/TopNTest.java    |  160 -
 .../phoenix/end2end/UpsertBigValuesTest.java    |  371 -
 .../end2end/UpsertSelectAutoCommitTest.java     |   88 -
 .../phoenix/end2end/UpsertSelectTest.java       |  689 --
 .../phoenix/end2end/UpsertValuesTest.java       |  366 -
 .../phoenix/end2end/VariableLengthPKTest.java   | 1954 -----
 .../end2end/index/BaseMutableIndexTest.java     |  144 -
 .../phoenix/end2end/index/DropViewTest.java     |  101 -
 .../end2end/index/ImmutableIndexTest.java       |  525 --
 .../end2end/index/IndexMetadataTest.java        |  380 -
 .../phoenix/end2end/index/IndexTestUtil.java    |  162 -
 .../end2end/index/MutableIndexFailureTest.java  |  114 -
 .../phoenix/end2end/index/MutableIndexTest.java |  677 --
 .../end2end/index/MutableSaltedIndexTest.java   |  189 -
 .../phoenix/end2end/salted/SaltedTableTest.java |  528 --
 .../salted/SaltedTableUpsertSelectTest.java     |  203 -
 .../salted/SaltedTableVarLengthRowKeyTest.java  |   68 -
 .../expression/CoerceExpressionTest.java        |  121 -
 .../expression/ColumnExpressionTest.java        |  117 -
 .../DescColumnSortOrderExpressionTest.java      |  309 -
 .../phoenix/expression/LikeExpressionTest.java  |   44 -
 .../filter/SkipScanFilterIntersectTest.java     |  514 --
 .../phoenix/filter/SkipScanFilterTest.java      |  395 -
 .../phoenix/index/IndexMaintainerTest.java      |  227 -
 .../iterate/AggregateResultScannerTest.java     |  132 -
 .../iterate/ConcatResultIteratorTest.java       |  127 -
 .../iterate/MergeSortResultIteratorTest.java    |   83 -
 .../iterate/SpoolingResultIteratorTest.java     |   73 -
 .../phoenix/jdbc/PhoenixEmbeddedDriverTest.java |  103 -
 .../apache/phoenix/jdbc/PhoenixTestDriver.java  |   82 -
 .../apache/phoenix/logging/PerformanceLog.java  |  119 -
 .../phoenix/memory/MemoryManagerTest.java       |  269 -
 .../apache/phoenix/parse/QueryParserTest.java   |  619 --
 .../query/BaseConnectionlessQueryTest.java      |   90 -
 .../java/org/apache/phoenix/query/BaseTest.java |  425 --
 .../phoenix/query/ConnectionlessUpsertTest.java |  184 -
 .../phoenix/query/HBaseFactoryProviderTest.java |   38 -
 .../phoenix/query/KeyRangeCoalesceTests.java    |  143 -
 .../phoenix/query/KeyRangeIntersectTests.java   |   79 -
 .../phoenix/query/KeyRangeUnionTests.java       |   79 -
 .../phoenix/query/QueryServicesTestImpl.java    |   84 -
 .../apache/phoenix/schema/PDataTypeTest.java    | 1540 ----
 .../apache/phoenix/schema/RowKeySchemaTest.java |  131 -
 .../phoenix/schema/RowKeyValueAccessorTest.java |   69 -
 .../apache/phoenix/schema/SchemaUtilTest.java   |   52 -
 .../apache/phoenix/schema/ValueBitSetTest.java  |  137 -
 .../org/apache/phoenix/util/AssertResults.java  |  132 -
 .../org/apache/phoenix/util/ByteUtilTest.java   |   70 -
 .../org/apache/phoenix/util/DateUtilTest.java   |   64 -
 .../phoenix/util/GeneratePerformanceData.java   |   65 -
 .../apache/phoenix/util/LikeExpressionTest.java |   42 -
 .../apache/phoenix/util/MetaDataUtilTest.java   |   46 -
 .../org/apache/phoenix/util/ScanUtilTest.java   |  343 -
 .../java/org/apache/phoenix/util/TestUtil.java  |  326 -
 src/test/resources/hbase-default.xml            |   36 -
 src/test/resources/log4j.properties             |   63 -
 1599 files changed, 145908 insertions(+), 124657 deletions(-)
----------------------------------------------------------------------



[20/51] [partial] Initial commit of master branch from github

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/join/MaxServerCacheSizeExceededException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/join/MaxServerCacheSizeExceededException.java b/phoenix-core/src/main/java/org/apache/phoenix/join/MaxServerCacheSizeExceededException.java
new file mode 100644
index 0000000..e72d628
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/join/MaxServerCacheSizeExceededException.java
@@ -0,0 +1,49 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.join;
+
+/**
+ * 
+ * Exception thrown when the size of the hash cache exceeds the
+ * maximum size as specified by the phoenix.query.maxHashCacheBytes
+ * parameter in the {@link org.apache.hadoop.conf.Configuration}
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class MaxServerCacheSizeExceededException extends RuntimeException {
+	private static final long serialVersionUID = 1L;
+
+	public MaxServerCacheSizeExceededException() {
+    }
+
+    public MaxServerCacheSizeExceededException(String message) {
+        super(message);
+    }
+
+    public MaxServerCacheSizeExceededException(Throwable cause) {
+        super(cause);
+    }
+
+    public MaxServerCacheSizeExceededException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/join/ScanProjector.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/join/ScanProjector.java b/phoenix-core/src/main/java/org/apache/phoenix/join/ScanProjector.java
new file mode 100644
index 0000000..9f8777d
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/join/ScanProjector.java
@@ -0,0 +1,242 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.join;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.KeyValue;
+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.io.WritableUtils;
+
+import org.apache.phoenix.compile.JoinCompiler.ProjectedPTableWrapper;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.ExpressionType;
+import org.apache.phoenix.schema.KeyValueSchema;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.ValueBitSet;
+import org.apache.phoenix.schema.KeyValueSchema.KeyValueSchemaBuilder;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.KeyValueUtil;
+import org.apache.phoenix.util.SchemaUtil;
+
+public class ScanProjector {    
+    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;
+    private final Expression[] expressions;
+    private ValueBitSet valueSet;
+    private final ImmutableBytesWritable ptr = new ImmutableBytesWritable();
+    
+    public ScanProjector(ProjectedPTableWrapper projected) {
+    	List<PColumn> columns = projected.getTable().getColumns();
+    	expressions = new Expression[columns.size() - projected.getTable().getPKColumns().size()];
+    	// we do not count minNullableIndex for we might do later merge.
+    	KeyValueSchemaBuilder builder = new KeyValueSchemaBuilder(0);
+    	int i = 0;
+        for (PColumn column : projected.getTable().getColumns()) {
+        	if (!SchemaUtil.isPKColumn(column)) {
+        		builder.addField(column);
+        		expressions[i++] = projected.getSourceExpression(column);
+        	}
+        }
+        schema = builder.build();
+        valueSet = ValueBitSet.newInstance(schema);
+    }
+    
+    private ScanProjector(KeyValueSchema schema, Expression[] expressions) {
+    	this.schema = schema;
+    	this.expressions = expressions;
+    	this.valueSet = ValueBitSet.newInstance(schema);
+    }
+    
+    public void setValueBitSet(ValueBitSet bitSet) {
+        this.valueSet = bitSet;
+    }
+    
+    public static void serializeProjectorIntoScan(Scan scan, ScanProjector projector) {
+        ByteArrayOutputStream stream = new ByteArrayOutputStream();
+        try {
+            DataOutputStream output = new DataOutputStream(stream);
+            projector.schema.write(output);
+            int count = projector.expressions.length;
+            WritableUtils.writeVInt(output, count);
+            for (int i = 0; i < count; i++) {
+            	WritableUtils.writeVInt(output, ExpressionType.valueOf(projector.expressions[i]).ordinal());
+            	projector.expressions[i].write(output);
+            }
+            scan.setAttribute(SCAN_PROJECTOR, stream.toByteArray());
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        } finally {
+            try {
+                stream.close();
+            } catch (IOException e) {
+                throw new RuntimeException(e);
+            }
+        }
+        
+    }
+    
+    public static ScanProjector deserializeProjectorFromScan(Scan scan) {
+        byte[] proj = scan.getAttribute(SCAN_PROJECTOR);
+        if (proj == null) {
+            return null;
+        }
+        ByteArrayInputStream stream = new ByteArrayInputStream(proj);
+        try {
+            DataInputStream input = new DataInputStream(stream);
+            KeyValueSchema schema = new KeyValueSchema();
+            schema.readFields(input);
+            int count = WritableUtils.readVInt(input);
+            Expression[] expressions = new Expression[count];
+            for (int i = 0; i < count; i++) {
+            	int ordinal = WritableUtils.readVInt(input);
+            	expressions[i] = ExpressionType.values()[ordinal].newInstance();
+            	expressions[i].readFields(input);
+            }
+            return new ScanProjector(schema, expressions);
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        } finally {
+            try {
+                stream.close();
+            } catch (IOException e) {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+    
+    public static class ProjectedValueTuple implements Tuple {
+        private ImmutableBytesWritable keyPtr = new ImmutableBytesWritable();
+        private long timestamp;
+        private byte[] projectedValue;
+        private int bitSetLen;
+        private KeyValue keyValue;
+
+        private ProjectedValueTuple(byte[] keyBuffer, int keyOffset, int keyLength, long timestamp, byte[] projectedValue, int bitSetLen) {
+            this.keyPtr.set(keyBuffer, keyOffset, keyLength);
+            this.timestamp = timestamp;
+            this.projectedValue = projectedValue;
+            this.bitSetLen = bitSetLen;
+        }
+        
+        public ImmutableBytesWritable getKeyPtr() {
+            return keyPtr;
+        }
+        
+        public long getTimestamp() {
+            return timestamp;
+        }
+        
+        public byte[] getProjectedValue() {
+            return projectedValue;
+        }
+        
+        public int getBitSetLength() {
+            return bitSetLen;
+        }
+        
+        @Override
+        public void getKey(ImmutableBytesWritable ptr) {
+            ptr.set(keyPtr.get(), keyPtr.getOffset(), keyPtr.getLength());
+        }
+
+        @Override
+        public KeyValue getValue(int index) {
+            if (index != 0) {
+                throw new IndexOutOfBoundsException(Integer.toString(index));
+            }
+            return getValue(VALUE_COLUMN_FAMILY, 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, VALUE_COLUMN_QUALIFIER, timestamp, projectedValue, 0, projectedValue.length);
+            }
+            return keyValue;
+        }
+
+        @Override
+        public boolean getValue(byte[] family, byte[] qualifier,
+                ImmutableBytesWritable ptr) {
+            ptr.set(projectedValue);
+            return true;
+        }
+
+        @Override
+        public boolean isImmutable() {
+            return true;
+        }
+
+        @Override
+        public int size() {
+            return 1;
+        }
+    }
+    
+    public ProjectedValueTuple projectResults(Tuple tuple) {
+    	byte[] bytesValue = schema.toBytes(tuple, expressions, valueSet, ptr);
+    	KeyValue base = tuple.getValue(0);
+        return new ProjectedValueTuple(base.getBuffer(), base.getRowOffset(), base.getRowLength(), base.getTimestamp(), bytesValue, 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.");
+    }
+    
+    public static ProjectedValueTuple mergeProjectedValue(ProjectedValueTuple dest, KeyValueSchema destSchema, ValueBitSet destBitSet,
+    		Tuple src, KeyValueSchema srcSchema, ValueBitSet srcBitSet, int offset) throws IOException {
+    	ImmutableBytesWritable destValue = new ImmutableBytesWritable(dest.getProjectedValue());
+    	destBitSet.clear();
+    	destBitSet.or(destValue);
+    	int origDestBitSetLen = dest.getBitSetLength();
+    	ImmutableBytesWritable srcValue = new ImmutableBytesWritable();
+    	decodeProjectedValue(src, srcValue);
+    	srcBitSet.clear();
+    	srcBitSet.or(srcValue);
+    	int origSrcBitSetLen = srcBitSet.getEstimatedLength();
+    	for (int i = 0; i < srcBitSet.getMaxSetBit(); i++) {
+    		if (srcBitSet.get(i)) {
+    			destBitSet.set(offset + i);
+    		}
+    	}
+    	int destBitSetLen = destBitSet.getEstimatedLength();
+    	byte[] merged = new byte[destValue.getLength() - origDestBitSetLen + srcValue.getLength() - origSrcBitSetLen + destBitSetLen];
+    	int o = Bytes.putBytes(merged, 0, destValue.get(), destValue.getOffset(), destValue.getLength() - origDestBitSetLen);
+    	o = Bytes.putBytes(merged, o, srcValue.get(), srcValue.getOffset(), srcValue.getLength() - origSrcBitSetLen);
+    	destBitSet.toBytes(merged, o);
+    	ImmutableBytesWritable keyPtr = dest.getKeyPtr();
+        return new ProjectedValueTuple(keyPtr.get(), keyPtr.getOffset(), keyPtr.getLength(), dest.getTimestamp(), merged, destBitSetLen);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/map/reduce/CSVBulkLoader.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/map/reduce/CSVBulkLoader.java b/phoenix-core/src/main/java/org/apache/phoenix/map/reduce/CSVBulkLoader.java
new file mode 100644
index 0000000..e7c6159
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/map/reduce/CSVBulkLoader.java
@@ -0,0 +1,401 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.map.reduce;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.FileReader;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Date;
+import java.util.Map;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.PosixParser;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.mapreduce.HFileOutputFormat;
+import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
+import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+
+import com.google.common.io.Closeables;
+import org.apache.phoenix.map.reduce.util.ConfigReader;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.StringUtil;
+
+public class CSVBulkLoader {
+	private static final String UNDERSCORE = "_";
+	
+	static FileWriter wr = null;
+	static BufferedWriter bw = null;
+	static boolean isDebug = false; //Set to true, if you need to log the bulk-import time.
+	static ConfigReader systemConfig = null;
+
+	static String schemaName = "";
+	static String tableName = "";
+	static String idxTable = "";
+	static String createPSQL[] = null;
+	static String skipErrors = null;
+	static String zookeeperIP = null;
+	static String mapredIP = null;
+	static String hdfsNameNode = null;
+
+	static{
+		/** load the log-file writer, if debug is true **/
+		if(isDebug){
+			try {
+			    wr = new FileWriter("phoenix-bulk-import.log", false);
+			    bw = new BufferedWriter(wr);
+			} catch (IOException e) {
+			    System.err.println("Error preparing writer for log file :: " + e.getMessage());
+			}
+		}
+
+		/** load the Map-Reduce configs **/
+		try {
+			systemConfig = new ConfigReader("csv-bulk-load-config.properties");
+		} catch (Exception e) {
+			System.err.println("Exception occurred while reading config properties");
+			System.err.println("The bulk loader will run slower than estimated");
+		}
+	}
+	
+	/**
+	 * -i		CSV data file path in hdfs
+	 * -s		Phoenix schema name
+	 * -t		Phoenix table name
+	 * -sql  	Phoenix create table sql path (1 SQL statement per line)
+	 * -zk		Zookeeper IP:<port>
+	 * -mr		MapReduce Job Tracker IP:<port>
+	 * -hd		HDFS NameNode IP:<port>
+	 * -o		Output directory path in hdfs (Optional)
+	 * -idx  	Phoenix index table name (Optional)
+	 * -error    	Ignore error while reading rows from CSV ? (1 - YES/0 - NO, defaults to 1) (OPtional)
+	 * -help	Print all options (Optional)
+	 */
+
+	@SuppressWarnings("deprecation")
+    	public static void main(String[] args) throws Exception{
+		
+		String inputFile = null;
+		String outFile = null;
+
+		Options options = new Options();
+		options.addOption("i", true, "CSV data file path");
+		options.addOption("o", true, "Output directory path");
+		options.addOption("s", true, "Phoenix schema name");
+		options.addOption("t", true, "Phoenix table name");
+		options.addOption("idx", true, "Phoenix index table name");
+		options.addOption("zk", true, "Zookeeper IP:<port>");
+		options.addOption("mr", true, "MapReduce Job Tracker IP:<port>");
+		options.addOption("hd", true, "HDFS NameNode IP:<port>");
+		options.addOption("sql", true, "Phoenix create table sql path");
+		options.addOption("error", true, "Ignore error while reading rows from CSV ? (1 - YES/0 - NO, defaults to 1)");
+		options.addOption("help", false, "All options");
+		
+		CommandLineParser parser = new PosixParser();
+		CommandLine cmd = parser.parse( options, args);
+		
+		if(cmd.hasOption("help")){
+			HelpFormatter formatter = new HelpFormatter();
+			formatter.printHelp( "help", options );
+			System.exit(0);
+		}
+		
+		String parser_error = "ERROR while parsing arguments. ";
+		//CSV input, table name, sql and zookeeper IP  are mandatory fields
+		if(cmd.hasOption("i")){
+			inputFile = cmd.getOptionValue("i");
+		}else{
+			System.err.println(parser_error + "Please provide CSV file input path");
+			System.exit(0);
+		}
+		if(cmd.hasOption("t")){
+			tableName = cmd.getOptionValue("t");
+		}else{
+			System.err.println(parser_error + "Please provide Phoenix table name");
+			System.exit(0);
+		}
+		if(cmd.hasOption("sql")){
+			String sqlPath = cmd.getOptionValue("sql");
+			createPSQL = getCreatePSQLstmts(sqlPath);
+		}
+		if(cmd.hasOption("zk")){
+			zookeeperIP = cmd.getOptionValue("zk");
+		}else{
+			System.err.println(parser_error + "Please provide Zookeeper address");
+			System.exit(0);
+		}
+		if(cmd.hasOption("mr")){
+			mapredIP = cmd.getOptionValue("mr");
+		}else{
+			System.err.println(parser_error + "Please provide MapReduce address");
+			System.exit(0);
+		}
+		if(cmd.hasOption("hd")){
+			hdfsNameNode = cmd.getOptionValue("hd");
+		}else{
+			System.err.println(parser_error + "Please provide HDFS NameNode address");
+			System.exit(0);
+		}
+		
+		if(cmd.hasOption("o")){
+			outFile = cmd.getOptionValue("o");
+		}else{
+			outFile = "phoenix-output-dir";
+		}
+		if(cmd.hasOption("s")){
+			schemaName = cmd.getOptionValue("s");
+		}
+		if(cmd.hasOption("idx")){
+			idxTable = cmd.getOptionValue("idx");
+		}
+		if(cmd.hasOption("error")){
+			skipErrors = cmd.getOptionValue("error");
+		}else{
+			skipErrors = "1";
+		}
+		
+		log("[TS - START] :: " + new Date() + "\n");
+
+		Path inputPath = new Path(inputFile);
+		Path outPath = new Path(outFile);
+		
+		//Create the Phoenix table in HBase
+		if (createPSQL != null) {
+    		for(String s : createPSQL){
+    			if(s == null || s.trim().length() == 0) {
+    				continue;
+    			}
+				createTable(s);
+    		}
+    		
+    		log("[TS - Table created] :: " + new Date() + "\n");
+		}
+
+        String dataTable = ""; 
+        if(schemaName != null && schemaName.trim().length() > 0)
+            dataTable = SchemaUtil.normalizeIdentifier(schemaName) + "." + SchemaUtil.normalizeIdentifier(tableName);
+        else
+            dataTable = SchemaUtil.normalizeIdentifier(tableName);
+        
+        try {
+            validateTable();
+        } catch (SQLException e) {
+            System.err.println(e.getMessage());
+            System.exit(0);
+        }
+
+        Configuration conf = new Configuration();
+		loadMapRedConfigs(conf);
+		
+		Job job = new Job(conf, "MapReduce - Phoenix bulk import");
+		job.setJarByClass(MapReduceJob.class);
+		job.setInputFormatClass(TextInputFormat.class);
+		FileInputFormat.addInputPath(job, inputPath);
+		
+		FileSystem fs = FileSystem.get(conf);
+		fs.delete(outPath);
+		FileOutputFormat.setOutputPath(job, outPath);
+		
+		job.setMapperClass(MapReduceJob.PhoenixMapper.class);
+		job.setMapOutputKeyClass(ImmutableBytesWritable.class);
+		job.setMapOutputValueClass(KeyValue.class);
+		
+		SchemaMetrics.configureGlobally(conf);
+
+		HTable hDataTable = new HTable(conf, dataTable);
+		
+		// Auto configure partitioner and reducer according to the Main Data table
+    	HFileOutputFormat.configureIncrementalLoad(job, hDataTable);
+
+		job.waitForCompletion(true);
+	    
+		log("[TS - M-R HFile generated..Now dumping to HBase] :: " + new Date() + "\n");
+		
+    		LoadIncrementalHFiles loader = new LoadIncrementalHFiles(conf);
+    		loader.doBulkLoad(new Path(outFile), hDataTable);
+	    
+		log("[TS - FINISH] :: " + new Date() + "\n");
+		if(isDebug) bw.close();
+		
+	}
+	
+	private static void createTable(String stmt) {
+		
+		Connection conn = null;
+		PreparedStatement statement = null;
+
+		try {
+			conn = DriverManager.getConnection(getUrl(), "", "");
+			try {
+    			statement = conn.prepareStatement(stmt);
+    			statement.execute();
+    			conn.commit();
+			} finally {
+			    if(statement != null) {
+			        statement.close();
+			    }
+			}
+		} catch (Exception e) {
+			System.err.println("Error creating the table :: " + e.getMessage());
+		} finally{
+			try {
+			    if(conn != null) {
+			        conn.close();
+			    }
+			} catch (Exception e) {
+				System.err.println("Failed to close connection :: " + e.getMessage());
+			}
+		}
+	}
+
+	/**
+	 * Perform any required validation on the table being bulk loaded into:
+	 * - ensure no column family names start with '_', as they'd be ignored leading to problems.
+	 * @throws SQLException
+	 */
+    private static void validateTable() throws SQLException {
+        
+        Connection conn = DriverManager.getConnection(getUrl());
+        try {
+            ResultSet rs = conn.getMetaData().getColumns(null, StringUtil.escapeLike(schemaName), StringUtil.escapeLike(tableName), null);
+            while (rs.next()) {
+                String familyName = rs.getString(1);
+                if (familyName != null && familyName.startsWith(UNDERSCORE)) {
+                    String msg;
+                    if (QueryConstants.DEFAULT_COLUMN_FAMILY.equals(familyName)) {
+                        msg = "CSV Bulk Loader error: All column names that are not part of the primary key constraint must be prefixed with a column family name (i.e. f.my_column VARCHAR)";
+                    } else {
+                        msg = "CSV Bulk Loader error: Column family name must not start with '_': " + familyName;
+                    }
+                    throw new SQLException(msg);
+                }
+            }
+        } finally{
+             conn.close();
+        }
+    }
+    
+	private static String getUrl() {
+        	return PhoenixRuntime.JDBC_PROTOCOL + PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR + zookeeperIP;
+    	}
+	
+	private static void loadMapRedConfigs(Configuration conf){
+
+		conf.set("IGNORE.INVALID.ROW", skipErrors);
+		conf.set("schemaName", schemaName);
+		conf.set("tableName", tableName);
+		conf.set("zk", zookeeperIP);
+		conf.set("hbase.zookeeper.quorum", zookeeperIP);
+		conf.set("fs.default.name", hdfsNameNode);
+		conf.set("mapred.job.tracker", mapredIP);
+		
+		//Load the other System-Configs
+		try {
+			
+			Map<String, String> configs = systemConfig.getAllConfigMap();
+			
+			if(configs.containsKey("mapreduce.map.output.compress")){
+				String s = configs.get("mapreduce.map.output.compress");
+				if(s != null && s.trim().length() > 0)
+					conf.set("mapreduce.map.output.compress", s);
+			}
+			
+			if(configs.containsKey("mapreduce.map.output.compress.codec")){
+				String s = configs.get("mapreduce.map.output.compress.codec");
+				if(s != null && s.trim().length() > 0)
+					conf.set("mapreduce.map.output.compress.codec", s);
+			}
+			
+			if(configs.containsKey("io.sort.record.percent")){
+				String s = configs.get("io.sort.record.percent");
+				if(s != null && s.trim().length() > 0)
+					conf.set("io.sort.record.percent", s);	
+			}
+				
+			if(configs.containsKey("io.sort.factor")){
+				String s = configs.get("io.sort.factor");
+				if(s != null && s.trim().length() > 0)
+					conf.set("io.sort.factor", s);
+			}
+			
+			if(configs.containsKey("mapred.tasktracker.map.tasks.maximum")){
+				String s = configs.get("mapred.tasktracker.map.tasks.maximum");
+				if(s != null && s.trim().length() > 0)
+					conf.set("mapred.tasktracker.map.tasks.maximum", s);
+			}
+				
+		} catch (Exception e) {
+			System.err.println("Error loading the configs :: " + e.getMessage());
+			System.err.println("The bulk loader will run slower than estimated");
+		}
+	}
+	
+	private static String[] getCreatePSQLstmts(String path){
+		
+	    BufferedReader br = null;
+		try {
+			FileReader file = new FileReader(path);
+			br = new BufferedReader(file);
+			//Currently, we can have at-most 2 SQL statements - 1 for create table and 1 for index
+			String[] sb = new String[2];
+			String line;
+			for(int i = 0; i < 2 && (line = br.readLine()) != null ; i++){
+				sb[i] = line;
+			}
+			return sb;
+			
+		} catch (IOException e) {
+			System.err.println("Error reading the file :: " + path + ", " + e.getMessage());
+		} finally {
+		    if (br != null) Closeables.closeQuietly(br);
+		}
+		return null;
+	}
+	
+	private static void log(String msg){
+		if(isDebug){
+			try {
+				bw.write(msg);
+			} catch (IOException e) {
+				System.err.println("Error logging the statement :: " + msg);
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/map/reduce/MapReduceJob.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/map/reduce/MapReduceJob.java b/phoenix-core/src/main/java/org/apache/phoenix/map/reduce/MapReduceJob.java
new file mode 100644
index 0000000..6dd9910
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/map/reduce/MapReduceJob.java
@@ -0,0 +1,204 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.map.reduce;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import au.com.bytecode.opencsv.CSVReader;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Mapper;
+
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.QueryUtil;
+
+public class MapReduceJob {
+
+	public static class PhoenixMapper extends Mapper<LongWritable, Text, ImmutableBytesWritable, KeyValue>{
+		
+		private Connection conn_zk 	= null;
+		private PreparedStatement[] stmtCache;
+		private String tableName;
+		private String schemaName;
+		Map<Integer, Integer> colDetails = new LinkedHashMap<Integer, Integer>();
+		boolean ignoreUpsertError = true;
+		private String zookeeperIP;
+		
+		/**
+		 * Get the phoenix jdbc connection.
+		 */
+		
+		private static String getUrl(String url) {
+	        	return PhoenixRuntime.JDBC_PROTOCOL + PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR + url;
+	  	}
+		
+		/***
+		 * Get the column information from the table metaData.
+		 * Cretae a map of col-index and col-data-type.
+		 * Create the upsert Prepared Statement based on the map-size.
+		 */
+		
+		@Override
+		public void setup(Context context) throws InterruptedException{
+			Properties props = new Properties();
+			
+			try {
+				zookeeperIP 		= context.getConfiguration().get("zk");
+				
+				//ZK connection used to get the table meta-data
+				conn_zk				= DriverManager.getConnection(getUrl(zookeeperIP), props);
+				
+				schemaName			= context.getConfiguration().get("schemaName");
+				tableName 			= context.getConfiguration().get("tableName");
+				ignoreUpsertError 	= context.getConfiguration().get("IGNORE.INVALID.ROW").equalsIgnoreCase("0") ? false : true;
+				
+				//Get the resultset from the actual zookeeper connection. Connectionless mode throws "UnSupportedOperation" exception for this
+				ResultSet rs 		= conn_zk.getMetaData().getColumns(null, schemaName, tableName, null);
+				//This map holds the key-value pair of col-position and its data type
+				int i = 1;
+				while(rs.next()){
+					colDetails.put(i, rs.getInt(QueryUtil.DATA_TYPE_POSITION));
+					i++;
+				}
+				
+				stmtCache = new PreparedStatement[colDetails.size()];
+				ArrayList<String> cols = new ArrayList<String>();
+				for(i = 0 ; i < colDetails.size() ; i++){
+					cols.add("?");
+					String prepValues = StringUtils.join(cols, ",");
+					String upsertStmt = ""; 
+					if(schemaName != null && schemaName.trim().length() > 0)
+						upsertStmt = "upsert into " + schemaName + "." + tableName + " values (" + prepValues + ")";
+					else
+						upsertStmt = "upsert into " + tableName + " values (" + prepValues + ")";
+					try {
+						stmtCache[i] = conn_zk.prepareStatement(upsertStmt);
+					} catch (SQLException e) {
+						System.err.println("Error preparing the upsert statement" + e.getMessage());
+						if(!ignoreUpsertError){
+							throw (new InterruptedException(e.getMessage()));
+						}
+					}
+				}
+			} catch (SQLException e) {
+					System.err.println("Error occurred in connecting to Phoenix HBase" + e.getMessage());
+			}
+			
+	  	}
+		
+		/* Tokenize the text input line based on the "," delimeter.
+		*  TypeCast the token based on the col-data-type using the convertTypeSpecificValue API below.
+		*  Upsert the data. DO NOT COMMIT.
+		*  Use Phoenix's getUncommittedDataIterator API to parse the uncommited data to KeyValue pairs.
+		*  Emit the row-key and KeyValue pairs from Mapper to allow sorting based on row-key.
+		*  Finally, do connection.rollback( to preserve table state).
+		*/
+		
+		@Override
+		public void map(LongWritable key, Text line, Context context) throws IOException, InterruptedException{
+			
+			CSVReader reader = new CSVReader(new InputStreamReader(new ByteArrayInputStream(line.toString().getBytes())), ',');			
+			try {
+				String[] tokens = reader.readNext();
+				
+				PreparedStatement upsertStatement;
+				if(tokens.length >= stmtCache.length){
+					//If CVS values are more than the number of cols in the table, apply the col count cap
+					upsertStatement = stmtCache[stmtCache.length - 1];
+				}else{
+					//Else, take the corresponding upsertStmt from cached array 
+					upsertStatement = stmtCache[tokens.length - 1];
+				}
+
+				for(int i = 0 ; i < tokens.length && i < colDetails.size() ;i++){
+					upsertStatement.setObject(i+1, convertTypeSpecificValue(tokens[i], colDetails.get(new Integer(i+1))));
+				}
+				
+				upsertStatement.execute();
+			} catch (SQLException e) {
+				System.err.println("Failed to upsert data in the Phoenix :: " + e.getMessage());
+				if(!ignoreUpsertError){
+					throw (new InterruptedException(e.getMessage()));
+				}
+			} catch (Exception e) {
+				System.err.println("Failed to upsert data in the Phoenix :: " + e.getMessage());
+			}finally {
+				reader.close();
+       			} 
+			
+			Iterator<Pair<byte[],List<KeyValue>>> dataIterator = null;
+			try {
+				dataIterator = PhoenixRuntime.getUncommittedDataIterator(conn_zk);
+			} catch (SQLException e) {
+				System.err.println("Failed to retrieve the data iterator for Phoenix table :: " + e.getMessage());
+			}
+			
+			while(dataIterator != null && dataIterator.hasNext()){
+				Pair<byte[],List<KeyValue>> row = dataIterator.next();
+				for(KeyValue kv : row.getSecond()){
+					context.write(new ImmutableBytesWritable(kv.getRow()), kv);
+				}
+			}
+			
+			try {
+			    conn_zk.rollback();
+			} catch (SQLException e) {
+				System.err.println("Transaction rollback failed.");
+			}
+		}
+		
+		/*
+		* Do connection.close()
+		*/
+		
+		@Override
+		public void cleanup(Context context) {
+	  		try {
+	  			conn_zk.close();
+			} catch (SQLException e) {
+				System.err.println("Failed to close the JDBC connection");
+			}
+	  	}
+		
+		private Object convertTypeSpecificValue(String s, Integer sqlType) throws Exception {
+			return PDataType.fromTypeId(sqlType).toObject(s);
+		}
+	}
+	
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/map/reduce/util/ConfigReader.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/map/reduce/util/ConfigReader.java b/phoenix-core/src/main/java/org/apache/phoenix/map/reduce/util/ConfigReader.java
new file mode 100644
index 0000000..8f739f5
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/map/reduce/util/ConfigReader.java
@@ -0,0 +1,114 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.map.reduce.util;
+
+import java.util.Enumeration;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ResourceBundle;
+
+/**
+ * Class to read configs.
+ * 
+ */
+
+public class ConfigReader
+ {
+
+	private String propertyFile = null;
+	private boolean loaded = false;
+	private static final Object _synObj = new Object();
+	private Map<String, String> properties = new HashMap<String, String>();
+	private Exception loadException = null;
+
+	/**
+	 * Retrieves singleton config objects from a hashmap of stored objects,
+	 * creates these objects if they aren't in the hashmap.
+	 */
+
+	public ConfigReader(String propertyFile) {
+		this.propertyFile = propertyFile;
+	}
+
+	public void load() throws Exception {
+		if (loaded) {
+			if (loadException != null) {
+				throw new Exception(loadException);
+			}
+			return;
+		}
+		synchronized (_synObj) {
+			if (!loaded) {
+				try {
+					String tmpFile = propertyFile.trim();
+					if (tmpFile.endsWith(".properties")) {
+						tmpFile = tmpFile
+								.substring(0, tmpFile.lastIndexOf("."));
+					}
+					ResourceBundle resource = ResourceBundle.getBundle(tmpFile);
+					Enumeration<String> enm = resource.getKeys();
+
+					while (enm.hasMoreElements()) {
+						String key = enm.nextElement();
+						String value = resource.getString(key);
+						properties.put(key, value);
+					}
+				} catch (Exception e) {
+					System.err
+							.println("Exception while loading the config.properties file :: "
+									+ e.getMessage());
+					loadException = e;
+					loaded = true;
+					throw e;
+				}
+				loaded = true;
+			}
+		}
+	}
+
+	public void addConfig(String key, String value) {
+		try {
+			load();
+		} catch (Exception e) {
+			System.err.println("ERROR :: " + e.getMessage());
+		}
+		properties.put(key, value);
+	}
+
+	public boolean hasConfig(String key) {
+		try {
+			load();
+		} catch (Exception e) {
+			System.err.println("ERROR :: " + e.getMessage());
+		}
+		return properties.containsKey(key);
+	}
+
+	public String getConfig(String key) throws Exception {
+		load();
+		return properties.get(key);
+	}
+
+	public Map<String, String> getAllConfigMap() throws Exception {
+		load();
+		return properties;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/memory/ChildMemoryManager.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/memory/ChildMemoryManager.java b/phoenix-core/src/main/java/org/apache/phoenix/memory/ChildMemoryManager.java
new file mode 100644
index 0000000..925cdfd
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/memory/ChildMemoryManager.java
@@ -0,0 +1,118 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.memory;
+
+import org.apache.http.annotation.GuardedBy;
+import org.apache.http.annotation.ThreadSafe;
+
+/**
+ * 
+ * Child memory manager that delegates through to global memory manager,
+ * but enforces that at most a threshold percentage is used by this
+ * memory manager.  No blocking is done if the threshold is exceeded,
+ * but the standard blocking will be done by the global memory manager.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+@ThreadSafe
+public class ChildMemoryManager extends DelegatingMemoryManager {
+    private final Object sync = new Object();
+    private final int maxPercOfTotal;
+    @GuardedBy("sync")
+    private long allocatedBytes;
+    
+    public ChildMemoryManager(MemoryManager mm, int maxPercOfTotal) {
+        super(mm);
+        if (mm instanceof ChildMemoryManager) {
+            throw new IllegalStateException("ChildMemoryManager cannot delegate to another ChildMemoryManager");
+        }
+        this.maxPercOfTotal = maxPercOfTotal;
+        if (maxPercOfTotal <= 0 || maxPercOfTotal > 100) {
+            throw new IllegalArgumentException("Max percentage of total memory (" + maxPercOfTotal + "%) must be greater than zero and less than or equal to 100");
+        }
+    }
+
+
+    private long adjustAllocation(long minBytes, long reqBytes) {
+        assert(reqBytes >= minBytes);
+        long availBytes = getAvailableMemory();
+        // Check if this memory managers percentage of allocated bytes exceeds its allowed maximum
+        if (minBytes > availBytes) {
+            throw new InsufficientMemoryException("Attempt to allocate more memory than the max allowed of " + maxPercOfTotal + "%");
+        }
+        // Revise reqBytes down to available memory if necessary
+        return Math.min(reqBytes,availBytes);
+    }
+    
+    @Override
+    public MemoryChunk allocate(long minBytes, long nBytes) {
+        synchronized (sync) {
+            nBytes = adjustAllocation(minBytes, nBytes);
+            final MemoryChunk chunk = super.allocate(minBytes, nBytes);
+            allocatedBytes += chunk.getSize();
+            // Instantiate delegate chunk to track allocatedBytes correctly
+            return new MemoryChunk() {
+                @Override
+                public void close() {
+                    synchronized (sync) {
+                        allocatedBytes -= chunk.getSize();
+                        chunk.close();
+                    }
+                }
+    
+                @Override
+                public long getSize() {
+                    return chunk.getSize();
+                }
+    
+                @Override
+                public void resize(long nBytes) {
+                    synchronized (sync) {
+                        long size = getSize();
+                        long deltaBytes = nBytes - size;
+                        if (deltaBytes > 0) {
+                            adjustAllocation(deltaBytes,deltaBytes); // Throw if too much memory
+                        }
+                        chunk.resize(nBytes);
+                        allocatedBytes += deltaBytes;
+                    }
+                }
+            };
+        }
+    }
+
+    @Override
+    public long getAvailableMemory() {
+        synchronized (sync) {
+            long availBytes = getMaxMemory() - allocatedBytes;
+            // Sanity check (should never happen)
+            if (availBytes < 0) {
+                throw new IllegalStateException("Available memory has become negative: " + availBytes + " bytes.  Allocated memory: " + allocatedBytes + " bytes.");
+            }
+            return availBytes;
+        }
+    }
+    
+    @Override
+    public long getMaxMemory() {
+        return maxPercOfTotal  * super.getMaxMemory() / 100;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/memory/DelegatingMemoryManager.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/memory/DelegatingMemoryManager.java b/phoenix-core/src/main/java/org/apache/phoenix/memory/DelegatingMemoryManager.java
new file mode 100644
index 0000000..f50c43d
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/memory/DelegatingMemoryManager.java
@@ -0,0 +1,61 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.memory;
+
+/**
+ * 
+ * Memory manager that delegates through to another memory manager.
+ * 
+ * @author jtaylor
+ * @since 0.1
+ */
+public class DelegatingMemoryManager implements MemoryManager {
+    private final MemoryManager parent;
+    
+    public DelegatingMemoryManager(MemoryManager globalMemoryManager){
+        this.parent = globalMemoryManager;
+    }
+    
+    @Override
+    public long getAvailableMemory() {
+        return parent.getAvailableMemory();
+    }
+
+    @Override
+    public long getMaxMemory() {
+        return parent.getMaxMemory();
+    }
+
+    @Override
+    public MemoryChunk allocate(long minBytes, long reqBytes) {
+        return parent.allocate(minBytes, reqBytes);
+    }
+
+
+    @Override
+    public MemoryChunk allocate(long nBytes) {
+        return allocate(nBytes, nBytes);
+    }
+
+    public MemoryManager getParent() {
+        return parent;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/memory/GlobalMemoryManager.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/memory/GlobalMemoryManager.java b/phoenix-core/src/main/java/org/apache/phoenix/memory/GlobalMemoryManager.java
new file mode 100644
index 0000000..38a3cd9
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/memory/GlobalMemoryManager.java
@@ -0,0 +1,178 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.memory;
+
+import org.apache.http.annotation.GuardedBy;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * 
+ * Global memory manager to track course grained memory usage across all requests.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class GlobalMemoryManager implements MemoryManager {
+    private static final Logger logger = LoggerFactory.getLogger(GlobalMemoryManager.class);
+    
+    private final Object sync = new Object();
+    private final long maxMemoryBytes;
+    private final int maxWaitMs;
+    @GuardedBy("sync")
+    private volatile long usedMemoryBytes;
+    
+    public GlobalMemoryManager(long maxBytes, int maxWaitMs) {
+        if (maxBytes <= 0) {
+            throw new IllegalStateException("Total number of available bytes (" + maxBytes + ") must be greater than zero");
+        }
+        if (maxWaitMs < 0) {
+            throw new IllegalStateException("Maximum wait time (" + maxWaitMs + ") must be greater than or equal to zero");
+        }
+        this.maxMemoryBytes = maxBytes;
+        this.maxWaitMs = maxWaitMs;
+        this.usedMemoryBytes = 0;
+    }
+    
+    @Override
+    public long getAvailableMemory() {
+        synchronized(sync) {
+            return maxMemoryBytes - usedMemoryBytes;
+        }
+    }
+
+    @Override
+    public long getMaxMemory() {
+        return maxMemoryBytes;
+    }
+
+
+    // TODO: Work on fairness: One big memory request can cause all others to block here.
+    private long allocateBytes(long minBytes, long reqBytes) {
+        if (minBytes < 0 || reqBytes < 0) {
+            throw new IllegalStateException("Minimum requested bytes (" + minBytes + ") and requested bytes (" + reqBytes + ") must be greater than zero");
+        }
+        if (minBytes > maxMemoryBytes) { // No need to wait, since we'll never have this much available
+            throw new InsufficientMemoryException("Requested memory of " + minBytes + " bytes is larger than global pool of " + maxMemoryBytes + " bytes.");
+        }
+        long startTimeMs = System.currentTimeMillis(); // Get time outside of sync block to account for waiting for lock
+        long nBytes;
+        synchronized(sync) {
+            while (usedMemoryBytes + minBytes > maxMemoryBytes) { // Only wait if minBytes not available
+                try {
+                    long remainingWaitTimeMs = maxWaitMs - (System.currentTimeMillis() - startTimeMs);
+                    if (remainingWaitTimeMs <= 0) { // Ran out of time waiting for some memory to get freed up
+                        throw new InsufficientMemoryException("Requested memory of " + minBytes + " bytes could not be allocated from remaining memory of " + usedMemoryBytes + " bytes from global pool of " + maxMemoryBytes + " bytes after waiting for " + maxWaitMs + "ms.");
+                    }
+                    sync.wait(remainingWaitTimeMs);
+                } catch (InterruptedException ie) {
+                    throw new RuntimeException("Interrupted allocation of " + minBytes + " bytes", ie);
+                }
+            }
+            // Allocate at most reqBytes, but at least minBytes
+            nBytes = Math.min(reqBytes, maxMemoryBytes - usedMemoryBytes);
+            if (nBytes < minBytes) {
+                throw new IllegalStateException("Allocated bytes (" + nBytes + ") should be at least the minimum requested bytes (" + minBytes + ")");
+            }
+            usedMemoryBytes += nBytes;
+        }
+        return nBytes;
+    }
+
+    @Override
+    public MemoryChunk allocate(long minBytes, long reqBytes) {
+        long nBytes = allocateBytes(minBytes, reqBytes);
+        return newMemoryChunk(nBytes);
+    }
+
+    @Override
+    public MemoryChunk allocate(long nBytes) {
+        return allocate(nBytes,nBytes);
+    }
+
+    protected MemoryChunk newMemoryChunk(long sizeBytes) {
+        return new GlobalMemoryChunk(sizeBytes);
+    }
+    
+    private class GlobalMemoryChunk implements MemoryChunk {
+        private volatile long size;
+
+        private GlobalMemoryChunk(long size) {
+            if (size < 0) {
+                throw new IllegalStateException("Size of memory chunk must be greater than zero, but instead is " + size);
+            }
+            this.size = size;
+        }
+
+        @Override
+        public long getSize() {
+            synchronized(sync) {
+                return size; // TODO: does this need to be synchronized?
+            }
+        }
+        
+        @Override
+        public void resize(long nBytes) {
+            if (nBytes < 0) {
+                throw new IllegalStateException("Number of bytes to resize to must be greater than zero, but instead is " + nBytes);
+            }
+            synchronized(sync) {
+                long nAdditionalBytes = (nBytes - size);
+                if (nAdditionalBytes < 0) {
+                    usedMemoryBytes += nAdditionalBytes;
+                    size = nBytes;
+                    sync.notifyAll();
+                } else {
+                    allocateBytes(nAdditionalBytes, nAdditionalBytes);
+                    size = nBytes;
+                }
+            }
+        }
+        
+        /**
+         * Check that MemoryChunk has previously been closed.
+         */
+        @Override
+        protected void finalize() throws Throwable {
+            try {
+                close();
+                if (size > 0) {
+                    logger.warn("Orphaned chunk of " + size + " bytes found during finalize");
+                }
+                // TODO: log error here, but we can't use SFDC logging
+                // because this runs in an hbase coprocessor.
+                // Create a gack-like API (talk with GridForce or HBase folks)
+            } finally {
+                super.finalize();
+            }
+        }
+        
+        @Override
+        public void close() {
+            synchronized(sync) {
+                usedMemoryBytes -= size;
+                size = 0;
+                sync.notifyAll();
+            }
+        }
+    }
+}
+ 

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/memory/InsufficientMemoryException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/memory/InsufficientMemoryException.java b/phoenix-core/src/main/java/org/apache/phoenix/memory/InsufficientMemoryException.java
new file mode 100644
index 0000000..8e7eaff
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/memory/InsufficientMemoryException.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.memory;
+
+/**
+ * 
+ * Exception thrown by MemoryManager when insufficient memory is available
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class InsufficientMemoryException extends RuntimeException {
+	private static final long serialVersionUID = 1L;
+
+    public InsufficientMemoryException() {
+    }
+
+    public InsufficientMemoryException(String message) {
+        super(message);
+    }
+
+    public InsufficientMemoryException(Throwable cause) {
+        super(cause);
+    }
+
+    public InsufficientMemoryException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/memory/MemoryManager.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/memory/MemoryManager.java b/phoenix-core/src/main/java/org/apache/phoenix/memory/MemoryManager.java
new file mode 100644
index 0000000..e9b9355
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/memory/MemoryManager.java
@@ -0,0 +1,97 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.memory;
+
+import java.io.Closeable;
+
+/**
+ * 
+ * Memory manager used to track memory usage.  Either throttles
+ * memory usage by blocking when the max memory is reached or
+ * allocates up to a maximum without blocking.
+ * 
+ * @author jtaylor
+ * @since 0.1
+ */
+public interface MemoryManager {
+    /**
+     * Get the total amount of memory (in bytes) that may be allocated.
+     */
+    long getMaxMemory();
+    
+    /**
+     * Get the amount of available memory (in bytes) not yet allocated.
+     */
+    long getAvailableMemory();
+    
+    /**
+     * Allocate up to reqBytes of memory, dialing the amount down to 
+     * minBytes if full amount is not available.  If minBytes is not
+     * available, then this call will block for a configurable amount
+     * of time and throw if minBytes does not become available.
+     * @param minBytes minimum number of bytes required
+     * @param reqBytes requested number of bytes.  Must be greater
+     * than or equal to minBytes
+     * @return MemoryChunk that was allocated
+     * @throws InsufficientMemoryException if unable to allocate minBytes
+     *  during configured amount of time
+     */
+    MemoryChunk allocate(long minBytes, long reqBytes);
+
+    /**
+     * Equivalent to calling {@link #allocate(long, long)} where
+     * minBytes and reqBytes being the same.
+     */
+    MemoryChunk allocate(long nBytes);
+    
+    /**
+     * 
+     * Chunk of allocated memory.  To reclaim the memory, call {@link #close()}
+     *
+     * @author jtaylor
+     * @since 0.1
+     */
+    public static interface MemoryChunk extends Closeable {
+        /**
+         * Get the size in bytes of the allocated chunk.
+         */
+        long getSize();
+        
+        /**
+         * Free up the memory associated with this chunk
+         */
+        @Override
+        void close();
+        
+        /**
+         * Resize an already allocated memory chunk up or down to a
+         * new amount.  If decreasing allocation, this call will not block.
+         * If increasing allocation, and nBytes is not available,  then
+         * this call will block for a configurable amount of time and
+         * throw if nBytes does not become available.  Most commonly
+         * used to adjust the allocation of a memory buffer that was
+         * originally sized for the worst case scenario.
+         * @param nBytes new number of bytes required for this chunk
+         * @throws InsufficientMemoryException if unable to allocate minBytes
+         *  during configured amount of time
+         */
+        void resize(long nBytes); 
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/optimize/QueryOptimizer.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/optimize/QueryOptimizer.java b/phoenix-core/src/main/java/org/apache/phoenix/optimize/QueryOptimizer.java
new file mode 100644
index 0000000..b60f742
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/optimize/QueryOptimizer.java
@@ -0,0 +1,239 @@
+package org.apache.phoenix.optimize;
+
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+
+import com.google.common.collect.Lists;
+import org.apache.phoenix.compile.ColumnProjector;
+import org.apache.phoenix.compile.IndexStatementRewriter;
+import org.apache.phoenix.compile.QueryCompiler;
+import org.apache.phoenix.compile.QueryPlan;
+import org.apache.phoenix.iterate.ParallelIterators.ParallelIteratorFactory;
+import org.apache.phoenix.jdbc.PhoenixStatement;
+import org.apache.phoenix.parse.HintNode;
+import org.apache.phoenix.parse.HintNode.Hint;
+import org.apache.phoenix.parse.ParseNodeFactory;
+import org.apache.phoenix.parse.SelectStatement;
+import org.apache.phoenix.parse.TableNode;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.ColumnNotFoundException;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PDatum;
+import org.apache.phoenix.schema.PIndexState;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTableType;
+
+public class QueryOptimizer {
+    private static final ParseNodeFactory FACTORY = new ParseNodeFactory();
+
+    private final QueryServices services;
+    private final boolean useIndexes;
+
+    public QueryOptimizer(QueryServices services) {
+        this.services = services;
+        this.useIndexes = this.services.getProps().getBoolean(QueryServices.USE_INDEXES_ATTRIB, QueryServicesOptions.DEFAULT_USE_INDEXES);
+    }
+
+    public QueryPlan optimize(SelectStatement select, PhoenixStatement statement) throws SQLException {
+        return optimize(select, statement, Collections.<PColumn>emptyList(), null);
+    }
+
+    public QueryPlan optimize(SelectStatement select, PhoenixStatement statement, List<? extends PDatum> targetColumns, ParallelIteratorFactory parallelIteratorFactory) throws SQLException {
+        QueryCompiler compiler = new QueryCompiler(statement, targetColumns, parallelIteratorFactory);
+        QueryPlan dataPlan = compiler.compile(select);
+        if (!useIndexes || select.getFrom().size() > 1) {
+            return dataPlan;
+        }
+        // Get the statement as it's been normalized now
+        // TODO: the recompile for the index tables could skip the normalize step
+        select = (SelectStatement)dataPlan.getStatement();
+        PTable dataTable = dataPlan.getTableRef().getTable();
+        List<PTable>indexes = Lists.newArrayList(dataTable.getIndexes());
+        if (indexes.isEmpty() || dataPlan.getTableRef().hasDynamicCols() || select.getHint().hasHint(Hint.NO_INDEX)) {
+            return dataPlan;
+        }
+        
+        // The targetColumns is set for UPSERT SELECT to ensure that the proper type conversion takes place.
+        // For a SELECT, it is empty. In this case, we want to set the targetColumns to match the projection
+        // from the dataPlan to ensure that the metadata for when an index is used matches the metadata for
+        // when the data table is used.
+        if (targetColumns.isEmpty()) {
+            List<? extends ColumnProjector> projectors = dataPlan.getProjector().getColumnProjectors();
+            List<PDatum> targetDatums = Lists.newArrayListWithExpectedSize(projectors.size());
+            for (ColumnProjector projector : projectors) {
+                targetDatums.add(projector.getExpression());
+            }
+            targetColumns = targetDatums;
+        }
+        
+        SelectStatement translatedIndexSelect = IndexStatementRewriter.translate(select, dataPlan.getContext().getResolver());
+        List<QueryPlan> plans = Lists.newArrayListWithExpectedSize(1 + indexes.size());
+        plans.add(dataPlan);
+        QueryPlan hintedPlan = getHintedQueryPlan(statement, translatedIndexSelect, indexes, targetColumns, parallelIteratorFactory, plans);
+        if (hintedPlan != null) {
+            return hintedPlan;
+        }
+        for (PTable index : indexes) {
+            addPlan(statement, translatedIndexSelect, index, targetColumns, parallelIteratorFactory, plans);
+        }
+        
+        return chooseBestPlan(select, plans);
+    }
+    
+    private static QueryPlan getHintedQueryPlan(PhoenixStatement statement, SelectStatement select, List<PTable> indexes, List<? extends PDatum> targetColumns, ParallelIteratorFactory parallelIteratorFactory, List<QueryPlan> plans) throws SQLException {
+        QueryPlan dataPlan = plans.get(0);
+        String indexHint = select.getHint().getHint(Hint.INDEX);
+        if (indexHint == null) {
+            return null;
+        }
+        int startIndex = 0;
+        String alias = dataPlan.getTableRef().getTableAlias();
+        String prefix = HintNode.PREFIX + (alias == null ? dataPlan.getTableRef().getTable().getName().getString() : alias) + HintNode.SEPARATOR;
+        while (startIndex < indexHint.length()) {
+            startIndex = indexHint.indexOf(prefix, startIndex);
+            if (startIndex < 0) {
+                return null;
+            }
+            startIndex += prefix.length();
+            boolean done = false; // true when SUFFIX found
+            while (startIndex < indexHint.length() && !done) {
+                int endIndex;
+                int endIndex1 = indexHint.indexOf(HintNode.SEPARATOR, startIndex);
+                int endIndex2 = indexHint.indexOf(HintNode.SUFFIX, startIndex);
+                if (endIndex1 < 0 && endIndex2 < 0) { // Missing SUFFIX shouldn't happen
+                    endIndex = indexHint.length();
+                } else if (endIndex1 < 0) {
+                    done = true;
+                    endIndex = endIndex2;
+                } else if (endIndex2 < 0) {
+                    endIndex = endIndex1;
+                } else {
+                    endIndex = Math.min(endIndex1, endIndex2);
+                    done = endIndex2 == endIndex;
+                }
+                String indexName = indexHint.substring(startIndex, endIndex);
+                int indexPos = getIndexPosition(indexes, indexName);
+                if (indexPos >= 0) {
+                    // Hinted index is applicable, so return it. It'll be the plan at position 1, after the data plan
+                    if (addPlan(statement, select, indexes.get(indexPos), targetColumns, parallelIteratorFactory, plans)) {
+                        return plans.get(1);
+                    }
+                    indexes.remove(indexPos);
+                }
+                startIndex = endIndex + 1;
+            }
+        }
+        return null;
+    }
+    
+    private static int getIndexPosition(List<PTable> indexes, String indexName) {
+        for (int i = 0; i < indexes.size(); i++) {
+            if (indexName.equals(indexes.get(i).getTableName().getString())) {
+                return i;
+            }
+        }
+        return -1;
+    }
+    
+    private static boolean addPlan(PhoenixStatement statement, SelectStatement select, PTable index, List<? extends PDatum> targetColumns, ParallelIteratorFactory parallelIteratorFactory, List<QueryPlan> plans) throws SQLException {
+        QueryPlan dataPlan = plans.get(0);
+        int nColumns = dataPlan.getProjector().getColumnCount();
+        String alias = '"' + dataPlan.getTableRef().getTableAlias() + '"'; // double quote in case it's case sensitive
+        String schemaName = dataPlan.getTableRef().getTable().getSchemaName().getString();
+        schemaName = schemaName.length() == 0 ? null :  '"' + schemaName + '"';
+
+        String tableName = '"' + index.getTableName().getString() + '"';
+        List<? extends TableNode> tables = Collections.singletonList(FACTORY.namedTable(alias, FACTORY.table(schemaName, tableName)));
+        try {
+            SelectStatement indexSelect = FACTORY.select(select, tables);            
+            QueryCompiler compiler = new QueryCompiler(statement, targetColumns, parallelIteratorFactory);
+            QueryPlan plan = compiler.compile(indexSelect);
+            // Checking the index status and number of columns handles the wildcard cases correctly
+            // We can't check the status earlier, because the index table may be out-of-date.
+            if (plan.getTableRef().getTable().getIndexState() == PIndexState.ACTIVE && plan.getProjector().getColumnCount() == nColumns) {
+                plans.add(plan);
+                return true;
+            }
+        } catch (ColumnNotFoundException e) {
+            /* Means that a column is being used that's not in our index.
+             * Since we currently don't keep stats, we don't know the selectivity of the index.
+             * For now, we just don't use this index (as opposed to trying to join back from
+             * the index table to the data table.
+             */
+        }
+        return false;
+    }
+    
+    /**
+     * Choose the best plan among all the possible ones.
+     * Since we don't keep stats yet, we use the following simple algorithm:
+     * 1) If the query has an ORDER BY and a LIMIT, choose the plan that has all the ORDER BY expression
+     * in the same order as the row key columns.
+     * 2) If there are more than one plan that meets (1), choose the plan with:
+     *    a) the most row key columns that may be used to form the start/stop scan key.
+     *    b) the plan that preserves ordering for a group by.
+     *    c) the data table plan
+     * @param plans the list of candidate plans
+     * @return
+     */
+    private QueryPlan chooseBestPlan(SelectStatement select, List<QueryPlan> plans) {
+        QueryPlan firstPlan = plans.get(0);
+        if (plans.size() == 1) {
+            return firstPlan;
+        }
+        
+        List<QueryPlan> candidates = Lists.newArrayListWithExpectedSize(plans.size());
+        if (firstPlan.getLimit() == null) {
+            candidates.addAll(plans);
+        } else {
+            for (QueryPlan plan : plans) {
+                // If ORDER BY optimized out (or not present at all)
+                if (plan.getOrderBy().getOrderByExpressions().isEmpty()) {
+                    candidates.add(plan);
+                }
+            }
+            if (candidates.isEmpty()) {
+                candidates.addAll(plans);
+            }
+        }
+        final int comparisonOfDataVersusIndexTable = select.getHint().hasHint(Hint.USE_DATA_OVER_INDEX_TABLE) ? -1 : 1;
+        Collections.sort(candidates, new Comparator<QueryPlan>() {
+
+            @Override
+            public int compare(QueryPlan plan1, QueryPlan plan2) {
+                int c = plan2.getContext().getScanRanges().getRanges().size() - plan1.getContext().getScanRanges().getRanges().size();
+                if (c != 0) return c;
+                if (plan1.getGroupBy()!=null && plan2.getGroupBy()!=null) {
+                    if (plan1.getGroupBy().isOrderPreserving() != plan2.getGroupBy().isOrderPreserving()) {
+                        return plan1.getGroupBy().isOrderPreserving() ? -1 : 1;
+                    }
+                }
+                // Use smaller table (table with fewest kv columns)
+                PTable table1 = plan1.getTableRef().getTable();
+                PTable table2 = plan2.getTableRef().getTable();
+                c = (table1.getColumns().size() - table1.getPKColumns().size()) - (table2.getColumns().size() - table2.getPKColumns().size());
+                if (c != 0) return c;
+                
+                // All things being equal, just use the index table
+                // TODO: have hint that drives this
+                if (plan1.getTableRef().getTable().getType() == PTableType.INDEX) {
+                    return comparisonOfDataVersusIndexTable;
+                }
+                if (plan2.getTableRef().getTable().getType() == PTableType.INDEX) {
+                    return -comparisonOfDataVersusIndexTable;
+                }
+                
+                return 0;
+            }
+            
+        });
+        
+        return candidates.get(0);
+        
+    }
+
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/AddColumnStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/AddColumnStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/AddColumnStatement.java
new file mode 100644
index 0000000..1059fce
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/AddColumnStatement.java
@@ -0,0 +1,51 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.phoenix.schema.PTableType;
+
+public class AddColumnStatement extends AlterTableStatement {
+    private final List<ColumnDef> columnDefs;
+    private final boolean ifNotExists;
+    private final Map<String,Object> props;
+    
+    protected AddColumnStatement(NamedTableNode table, PTableType tableType, List<ColumnDef> columnDefs, boolean ifNotExists, Map<String, Object> props) {
+        super(table, tableType);
+        this.columnDefs = columnDefs;
+        this.props = props == null ? Collections.<String,Object>emptyMap() : props;
+        this.ifNotExists = ifNotExists;
+    }
+
+    public List<ColumnDef> getColumnDefs() {
+        return columnDefs;
+    }
+    
+    public boolean ifNotExists() {
+        return ifNotExists;
+    }
+
+    public Map<String,Object> getProps() {
+        return props;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/AddParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/AddParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/AddParseNode.java
new file mode 100644
index 0000000..f4ae491
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/AddParseNode.java
@@ -0,0 +1,49 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.List;
+
+
+
+/**
+ * 
+ * Node representing addition in a SQL expression
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class AddParseNode extends ArithmeticParseNode {
+
+    AddParseNode(List<ParseNode> children) {
+        super(children);
+    }
+
+    @Override
+    public <T> T accept(ParseNodeVisitor<T> visitor) throws SQLException {
+        List<T> l = Collections.emptyList();
+        if (visitor.visitEnter(this)) {
+            l = acceptChildren(visitor);
+        }
+        return visitor.visitLeave(this, l);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/AggregateFunctionParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/AggregateFunctionParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/AggregateFunctionParseNode.java
new file mode 100644
index 0000000..a715dd7
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/AggregateFunctionParseNode.java
@@ -0,0 +1,39 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.util.List;
+
+
+public class AggregateFunctionParseNode extends FunctionParseNode {
+
+    public AggregateFunctionParseNode(String name, List<ParseNode> children, BuiltInFunctionInfo info) {
+        super(name, children, info);
+    }
+    
+    /**
+     * Aggregate function are not stateless, even though all the args may be stateless,
+     * for example, COUNT(1)
+     */
+    @Override
+    public boolean isStateless() {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/AliasedNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/AliasedNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/AliasedNode.java
new file mode 100644
index 0000000..d2302fd
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/AliasedNode.java
@@ -0,0 +1,53 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import org.apache.phoenix.util.SchemaUtil;
+
+/**
+ *
+ * Node representing an aliased parse node in a SQL select clause
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class AliasedNode {
+    private final String alias;
+    private final ParseNode node;
+    private final boolean isCaseSensitve;
+
+    public AliasedNode(String alias, ParseNode node) {
+        this.isCaseSensitve = alias != null && SchemaUtil.isCaseSensitive(alias);
+        this.alias = alias == null ? null : SchemaUtil.normalizeIdentifier(alias);
+        this.node = node;
+    }
+
+    public String getAlias() {
+        return alias;
+    }
+
+    public ParseNode getNode() {
+        return node;
+    }
+
+    public boolean isCaseSensitve() {
+        return isCaseSensitve;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/AlterIndexStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/AlterIndexStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/AlterIndexStatement.java
new file mode 100644
index 0000000..bee7498
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/AlterIndexStatement.java
@@ -0,0 +1,53 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import org.apache.phoenix.schema.PIndexState;
+
+public class AlterIndexStatement extends SingleTableSQLStatement {
+    private final String dataTableName;
+    private final boolean ifExists;
+    private final PIndexState indexState;
+
+    public AlterIndexStatement(NamedTableNode indexTableNode, String dataTableName, boolean ifExists, PIndexState indexState) {
+        super(indexTableNode,0);
+        this.dataTableName = dataTableName;
+        this.ifExists = ifExists;
+        this.indexState = indexState;
+    }
+
+    public String getTableName() {
+        return dataTableName;
+    }
+
+    @Override
+    public int getBindCount() {
+        return 0;
+    }
+
+    public boolean ifExists() {
+        return ifExists;
+    }
+
+    public PIndexState getIndexState() {
+        return indexState;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/AlterTableStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/AlterTableStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/AlterTableStatement.java
new file mode 100644
index 0000000..e6d4c80
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/AlterTableStatement.java
@@ -0,0 +1,35 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import org.apache.phoenix.schema.PTableType;
+
+public abstract class AlterTableStatement extends SingleTableSQLStatement {
+    private final PTableType tableType;
+
+    AlterTableStatement(NamedTableNode table, PTableType tableType) {
+        super(table, 0);
+        this.tableType = tableType;
+    }
+
+    public PTableType getTableType() {
+        return tableType;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/AndParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/AndParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/AndParseNode.java
new file mode 100644
index 0000000..452d893
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/AndParseNode.java
@@ -0,0 +1,51 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.List;
+
+
+
+
+/**
+ * 
+ * Node representing AND in a SQL expression
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class AndParseNode extends CompoundParseNode {
+
+    AndParseNode(List<ParseNode> children) {
+        super(children);
+    }
+
+    @Override
+    public <T> T accept(ParseNodeVisitor<T> visitor) throws SQLException {
+        List<T> l = Collections.emptyList();
+        if (visitor.visitEnter(this)) {
+            l = acceptChildren(visitor);
+        }
+        return visitor.visitLeave(this, l);
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/ArithmeticParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ArithmeticParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ArithmeticParseNode.java
new file mode 100644
index 0000000..4e90960
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ArithmeticParseNode.java
@@ -0,0 +1,30 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.util.List;
+
+public abstract class ArithmeticParseNode extends CompoundParseNode {
+
+    public ArithmeticParseNode(List<ParseNode> children) {
+        super(children);
+    }
+
+}


[33/51] [partial] Initial commit of master branch from github

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/AndExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/AndExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/AndExpression.java
new file mode 100644
index 0000000..c35eca5
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/AndExpression.java
@@ -0,0 +1,79 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.util.List;
+
+import org.apache.phoenix.expression.visitor.ExpressionVisitor;
+
+
+/**
+ * 
+ * AND expression implementation
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class AndExpression extends AndOrExpression {
+    private static final String AND = "AND";
+    
+    public static String combine(String expression1, String expression2) {
+        if (expression1 == null) {
+            return expression2;
+        }
+        if (expression2 == null) {
+            return expression1;
+        }
+        return "(" + expression1 + ") " + AND + " (" + expression2 + ")";
+    }
+    
+    public AndExpression() {
+    }
+
+    public AndExpression(List<Expression> children) {
+        super(children);
+    }
+
+    @Override
+    protected boolean getStopValue() {
+        return Boolean.FALSE;
+    }
+
+    @Override
+    public String toString() {
+        StringBuilder buf = new StringBuilder("(");
+        for (int i = 0; i < children.size() - 1; i++) {
+            buf.append(children.get(i) + " " + AND + " ");
+        }
+        buf.append(children.get(children.size()-1));
+        buf.append(')');
+        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;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/AndOrExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/AndOrExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/AndOrExpression.java
new file mode 100644
index 0000000..aebd63a
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/AndOrExpression.java
@@ -0,0 +1,99 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.util.BitSet;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+
+/**
+ * 
+ * Abstract expression implementation for compound AND and OR expressions
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public abstract class AndOrExpression extends BaseCompoundExpression {
+    // Remember evaluation of child expression for partial evaluation
+    private BitSet partialEvalState;
+   
+    public AndOrExpression() {
+    }
+    
+    public AndOrExpression(List<Expression> children) {
+        super(children);
+    }
+    
+    @Override
+    public int hashCode() {
+        return 31 * super.hashCode() + Boolean.valueOf(this.getStopValue()).hashCode();
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return PDataType.BOOLEAN;
+    }
+
+    @Override
+    public void reset() {
+        if (partialEvalState == null) {
+            partialEvalState = new BitSet(children.size());
+        } else {
+            partialEvalState.clear();
+        }
+        super.reset();
+    }
+    
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        boolean isNull = false;
+        boolean stopValue = getStopValue();
+        for (int i = 0; i < children.size(); i++) {
+            Expression child = children.get(i);
+            // If partial state is available, then use that to know we've already evaluated this
+            // child expression and do not need to do so again.
+            if (partialEvalState == null || !partialEvalState.get(i)) {
+                // Call through to child evaluate method matching parent call to allow child to optimize
+                // evaluate versus getValue code path.
+                if (child.evaluate(tuple, ptr)) {
+                    // Short circuit if we see our stop value
+                    if (Boolean.valueOf(stopValue).equals(PDataType.BOOLEAN.toObject(ptr, child.getDataType()))) {
+                        return true;
+                    } else if (partialEvalState != null) {
+                        partialEvalState.set(i);
+                    }
+                } else {
+                    isNull = true;
+                }
+            }
+        }
+        if (isNull) {
+            return false;
+        }
+        return true;
+    }
+
+    protected abstract boolean getStopValue();
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/ArithmeticExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/ArithmeticExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/ArithmeticExpression.java
new file mode 100644
index 0000000..622d709
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/ArithmeticExpression.java
@@ -0,0 +1,45 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.util.List;
+
+public abstract class ArithmeticExpression extends BaseCompoundExpression {
+
+    public ArithmeticExpression() {
+    }
+
+    public ArithmeticExpression(List<Expression> children) {
+        super(children);
+    }
+
+    @Override
+    public String toString() {
+        StringBuilder buf = new StringBuilder("(");
+        for (int i = 0; i < children.size() - 1; i++) {
+            buf.append(children.get(i) + getOperatorString());
+        }
+        buf.append(children.get(children.size()-1));
+        buf.append(')');
+        return buf.toString();
+    }
+    
+    abstract protected String getOperatorString();
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..6dbf865
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/ArrayConstructorExpression.java
@@ -0,0 +1,101 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.sql.Types;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.io.WritableUtils;
+
+import org.apache.phoenix.schema.PArrayDataType;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.PhoenixArray;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+/**
+ * Creates an expression for Upsert with Values/Select using ARRAY
+ */
+public class ArrayConstructorExpression extends BaseCompoundExpression {
+    private PDataType baseType;
+    private int position = -1;
+    private Object[] elements;
+    
+    
+    public ArrayConstructorExpression(List<Expression> children, PDataType baseType) {
+        super(children);
+        init(baseType);
+    }
+
+    private void init(PDataType baseType) {
+        this.baseType = baseType;
+        elements = new Object[getChildren().size()];
+    }
+    
+    @Override
+    public PDataType getDataType() {
+        return PDataType.fromTypeId(baseType.getSqlType() + Types.ARRAY);
+    }
+
+    @Override
+    public void reset() {
+        super.reset();
+        position = 0;
+        Arrays.fill(elements, null);
+    }
+    
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        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 {
+                elements[i] = baseType.toObject(ptr, child.getDataType(), child.getColumnModifier());
+            }
+        }
+        if (position >= 0) position = elements.length;
+        PhoenixArray array = PArrayDataType.instantiatePhoenixArray(baseType, elements);
+        // FIXME: Need to see if this creation of an array and again back to byte[] can be avoided
+        ptr.set(getDataType().toBytes(array));
+        return true;
+    }
+    
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        super.readFields(input);
+        int baseTypeOrdinal = WritableUtils.readVInt(input);
+        init(PDataType.values()[baseTypeOrdinal]);
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+        super.write(output);
+        WritableUtils.writeVInt(output, baseType.ordinal());
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/BaseAddSubtractExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/BaseAddSubtractExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/BaseAddSubtractExpression.java
new file mode 100644
index 0000000..1b9e4e5
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/BaseAddSubtractExpression.java
@@ -0,0 +1,53 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.util.List;
+
+import org.apache.phoenix.schema.PDataType;
+
+
+abstract public class BaseAddSubtractExpression extends ArithmeticExpression {
+    public BaseAddSubtractExpression() {
+    }
+
+    public BaseAddSubtractExpression(List<Expression> children) {
+        super(children);
+    }
+
+    protected static Integer getPrecision(Integer lp, Integer rp, Integer ls, Integer rs) {
+    	if (ls == null || rs == null) {
+    		return PDataType.MAX_PRECISION;
+    	}
+        int val = getScale(lp, rp, ls, rs) + Math.max(lp - ls, rp - rs) + 1;
+        return Math.min(PDataType.MAX_PRECISION, val);
+    }
+
+    protected static Integer getScale(Integer lp, Integer rp, Integer ls, Integer rs) {
+    	// If we are adding a decimal with scale and precision to a decimal
+    	// with no precision nor scale, the scale system does not apply.
+    	if (ls == null || rs == null) {
+    		return null;
+    	}
+        int val = Math.max(ls, rs);
+        return Math.min(PDataType.MAX_PRECISION, val);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/BaseCompoundExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/BaseCompoundExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/BaseCompoundExpression.java
new file mode 100644
index 0000000..82a130d
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/BaseCompoundExpression.java
@@ -0,0 +1,145 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.io.WritableUtils;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.phoenix.expression.visitor.ExpressionVisitor;
+
+
+public abstract class BaseCompoundExpression extends BaseExpression {
+    protected List<Expression> children;
+    private boolean isNullable;
+    private boolean isStateless;
+    private boolean isDeterministic;
+   
+    public BaseCompoundExpression() {
+    }
+    
+    public BaseCompoundExpression(List<Expression> children) {
+        init(children);
+    }
+    
+    private void init(List<Expression> children) {
+        this.children = ImmutableList.copyOf(children);
+        boolean isStateless = true;
+        boolean isDeterministic = true;
+        boolean isNullable = false;
+        for (int i = 0; i < children.size(); i++) {
+            Expression child = children.get(i);
+            isNullable |= child.isNullable();
+            isStateless &= child.isStateless();
+            isDeterministic &= child.isDeterministic();
+        }
+        this.isStateless = isStateless;
+        this.isDeterministic = isDeterministic;
+        this.isNullable = isNullable;
+    }
+    
+    @Override
+    public List<Expression> getChildren() {
+        return children;
+    }
+    
+    
+    @Override
+    public boolean isDeterministic() {
+        return isDeterministic;
+    }
+    
+    @Override
+    public boolean isStateless() {
+        return isStateless;
+    }
+
+    @Override
+    public boolean isNullable() {
+        return isNullable;
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + children.hashCode();
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) return true;
+        if (obj == null) return false;
+        if (getClass() != obj.getClass()) return false;
+        BaseCompoundExpression other = (BaseCompoundExpression)obj;
+        if (!children.equals(other.children)) return false;
+        return true;
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        int len = WritableUtils.readVInt(input);
+        List<Expression>children = new ArrayList<Expression>(len);
+        for (int i = 0; i < len; i++) {
+            Expression child = ExpressionType.values()[WritableUtils.readVInt(input)].newInstance();
+            child.readFields(input);
+            children.add(child);
+        }
+        init(children);
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+        WritableUtils.writeVInt(output, children.size());
+        for (int i = 0; i < children.size(); i++) {
+            Expression child = children.get(i);
+            WritableUtils.writeVInt(output, ExpressionType.valueOf(child).ordinal());
+            child.write(output);
+        }
+    }
+
+    @Override
+    public void reset() {
+        for (int i = 0; i < children.size(); i++) {
+            children.get(i).reset();
+        }
+    }
+    
+    @Override
+    public <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;
+    }
+    
+    @Override
+    public String toString() {
+        return this.getClass().getName() + " [children=" + children + "]";
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/BaseDecimalAddSubtractExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/BaseDecimalAddSubtractExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/BaseDecimalAddSubtractExpression.java
new file mode 100644
index 0000000..4f641fe
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/BaseDecimalAddSubtractExpression.java
@@ -0,0 +1,5 @@
+package org.apache.phoenix.expression;
+
+public class BaseDecimalAddSubtractExpression {
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/BaseExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/BaseExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/BaseExpression.java
new file mode 100644
index 0000000..9bd5581
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/BaseExpression.java
@@ -0,0 +1,108 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.phoenix.expression.visitor.ExpressionVisitor;
+import org.apache.phoenix.schema.ColumnModifier;
+
+
+
+/**
+ * 
+ * Base class for Expression hierarchy that provides common
+ * default implementations for most methods
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public abstract class BaseExpression implements Expression {
+    @Override
+    public boolean isNullable() {
+        return false;
+    }
+
+    @Override
+    public Integer getByteSize() {
+        return getDataType().isFixedWidth() ? getDataType().getByteSize() : null;
+    }
+
+    @Override
+    public Integer getMaxLength() {
+        return null;
+    }
+
+    @Override
+    public Integer getScale() {
+        return null;
+    }
+    
+    @Override
+    public ColumnModifier getColumnModifier() {
+    	    return null;
+    }    
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+    }
+
+    @Override
+    public void reset() {
+    }
+    
+    protected final <T> List<T> acceptChildren(ExpressionVisitor<T> visitor, Iterator<Expression> iterator) {
+        if (iterator == null) {
+            iterator = visitor.defaultIterator(this);
+        }
+        List<T> l = Collections.emptyList();
+        while (iterator.hasNext()) {
+            Expression child = iterator.next();
+            T t = child.accept(visitor);
+            if (t != null) {
+                if (l.isEmpty()) {
+                    l = new ArrayList<T>(getChildren().size());
+                }
+                l.add(t);
+            }
+        }
+        return l;
+    }
+    
+    @Override
+    public boolean isDeterministic() {
+        return true;
+    }
+    
+    @Override
+    public boolean isStateless() {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/BaseSingleExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/BaseSingleExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/BaseSingleExpression.java
new file mode 100644
index 0000000..1758438
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/BaseSingleExpression.java
@@ -0,0 +1,91 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.io.WritableUtils;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.phoenix.expression.visitor.ExpressionVisitor;
+
+
+/**
+ * 
+ * Base class for expressions which have a single child expression
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public abstract class BaseSingleExpression extends BaseExpression {
+
+    protected List<Expression> children;
+    
+    public BaseSingleExpression() {
+    }
+
+    public BaseSingleExpression(Expression expression) {
+        this.children = ImmutableList.of(expression);
+    }
+
+    @Override
+    public List<Expression> getChildren() {
+        return children;
+    }
+    
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        Expression expression = ExpressionType.values()[WritableUtils.readVInt(input)].newInstance();
+        expression.readFields(input);
+        children = ImmutableList.of(expression);
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+        WritableUtils.writeVInt(output, ExpressionType.valueOf(children.get(0)).ordinal());
+        children.get(0).write(output);
+    }
+
+    @Override
+    public boolean isNullable() {
+        return children.get(0).isNullable();
+    }
+
+    @Override
+    public void reset() {
+        children.get(0).reset();
+    }
+
+    @Override
+    public <T> T accept(ExpressionVisitor<T> visitor) {
+        List<T> l = acceptChildren(visitor, null);
+        if (l.isEmpty()) {
+            return visitor.defaultReturn(this, l);
+        }
+        return l.get(0);
+    }
+    
+    public Expression getChild() {
+        return children.get(0);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/BaseTerminalExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/BaseTerminalExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/BaseTerminalExpression.java
new file mode 100644
index 0000000..aaa4371
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/BaseTerminalExpression.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.util.Collections;
+import java.util.List;
+
+import org.apache.phoenix.expression.visitor.ExpressionVisitor;
+
+
+
+/**
+ * 
+ * Grouping class for expression that have no expression children
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public abstract class BaseTerminalExpression extends BaseExpression {
+    @Override
+    public List<Expression> getChildren() {
+        return Collections.emptyList();
+    }
+    
+    @Override
+    public <T> T accept(ExpressionVisitor<T> visitor) {
+        return null;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/CaseExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/CaseExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/CaseExpression.java
new file mode 100644
index 0000000..f9a524a
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/CaseExpression.java
@@ -0,0 +1,234 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.sql.SQLException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.io.WritableUtils;
+
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+import org.apache.phoenix.expression.visitor.ExpressionVisitor;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+
+/**
+ * 
+ * CASE/WHEN expression implementation
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class CaseExpression extends BaseCompoundExpression {
+    private static final int FULLY_EVALUATE = -1;
+    
+    private short evalIndex = FULLY_EVALUATE;
+    private boolean foundIndex;
+    private PDataType returnType;
+   
+    public CaseExpression() {
+    }
+    
+    private static List<Expression> coerceIfNecessary(List<Expression> children) throws SQLException {
+        boolean isChildTypeUnknown = false;
+        PDataType returnType = children.get(0).getDataType();
+        for (int i = 2; i < children.size(); i+=2) {
+            Expression child = children.get(i);
+            PDataType childType = child.getDataType();
+            if (childType == null) {
+                isChildTypeUnknown = true;
+            } else if (returnType == null) {
+                returnType = childType;
+                isChildTypeUnknown = true;
+            } else if (returnType == childType || childType.isCoercibleTo(returnType)) {
+                continue;
+            } else if (returnType.isCoercibleTo(childType)) {
+                returnType = childType;
+            } else {
+                throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_CONVERT_TYPE)
+                    .setMessage("Case expressions must have common type: " + returnType + " cannot be coerced to " + childType)
+                    .build().buildException();
+            }
+        }
+        // If we found an "unknown" child type and the return type is a number
+        // make the return type be the most general number type of DECIMAL.
+        if (isChildTypeUnknown && returnType != null && returnType.isCoercibleTo(PDataType.DECIMAL)) {
+            returnType = PDataType.DECIMAL;
+        }
+        List<Expression> newChildren = children;
+        for (int i = 0; i < children.size(); i+=2) {
+            Expression child = children.get(i);
+            PDataType childType = child.getDataType();
+            if (childType != returnType) {
+                if (newChildren == children) {
+                    newChildren = new ArrayList<Expression>(children);
+                }
+                newChildren.set(i, CoerceExpression.create(child, returnType));
+            }
+        }
+        return newChildren;
+    }
+    /**
+     * Construct CASE/WHEN expression
+     * @param expressions list of expressions in the form of:
+     *  ((<result expression>, <boolean expression>)+, [<optional else result expression>])
+     * @throws SQLException if return type of case expressions do not match and cannot
+     *  be coerced to a common type
+     */
+    public CaseExpression(List<Expression> expressions) throws SQLException {
+        super(coerceIfNecessary(expressions));
+        returnType = children.get(0).getDataType();
+    }
+    
+    private boolean isPartiallyEvaluating() {
+        return evalIndex != FULLY_EVALUATE;
+    }
+    
+    public boolean hasElse() {
+        return children.size() % 2 != 0;
+    }
+    
+    @Override
+    public boolean isNullable() {
+        // If any expression is nullable or there's no else clause
+        // return true since null may be returned.
+        if (super.isNullable() || !hasElse()) {
+            return true;
+        }
+        return children.get(children.size()-1).isNullable();
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return returnType;
+    }
+
+    @Override
+    public void reset() {
+        foundIndex = false;
+        evalIndex = 0;
+    }
+    
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        super.readFields(input);
+        this.returnType = PDataType.values()[WritableUtils.readVInt(input)];
+    }
+    
+    @Override
+    public void write(DataOutput output) throws IOException {
+        super.write(output);
+        WritableUtils.writeVInt(output, this.returnType.ordinal());
+    }
+    
+    public int evaluateIndexOf(Tuple tuple, ImmutableBytesWritable ptr) {
+        if (foundIndex) {
+            return evalIndex;
+        }
+        int size = children.size();
+        // If we're doing partial evaluation, start where we left off
+        for (int i = isPartiallyEvaluating() ? evalIndex : 0; i < size; i+=2) {
+            // Short circuit if we see our stop value
+            if (i+1 == size) {
+                return i;
+            }
+            // If we get null, we have to re-evaluate from that point (special case this in filter, like is null)
+            // We may only run this when we're done/have all values
+            boolean evaluated = children.get(i+1).evaluate(tuple, ptr);
+            if (evaluated && Boolean.TRUE.equals(PDataType.BOOLEAN.toObject(ptr))) {
+                if (isPartiallyEvaluating()) {
+                    foundIndex = true;
+                }
+                return i;
+            }
+            if (isPartiallyEvaluating()) {
+                if (evaluated || tuple.isImmutable()) {
+                    evalIndex+=2;
+                } else {
+                    /*
+                     * Return early here if incrementally evaluating and we don't
+                     * have all the key values yet. We can't continue because we'd
+                     * potentially be bypassing cases which we could later evaluate
+                     * once we have more column values.
+                     */
+                    return -1;
+                }
+            }
+        }
+        // No conditions matched, return size to indicate that we were able
+        // to evaluate all cases, but didn't find any matches.
+        return size;
+    }
+    
+    /**
+     * Only expression that currently uses the isPartial flag. The IS NULL
+     * expression will use it too. TODO: We could alternatively have a non interface
+     * method, like setIsPartial in which we set to false prior to calling
+     * evaluate.
+     */
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        int index = evaluateIndexOf(tuple, ptr);
+        if (index < 0) {
+            return false;
+        } else if (index == children.size()) {
+            ptr.set(PDataType.NULL_BYTES);
+            return true;
+        }
+        if (children.get(index).evaluate(tuple, ptr)) {
+            return true;
+        }
+        return false;
+    }
+    
+    @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;
+    }
+    
+    @Override
+    public String toString() {
+        StringBuilder buf = new StringBuilder("CASE ");
+        for (int i = 0; i < children.size() - 1; i+=2) {
+            buf.append("WHEN ");
+            buf.append(children.get(i+1));
+            buf.append(" THEN ");
+            buf.append(children.get(i));
+        }
+        if (hasElse()) {
+            buf.append(" ELSE " + children.get(children.size()-1));
+        }
+        buf.append(" END");
+        return buf.toString();
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/CoerceExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/CoerceExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/CoerceExpression.java
new file mode 100644
index 0000000..827d70e
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/CoerceExpression.java
@@ -0,0 +1,153 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.sql.SQLException;
+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.ColumnModifier;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+
+public class CoerceExpression extends BaseSingleExpression {
+    private PDataType toType;
+    private ColumnModifier toMod;
+    private Integer byteSize;
+    
+    public CoerceExpression() {
+    }
+
+    public static Expression create(Expression expression, PDataType toType) throws SQLException {
+        if (toType == expression.getDataType()) {
+            return expression;
+        }
+        return new CoerceExpression(expression, toType);
+    }
+    
+    //Package protected for tests
+    CoerceExpression(Expression expression, PDataType toType) {
+        this(expression, toType, null, null);
+    }
+    
+    CoerceExpression(Expression expression, PDataType toType, ColumnModifier toMod, Integer byteSize) {
+        super(expression);
+        this.toType = toType;
+        this.toMod = toMod;
+        this.byteSize = byteSize;
+    }
+
+    @Override
+    public Integer getByteSize() {
+        return byteSize;
+    }
+    
+    @Override
+    public Integer getMaxLength() {
+        return byteSize;
+    }
+    
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + ((byteSize == null) ? 0 : byteSize.hashCode());
+        result = prime * result + ((toMod == null) ? 0 : toMod.hashCode());
+        result = prime * result + ((toType == null) ? 0 : toType.hashCode());
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) return true;
+        if (obj == null) return false;
+        if (getClass() != obj.getClass()) return false;
+        CoerceExpression other = (CoerceExpression)obj;
+        if (byteSize == null) {
+            if (other.byteSize != null) return false;
+        } else if (!byteSize.equals(other.byteSize)) return false;
+        if (toMod != other.toMod) return false;
+        if (toType != other.toType) return false;
+        return true;
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        super.readFields(input);
+        toType = PDataType.values()[WritableUtils.readVInt(input)];
+        toMod = ColumnModifier.fromSystemValue(WritableUtils.readVInt(input));
+        int byteSize = WritableUtils.readVInt(input);
+        this.byteSize = byteSize == -1 ? null : byteSize;
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+        super.write(output);
+        WritableUtils.writeVInt(output, toType.ordinal());
+        WritableUtils.writeVInt(output, ColumnModifier.toSystemValue(toMod));
+        WritableUtils.writeVInt(output, byteSize == null ? -1 : byteSize);
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        if (getChild().evaluate(tuple, ptr)) {
+            getDataType().coerceBytes(ptr, getChild().getDataType(), getChild().getColumnModifier(), getColumnModifier());
+            return true;
+        }
+        return false;
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return toType;
+    }
+    
+    @Override
+    public ColumnModifier getColumnModifier() {
+            return toMod;
+    }    
+
+    @Override
+    public <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;
+    }
+    
+    @Override
+    public String toString() {
+        StringBuilder buf = new StringBuilder("TO_" + toType.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();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/ColumnExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/ColumnExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/ColumnExpression.java
new file mode 100644
index 0000000..bfb0d70
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/ColumnExpression.java
@@ -0,0 +1,154 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.expression;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.WritableUtils;
+
+import com.google.common.base.Objects;
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.PDatum;
+
+/**
+ * 
+ * Common base class for column value accessors
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+abstract public class ColumnExpression extends BaseTerminalExpression {
+    protected PDataType type;
+    private Integer byteSize;
+    private boolean isNullable;
+    private Integer maxLength;
+    private Integer scale;
+    private ColumnModifier columnModifier;
+
+    public ColumnExpression() {
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + (isNullable() ? 1231 : 1237);
+        Integer maxLength = this.getByteSize();
+        result = prime * result + ((maxLength == null) ? 0 : maxLength.hashCode());
+        PDataType type = this.getDataType();
+        result = prime * result + ((type == null) ? 0 : type.hashCode());
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) return true;
+        if (obj == null) return false;
+        if (getClass() != obj.getClass()) return false;
+        ColumnExpression other = (ColumnExpression)obj;
+        if (this.isNullable() != other.isNullable()) return false;
+        if (!Objects.equal(this.getByteSize(),other.getByteSize())) return false;
+        if (this.getDataType() != other.getDataType()) return false;
+        return true;
+    }
+
+    public ColumnExpression(PDatum datum) {
+        this.type = datum.getDataType();
+        this.isNullable = datum.isNullable();
+        if (type.isFixedWidth() && type.getByteSize() == null) {
+            this.byteSize = datum.getByteSize();
+        }
+        this.maxLength = datum.getMaxLength();
+        this.scale = datum.getScale();
+        this.columnModifier = datum.getColumnModifier();
+    }
+
+    @Override
+    public boolean isNullable() {
+       return isNullable;
+    }
+    
+    @Override
+    public PDataType getDataType() {
+        return type;
+    }
+    
+    @Override
+    public ColumnModifier getColumnModifier() {
+    	return columnModifier;
+    }
+
+    @Override
+    public Integer getByteSize() {
+        if (byteSize != null) {
+            return byteSize;
+        }
+        return super.getByteSize();
+    }
+
+    @Override
+    public Integer getMaxLength() {
+        return maxLength;
+    }
+
+    @Override
+    public Integer getScale() {
+        return scale;
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        // read/write type ordinal, maxLength presence, scale presence and isNullable bit together to save space
+        int typeAndFlag = WritableUtils.readVInt(input);
+        isNullable = (typeAndFlag & 0x01) != 0;
+        if ((typeAndFlag & 0x02) != 0) {
+            scale = WritableUtils.readVInt(input);
+        }
+        if ((typeAndFlag & 0x04) != 0) {
+            maxLength = WritableUtils.readVInt(input);
+        }
+        type = PDataType.values()[typeAndFlag >>> 3];
+        if (type.isFixedWidth() && type.getByteSize() == null) {
+            byteSize = WritableUtils.readVInt(input);
+        }
+        columnModifier = ColumnModifier.fromSystemValue(WritableUtils.readVInt(input));
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+        // read/write type ordinal, maxLength presence, scale presence and isNullable bit together to save space
+        int typeAndFlag = (isNullable ? 1 : 0) | ((scale != null ? 1 : 0) << 1) | ((maxLength != null ? 1 : 0) << 2)
+                | (type.ordinal() << 3);
+        WritableUtils.writeVInt(output,typeAndFlag);
+        if (scale != null) {
+            WritableUtils.writeVInt(output, scale);
+        }
+        if (maxLength != null) {
+            WritableUtils.writeVInt(output, maxLength);
+        }
+        if (type.isFixedWidth() && type.getByteSize() == null) {
+            WritableUtils.writeVInt(output, byteSize);
+        }
+        WritableUtils.writeVInt(output, ColumnModifier.toSystemValue(columnModifier));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/ComparisonExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/ComparisonExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/ComparisonExpression.java
new file mode 100644
index 0000000..5d3af1b
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/ComparisonExpression.java
@@ -0,0 +1,159 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.filter.CompareFilter.CompareOp;
+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.ColumnModifier;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.StringUtil;
+
+
+/**
+ * 
+ * Implementation for <,<=,>,>=,=,!= comparison expressions
+ * @author jtaylor
+ * @since 0.1
+ */
+public class ComparisonExpression extends BaseCompoundExpression {
+    private CompareOp op;
+    private static final String[] CompareOpString = new String[CompareOp.values().length];
+    static {
+        CompareOpString[CompareOp.EQUAL.ordinal()] = " = ";
+        CompareOpString[CompareOp.NOT_EQUAL.ordinal()] = " != ";
+        CompareOpString[CompareOp.GREATER.ordinal()] = " > ";
+        CompareOpString[CompareOp.LESS.ordinal()] = " < ";
+        CompareOpString[CompareOp.GREATER_OR_EQUAL.ordinal()] = " >= ";
+        CompareOpString[CompareOp.LESS_OR_EQUAL.ordinal()] = " <= ";
+    }
+    
+    public ComparisonExpression() {
+    }
+
+    public ComparisonExpression(CompareOp op, List<Expression> children) {
+        super(children);
+        if (op == null) {
+            throw new NullPointerException();
+        }
+        this.op = op;
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = super.hashCode();
+        result = prime * result + op.hashCode();
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) return true;
+        if (!super.equals(obj)) return false;
+        if (getClass() != obj.getClass()) return false;
+        ComparisonExpression other = (ComparisonExpression)obj;
+        if (op != other.op) return false;
+        return true;
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return PDataType.BOOLEAN;
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        if (!children.get(0).evaluate(tuple, ptr)) {
+            return false;
+        }
+        byte[] lhsBytes = ptr.get();
+        int lhsOffset = ptr.getOffset();
+        int lhsLength = ptr.getLength();
+        PDataType lhsDataType = children.get(0).getDataType();
+        ColumnModifier lhsColumnModifier = children.get(0).getColumnModifier();
+        
+        if (!children.get(1).evaluate(tuple, ptr)) {
+            return false;
+        }
+        
+        byte[] rhsBytes = ptr.get();
+        int rhsOffset = ptr.getOffset();
+        int rhsLength = ptr.getLength();
+        PDataType rhsDataType = children.get(1).getDataType();
+        ColumnModifier rhsColumnModifier = children.get(1).getColumnModifier();   
+        if (rhsDataType == PDataType.CHAR) {
+            rhsLength = StringUtil.getUnpaddedCharLength(rhsBytes, rhsOffset, rhsLength, rhsColumnModifier);
+        }
+        if (lhsDataType == PDataType.CHAR) {
+            lhsLength = StringUtil.getUnpaddedCharLength(lhsBytes, lhsOffset, lhsLength, lhsColumnModifier);
+        }
+        
+        
+        int comparisonResult = lhsDataType.compareTo(lhsBytes, lhsOffset, lhsLength, lhsColumnModifier, 
+                rhsBytes, rhsOffset, rhsLength, rhsColumnModifier, rhsDataType);
+        ptr.set(ByteUtil.compare(op, comparisonResult) ? PDataType.TRUE_BYTES : PDataType.FALSE_BYTES);
+        return true;
+    }
+    
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        op = CompareOp.values()[WritableUtils.readVInt(input)];
+        super.readFields(input);
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+        WritableUtils.writeVInt(output, op.ordinal());
+        super.write(output);
+    }
+
+    @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 CompareOp getFilterOp() {
+        return op;
+    }
+    
+    public static String toString(CompareOp op, List<Expression> children) {
+        return (children.get(0) + CompareOpString[op.ordinal()] + children.get(1));
+    }
+    
+    @Override
+    public String toString() {
+        return toString(getFilterOp(), children);
+    }    
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/CurrentDateTimeFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/CurrentDateTimeFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/CurrentDateTimeFunction.java
new file mode 100644
index 0000000..b221c1c
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/CurrentDateTimeFunction.java
@@ -0,0 +1,44 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.util.List;
+
+import org.apache.phoenix.expression.function.ScalarFunction;
+
+public abstract class CurrentDateTimeFunction extends ScalarFunction {
+
+    public CurrentDateTimeFunction() {
+    }
+
+    public CurrentDateTimeFunction(List<Expression> children) {
+        super(children);
+    }
+    
+    @Override
+    public boolean isStateless() {
+        return true;
+    }
+
+    @Override
+    public boolean isDeterministic() {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/DateAddExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/DateAddExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/DateAddExpression.java
new file mode 100644
index 0000000..a36bb48
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/DateAddExpression.java
@@ -0,0 +1,80 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.math.BigDecimal;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+
+public class DateAddExpression extends AddExpression {
+    static private final BigDecimal BD_MILLIS_IN_DAY = BigDecimal.valueOf(QueryConstants.MILLIS_IN_DAY);
+    
+    public DateAddExpression() {
+    }
+
+    public DateAddExpression(List<Expression> children) {
+        super(children);
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        long finalResult=0;
+        
+        for(int i=0;i<children.size();i++) {
+            if (!children.get(i).evaluate(tuple, ptr)) {
+                return false;
+            }
+            if (ptr.getLength() == 0) {
+                return true;
+            }
+            long value;
+            PDataType type = children.get(i).getDataType();
+            ColumnModifier columnModifier = children.get(i).getColumnModifier();
+            if (type == PDataType.DECIMAL) {
+                BigDecimal bd = (BigDecimal)PDataType.DECIMAL.toObject(ptr, columnModifier);
+                value = bd.multiply(BD_MILLIS_IN_DAY).longValue();
+            } else if (type.isCoercibleTo(PDataType.LONG)) {
+                value = type.getCodec().decodeLong(ptr, columnModifier) * QueryConstants.MILLIS_IN_DAY;
+            } else if (type.isCoercibleTo(PDataType.DOUBLE)) {
+                value = (long)(type.getCodec().decodeDouble(ptr, columnModifier) * QueryConstants.MILLIS_IN_DAY);
+            } else {
+                value = type.getCodec().decodeLong(ptr, columnModifier);
+            }
+            finalResult += value;
+        }
+        byte[] resultPtr = new byte[getDataType().getByteSize()];
+        ptr.set(resultPtr);
+        getDataType().getCodec().encodeLong(finalResult, ptr);
+        return true;
+    }
+
+    @Override
+    public final PDataType getDataType() {
+        return PDataType.DATE;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/DateSubtractExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/DateSubtractExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/DateSubtractExpression.java
new file mode 100644
index 0000000..7329aff
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/DateSubtractExpression.java
@@ -0,0 +1,80 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.math.BigDecimal;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+
+public class DateSubtractExpression extends SubtractExpression {
+    
+    public DateSubtractExpression() {
+    }
+
+    public DateSubtractExpression(List<Expression> children) {
+        super(children);
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        long finalResult=0;
+        
+        for(int i=0;i<children.size();i++) {
+            if (!children.get(i).evaluate(tuple, ptr) || ptr.getLength() == 0) {
+                return false;
+            }
+            long value;
+            PDataType type = children.get(i).getDataType();
+            ColumnModifier columnModifier = children.get(i).getColumnModifier();
+            if (type == PDataType.DECIMAL) {
+                BigDecimal bd = (BigDecimal)PDataType.DECIMAL.toObject(ptr, columnModifier);
+                value = bd.multiply(BD_MILLIS_IN_DAY).longValue();
+            } else if (type.isCoercibleTo(PDataType.LONG)) {
+                value = type.getCodec().decodeLong(ptr, columnModifier) * QueryConstants.MILLIS_IN_DAY;
+            } else if (type.isCoercibleTo(PDataType.DOUBLE)) {
+                value = (long)(type.getCodec().decodeDouble(ptr, columnModifier) * QueryConstants.MILLIS_IN_DAY);
+            } else {
+                value = type.getCodec().decodeLong(ptr, columnModifier);
+            }
+            if (i == 0) {
+                finalResult = value;
+            } else {
+                finalResult -= value;
+            }
+        }
+        byte[] resultPtr=new byte[getDataType().getByteSize()];
+        ptr.set(resultPtr);
+        getDataType().getCodec().encodeLong(finalResult, ptr);
+        return true;
+    }
+
+    @Override
+    public final PDataType getDataType() {
+        return PDataType.DATE;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/DecimalAddExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/DecimalAddExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/DecimalAddExpression.java
new file mode 100644
index 0000000..24444a6
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/DecimalAddExpression.java
@@ -0,0 +1,101 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.math.BigDecimal;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.exception.ValueTypeIncompatibleException;
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.NumberUtil;
+
+
+public class DecimalAddExpression extends AddExpression {
+    private Integer maxLength;
+    private Integer scale;
+
+    public DecimalAddExpression() {
+    }
+
+    public DecimalAddExpression(List<Expression> children) {
+        super(children);
+        for (int i=0; i<children.size(); i++) {
+            Expression childExpr = children.get(i);
+            if (i == 0) {
+                maxLength = childExpr.getMaxLength();
+                scale = childExpr.getScale();
+            } else {
+                maxLength = getPrecision(maxLength, childExpr.getMaxLength(), scale, childExpr.getScale());
+                scale = getScale(maxLength, childExpr.getMaxLength(), scale, childExpr.getScale());
+            }
+        }
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        BigDecimal result = null;
+        for (int i=0; i<children.size(); i++) {
+            Expression childExpr = children.get(i);
+            if (!childExpr.evaluate(tuple, ptr)) {
+                return false;
+            }
+            if (ptr.getLength() == 0) {
+                return true;
+            }
+            
+            PDataType childType = childExpr.getDataType();
+            ColumnModifier childColumnModifier = childExpr.getColumnModifier();
+            BigDecimal bd = (BigDecimal)PDataType.DECIMAL.toObject(ptr, childType, childColumnModifier);
+            
+            if (result == null) {
+                result = bd;
+            } else {
+                result = result.add(bd);
+            }
+        }
+        if (maxLength != null && scale != null) {
+            result = NumberUtil.setDecimalWidthAndScale(result, maxLength, scale);
+        }
+        if (result == null) {
+            throw new ValueTypeIncompatibleException(PDataType.DECIMAL, maxLength, scale);
+        }
+        ptr.set(PDataType.DECIMAL.toBytes(result));
+        return true;
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return PDataType.DECIMAL;
+    }
+
+    @Override
+    public Integer getScale() {
+        return scale;
+    }
+
+    @Override
+    public Integer getMaxLength() {
+        return maxLength;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/DecimalDivideExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/DecimalDivideExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/DecimalDivideExpression.java
new file mode 100644
index 0000000..a1e3b34
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/DecimalDivideExpression.java
@@ -0,0 +1,79 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.math.BigDecimal;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.exception.ValueTypeIncompatibleException;
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.NumberUtil;
+
+
+public class DecimalDivideExpression extends DivideExpression {
+
+    public DecimalDivideExpression() {
+    }
+
+    public DecimalDivideExpression(List<Expression> children) {
+        super(children);
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        BigDecimal result = null;
+        for (int i=0; i<children.size(); i++) {
+            Expression childExpr = children.get(i);
+            if (!childExpr.evaluate(tuple, ptr)) {
+                return false;
+            }
+            if (ptr.getLength() == 0) {
+                return true;
+            }
+            
+            PDataType childType = childExpr.getDataType();
+            ColumnModifier childColumnModifier = childExpr.getColumnModifier();
+            BigDecimal bd= (BigDecimal)PDataType.DECIMAL.toObject(ptr, childType, childColumnModifier);
+            
+            if (result == null) {
+                result = bd;
+            } else {
+                result = result.divide(bd, PDataType.DEFAULT_MATH_CONTEXT);
+            }
+        }
+        if (getMaxLength() != null && getScale() != null) {
+            result = NumberUtil.setDecimalWidthAndScale(result, getMaxLength(), getScale());
+        }
+        if (result == null) {
+            throw new ValueTypeIncompatibleException(PDataType.DECIMAL, getMaxLength(), getScale());
+        }
+        ptr.set(PDataType.DECIMAL.toBytes(result));
+        return true;
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return PDataType.DECIMAL;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/DecimalMultiplyExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/DecimalMultiplyExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/DecimalMultiplyExpression.java
new file mode 100644
index 0000000..8b5ffae
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/DecimalMultiplyExpression.java
@@ -0,0 +1,79 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.math.BigDecimal;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.exception.ValueTypeIncompatibleException;
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.NumberUtil;
+
+
+public class DecimalMultiplyExpression extends MultiplyExpression {
+
+    public DecimalMultiplyExpression() {
+    }
+
+    public DecimalMultiplyExpression(List<Expression> children) {
+        super(children);
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        BigDecimal result = null;
+        for (int i=0; i<children.size(); i++) {
+            Expression childExpr = children.get(i);
+            if (!childExpr.evaluate(tuple, ptr)) {
+                return false;
+            }
+            if (ptr.getLength() == 0) {
+                return true;
+            }
+            
+            PDataType childType = children.get(i).getDataType();
+            ColumnModifier childColumnModifier = children.get(i).getColumnModifier();
+            BigDecimal bd = (BigDecimal)PDataType.DECIMAL.toObject(ptr, childType, childColumnModifier);
+            
+            if (result == null) {
+                result = bd;
+            } else {
+                result = result.multiply(bd);
+            }
+        }
+        if (getMaxLength() != null && getScale() != null) {
+            result = NumberUtil.setDecimalWidthAndScale(result, getMaxLength(), getScale());
+        }
+        if (result == null) {
+            throw new ValueTypeIncompatibleException(PDataType.DECIMAL, getMaxLength(), getScale());
+        }
+        ptr.set(PDataType.DECIMAL.toBytes(result));
+        return true;
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return PDataType.DECIMAL;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/DecimalSubtractExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/DecimalSubtractExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/DecimalSubtractExpression.java
new file mode 100644
index 0000000..a572cf2
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/DecimalSubtractExpression.java
@@ -0,0 +1,118 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.math.BigDecimal;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.exception.ValueTypeIncompatibleException;
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.NumberUtil;
+
+
+/**
+ * 
+ * Subtract expression implementation
+ *
+ * @author kmahadik
+ * @since 0.1
+ */
+public class DecimalSubtractExpression extends SubtractExpression {
+    private Integer maxLength;
+    private Integer scale;
+
+    public DecimalSubtractExpression() {
+    }
+
+    public DecimalSubtractExpression(List<Expression> children) {
+        super(children);
+        for (int i=0; i<children.size(); i++) {
+            Expression childExpr = children.get(i);
+            if (i == 0) {
+                maxLength = childExpr.getMaxLength();
+                scale = childExpr.getScale();
+            } else {
+                maxLength = getPrecision(maxLength, childExpr.getMaxLength(), scale, childExpr.getScale());
+                scale = getScale(maxLength, childExpr.getMaxLength(), scale, childExpr.getScale());
+            }
+        }
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        BigDecimal result = null;
+        for (int i=0; i<children.size(); i++) {
+            Expression childExpr = children.get(i);
+            if (!childExpr.evaluate(tuple, ptr)) { 
+                return false;
+            }
+            if (ptr.getLength() == 0) {
+                return true;
+            }
+            
+            PDataType childType = childExpr.getDataType();
+            boolean isDate = childType.isCoercibleTo(PDataType.DATE);
+            ColumnModifier childColumnModifier = childExpr.getColumnModifier();
+            BigDecimal bd = isDate ?
+                    BigDecimal.valueOf(childType.getCodec().decodeLong(ptr, childColumnModifier)) :
+                    (BigDecimal)PDataType.DECIMAL.toObject(ptr, childType, childColumnModifier);
+            
+            if (result == null) {
+                result = bd;
+            } else {
+                result = result.subtract(bd);
+                /*
+                 * Special case for date subtraction - note that only first two expression may be dates.
+                 * We need to convert the date to a unit of "days" because that's what sql expects.
+                 */
+                if (isDate) {
+                    result = result.divide(BD_MILLIS_IN_DAY, PDataType.DEFAULT_MATH_CONTEXT);
+                }
+            }
+        }
+        if (maxLength != null && scale != null) {
+            result = NumberUtil.setDecimalWidthAndScale(result, maxLength, scale);
+        }
+        if (result == null) {
+            throw new ValueTypeIncompatibleException(PDataType.DECIMAL, maxLength, scale);
+        }
+        ptr.set(PDataType.DECIMAL.toBytes(result));
+        return true;
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return PDataType.DECIMAL;
+    }
+
+    @Override
+    public Integer getScale() {
+        return scale;
+    }
+
+    @Override
+    public Integer getMaxLength() {
+        return maxLength;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/DivideExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/DivideExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/DivideExpression.java
new file mode 100644
index 0000000..ab9d0d7
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/DivideExpression.java
@@ -0,0 +1,98 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.util.List;
+
+import org.apache.phoenix.expression.visitor.ExpressionVisitor;
+import org.apache.phoenix.schema.PDataType;
+
+
+/**
+ * 
+ * Divide expression implementation
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public abstract class DivideExpression extends ArithmeticExpression {
+    private Integer maxLength;
+    private Integer scale;
+
+    public DivideExpression() {
+    }
+
+    public DivideExpression(List<Expression> children) {
+        super(children);
+        for (int i=0; i<children.size(); i++) {
+            Expression childExpr = children.get(i);
+            if (i == 0) {
+                maxLength = childExpr.getMaxLength();
+                scale = childExpr.getScale();
+            } else {
+                maxLength = getPrecision(maxLength, childExpr.getMaxLength(), scale, childExpr.getScale());
+                scale = getScale(maxLength, childExpr.getMaxLength(), scale, childExpr.getScale());
+            }
+        }
+    }
+
+    @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;
+    }
+
+    @Override
+    public String getOperatorString() {
+        return " / ";
+    }
+    
+    private static Integer getPrecision(Integer lp, Integer rp, Integer ls, Integer rs) {
+    	if (ls == null || rs == null) {
+    		return PDataType.MAX_PRECISION;
+    	}
+        int val = getScale(lp, rp, ls, rs) + lp - ls + rp;
+        return Math.min(PDataType.MAX_PRECISION, val);
+    }
+
+    private static Integer getScale(Integer lp, Integer rp, Integer ls, Integer rs) {
+    	// If we are adding a decimal with scale and precision to a decimal
+    	// with no precision nor scale, the scale system does not apply.
+    	if (ls == null || rs == null) {
+    		return null;
+    	}
+        int val = Math.max(PDataType.MAX_PRECISION - lp + ls - rs, 0);
+        return Math.min(PDataType.MAX_PRECISION, val);
+    }
+
+    @Override
+    public Integer getScale() {
+        return scale;
+    }
+
+    @Override
+    public Integer getMaxLength() {
+        return maxLength;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/DoubleAddExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/DoubleAddExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/DoubleAddExpression.java
new file mode 100644
index 0000000..96fc9fa
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/DoubleAddExpression.java
@@ -0,0 +1,70 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+public class DoubleAddExpression extends AddExpression {
+
+    public DoubleAddExpression() {
+    }
+
+    public DoubleAddExpression(List<Expression> children) {
+        super(children);
+    }
+    
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        double result = 0.0;
+        for (int i = 0; i < children.size(); i++) {
+            Expression child = children.get(i);
+            if (!child.evaluate(tuple, ptr)) {
+                return false;
+            }
+            if (ptr.getLength() == 0) {
+                return true;
+            }
+            double childvalue = child.getDataType().getCodec()
+                    .decodeDouble(ptr, child.getColumnModifier());
+            if (!Double.isNaN(childvalue)
+                    && childvalue != Double.NEGATIVE_INFINITY
+                    && childvalue != Double.POSITIVE_INFINITY) {
+                result += childvalue;
+            } else {
+                return false;
+            }
+        }
+        byte[] resultPtr = new byte[getDataType().getByteSize()];
+        ptr.set(resultPtr);
+        getDataType().getCodec().encodeDouble(result, ptr);
+        return true;
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return PDataType.DOUBLE;
+    }
+
+}


[47/51] [partial] Initial commit of master branch from github

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/filter/ApplyAndFilterDeletesFilter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/filter/ApplyAndFilterDeletesFilter.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/filter/ApplyAndFilterDeletesFilter.java
new file mode 100644
index 0000000..ebec75a
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/filter/ApplyAndFilterDeletesFilter.java
@@ -0,0 +1,310 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index.covered.filter;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValue.Type;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.filter.FilterBase;
+
+import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+
+/**
+ * Only allow the 'latest' timestamp of each family:qualifier pair, ensuring that they aren't
+ * covered by a previous delete. This is similar to some of the work the ScanQueryMatcher does to
+ * ensure correct visibility of keys based on deletes.
+ * <p>
+ * No actual delete {@link KeyValue}s are allowed to pass through this filter - they are always
+ * skipped.
+ * <p>
+ * Note there is a little bit of conceptually odd behavior (though it matches the HBase
+ * specifications) around point deletes ({@link KeyValue} of type {@link Type#Delete}. These deletes
+ * only apply to a single {@link KeyValue} at a single point in time - they essentially completely
+ * 'cover' the existing {@link Put} at that timestamp. However, they don't 'cover' any other
+ * keyvalues at older timestamps. Therefore, if there is a point-delete at ts = 5, and puts at ts =
+ * 4, and ts = 5, we will only allow the put at ts = 4.
+ * <p>
+ * Expects {@link KeyValue}s to arrive in sorted order, with 'Delete' {@link Type} {@link KeyValue}s
+ * ({@link Type#DeleteColumn}, {@link Type#DeleteFamily}, {@link Type#Delete})) before their regular
+ * {@link Type#Put} counterparts.
+ */
+public class ApplyAndFilterDeletesFilter extends FilterBase {
+
+  private boolean done = false;
+  List<ImmutableBytesPtr> families;
+  private final DeleteTracker coveringDelete = new DeleteTracker();
+  private Hinter currentHint;
+  private DeleteColumnHinter columnHint = new DeleteColumnHinter();
+  private DeleteFamilyHinter familyHint = new DeleteFamilyHinter();
+  
+  /**
+   * Setup the filter to only include the given families. This allows us to seek intelligently pass
+   * families we don't care about.
+   * @param families
+   */
+  public ApplyAndFilterDeletesFilter(Set<ImmutableBytesPtr> families) {
+    this.families = new ArrayList<ImmutableBytesPtr>(families);
+    Collections.sort(this.families);
+  }
+      
+  
+  private ImmutableBytesPtr getNextFamily(ImmutableBytesPtr family) {
+    int index = Collections.binarySearch(families, family);
+    //doesn't match exactly, be we can find the right next match
+    //this is pretty unlikely, but just incase
+    if(index < 0){
+      //the actual location of the next match
+      index = -index -1;
+    }else{
+      //its an exact match for a family, so we get the next entry
+      index = index +1;
+    }
+    //now we have the location of the next entry
+    if(index >= families.size()){
+      return null;
+    }
+    return  families.get(index);
+  }
+  
+  @Override
+  public void reset(){
+    this.coveringDelete.reset();
+    this.done = false;
+  }
+  
+  
+  @Override
+  public KeyValue getNextKeyHint(KeyValue peeked){
+    return currentHint.getHint(peeked);
+  }
+
+  @Override
+  public ReturnCode filterKeyValue(KeyValue next) {
+    // we marked ourselves done, but the END_ROW_KEY didn't manage to seek to the very last key
+    if (this.done) {
+      return ReturnCode.SKIP;
+    }
+
+    switch (KeyValue.Type.codeToType(next.getType())) {
+    /*
+     * DeleteFamily will always sort first because those KVs (we assume) don't have qualifiers (or
+     * rather are null). Therefore, we have to keep a hold of all the delete families until we get
+     * to a Put entry that is covered by that delete (in which case, we are done with the family).
+     */
+    case DeleteFamily:
+      // track the family to delete. If we are updating the delete, that means we have passed all
+      // kvs in the last column, so we can safely ignore the last deleteFamily, and just use this
+      // one. In fact, it means that all the previous deletes can be ignored because the family must
+      // not match anymore.
+      this.coveringDelete.reset();
+      this.coveringDelete.deleteFamily = next;
+      return ReturnCode.SKIP;
+    case DeleteColumn:
+      // similar to deleteFamily, all the newer deletes/puts would have been seen at this point, so
+      // we can safely replace the more recent delete column with the more recent one
+      this.coveringDelete.pointDelete = null;
+      this.coveringDelete.deleteColumn = next;
+      return ReturnCode.SKIP;
+    case Delete:
+      // we are just deleting the single column value at this point.
+      // therefore we just skip this entry and go onto the next one. The only caveat is that
+      // we should still cover the next entry if this delete applies to the next entry, so we
+      // have to keep around a reference to the KV to compare against the next valid entry
+      this.coveringDelete.pointDelete = next;
+      return ReturnCode.SKIP;
+    default:
+      // no covering deletes
+      if (coveringDelete.empty()) {
+        return ReturnCode.INCLUDE;
+      }
+
+      if (coveringDelete.matchesFamily(next)) {
+        this.currentHint = familyHint;
+        return ReturnCode.SEEK_NEXT_USING_HINT;
+      }
+
+      if (coveringDelete.matchesColumn(next)) {
+        // hint to the next column
+        this.currentHint = columnHint;
+        return ReturnCode.SEEK_NEXT_USING_HINT;
+      }
+
+      if (coveringDelete.matchesPoint(next)) {
+        return ReturnCode.SKIP;
+      }
+
+    }
+
+    // none of the deletes matches, we are done
+    return ReturnCode.INCLUDE;
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    throw new UnsupportedOperationException("Server-side only filter, cannot be serialized!");
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    throw new UnsupportedOperationException("Server-side only filter, cannot be deserialized!");
+  }
+
+  /**
+   * Get the next hint for a given peeked keyvalue
+   */
+  interface Hinter {
+    public abstract KeyValue getHint(KeyValue peek);
+  }
+
+  /**
+   * Entire family has been deleted, so either seek to the next family, or if none are present in
+   * the original set of families to include, seek to the "last possible key"(or rather our best
+   * guess) and be done.
+   */
+  class DeleteFamilyHinter implements Hinter {
+
+    @Override
+    public KeyValue getHint(KeyValue peeked) {
+      // check to see if we have another column to seek
+      ImmutableBytesPtr nextFamily =
+          getNextFamily(new ImmutableBytesPtr(peeked.getBuffer(), peeked.getFamilyOffset(),
+              peeked.getFamilyLength()));
+      if (nextFamily == null) {
+        // no known next family, so we can be completely done
+        done = true;
+        return KeyValue.LOWESTKEY;
+      }
+        // there is a valid family, so we should seek to that
+      return KeyValue.createFirstOnRow(peeked.getRow(), nextFamily.copyBytesIfNecessary(),
+        HConstants.EMPTY_BYTE_ARRAY);
+    }
+
+  }
+
+  /**
+   * Hint the next column-qualifier after the given keyvalue. We can't be smart like in the
+   * ScanQueryMatcher since we don't know the columns ahead of time.
+   */
+  class DeleteColumnHinter implements Hinter {
+
+    @Override
+    public KeyValue getHint(KeyValue kv) {
+      return KeyValue.createLastOnRow(kv.getBuffer(), kv.getRowOffset(), kv.getRowLength(),
+        kv.getBuffer(), kv.getFamilyOffset(), kv.getFamilyLength(), kv.getBuffer(),
+        kv.getQualifierOffset(), kv.getQualifierLength());
+    }
+  }
+
+  class DeleteTracker {
+
+    public KeyValue deleteFamily;
+    public KeyValue deleteColumn;
+    public KeyValue pointDelete;
+
+    public void reset() {
+      this.deleteFamily = null;
+      this.deleteColumn = null;
+      this.pointDelete = null;
+
+    }
+
+    /**
+     * Check to see if we should skip this {@link KeyValue} based on the family.
+     * <p>
+     * Internally, also resets the currently tracked "Delete Family" marker we are tracking if the
+     * keyvalue is into another family (since CFs sort lexicographically, we can discard the current
+     * marker since it must not be applicable to any more kvs in a linear scan).
+     * @param next
+     * @return <tt>true</tt> if this {@link KeyValue} matches a delete.
+     */
+    public boolean matchesFamily(KeyValue next) {
+      if (deleteFamily == null) {
+        return false;
+      }
+      if (deleteFamily.matchingFamily(next)) {
+        // falls within the timestamp range
+        if (deleteFamily.getTimestamp() >= next.getTimestamp()) {
+          return true;
+        }
+      } else {
+        // only can reset the delete family because we are on to another family
+        deleteFamily = null;
+      }
+
+      return false;
+    }
+
+
+    /**
+     * @param next
+     * @return
+     */
+    public boolean matchesColumn(KeyValue next) {
+      if (deleteColumn == null) {
+        return false;
+      }
+      if (deleteColumn.matchingFamily(next) && deleteColumn.matchingQualifier(next)) {
+        // falls within the timestamp range
+        if (deleteColumn.getTimestamp() >= next.getTimestamp()) {
+          return true;
+        }
+      } else {
+        deleteColumn = null;
+      }
+      return false;
+    }
+
+    /**
+     * @param next
+     * @return
+     */
+    public boolean matchesPoint(KeyValue next) {
+      // point deletes only apply to the exact KV that they reference, so we only need to ensure
+      // that the timestamp matches exactly. Because we sort by timestamp first, either the next
+      // keyvalue has the exact timestamp or is an older (smaller) timestamp, and we can allow that
+      // one.
+      if (pointDelete != null && pointDelete.matchingFamily(next)
+          && pointDelete.matchingQualifier(next)) {
+        if (pointDelete.getTimestamp() == next.getTimestamp()) {
+          return true;
+        }
+        // clear the point delete since the TS must not be matching
+        coveringDelete.pointDelete = null;
+      }
+      return false;
+    }
+
+    /**
+     * @return <tt>true</tt> if no delete has been set
+     */
+    public boolean empty() {
+      return deleteFamily == null && deleteColumn == null && pointDelete == null;
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/filter/ColumnTrackingNextLargestTimestampFilter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/filter/ColumnTrackingNextLargestTimestampFilter.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/filter/ColumnTrackingNextLargestTimestampFilter.java
new file mode 100644
index 0000000..b23368d
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/filter/ColumnTrackingNextLargestTimestampFilter.java
@@ -0,0 +1,74 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index.covered.filter;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.filter.FilterBase;
+
+import org.apache.hadoop.hbase.index.covered.update.ColumnTracker;
+
+/**
+ * Similar to the {@link MaxTimestampFilter}, but also updates the 'next largest' timestamp seen
+ * that is not skipped by the below criteria. Note that it isn't as quick as the
+ * {@link MaxTimestampFilter} as we can't just seek ahead to a key with the matching timestamp, but
+ * have to iterate each kv until we find the right one with an allowed timestamp.
+ * <p>
+ * Inclusively filter on the maximum timestamp allowed. Excludes all elements greater than (but not
+ * equal to) the given timestamp, so given ts = 5, a {@link KeyValue} with ts 6 is excluded, but not
+ * one with ts = 5.
+ * <p>
+ * This filter generally doesn't make sense on its own - it should follow a per-column filter and
+ * possible a per-delete filter to only track the most recent (but not exposed to the user)
+ * timestamp.
+ */
+public class ColumnTrackingNextLargestTimestampFilter extends FilterBase {
+
+  private long ts;
+  private ColumnTracker column;
+
+  public ColumnTrackingNextLargestTimestampFilter(long maxTime, ColumnTracker toTrack) {
+    this.ts = maxTime;
+    this.column = toTrack;
+  }
+
+  @Override
+  public ReturnCode filterKeyValue(KeyValue v) {
+    long timestamp = v.getTimestamp();
+    if (timestamp > ts) {
+      this.column.setTs(timestamp);
+      return ReturnCode.SKIP;
+    }
+    return ReturnCode.INCLUDE;
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    throw new UnsupportedOperationException("Server-side only filter, cannot be serialized!");
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    throw new UnsupportedOperationException("Server-side only filter, cannot be deserialized!");
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/filter/FamilyOnlyFilter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/filter/FamilyOnlyFilter.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/filter/FamilyOnlyFilter.java
new file mode 100644
index 0000000..8591f88
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/filter/FamilyOnlyFilter.java
@@ -0,0 +1,80 @@
+/**
+ * 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.hadoop.hbase.index.covered.filter;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.filter.BinaryComparator;
+import org.apache.hadoop.hbase.filter.FamilyFilter;
+import org.apache.hadoop.hbase.filter.WritableByteArrayComparable;
+
+/**
+ * Similar to the {@link FamilyFilter} but stops when the end of the family is reached and only
+ * supports equality
+ */
+public class FamilyOnlyFilter extends FamilyFilter {
+
+  boolean done = false;
+  private boolean previousMatchFound;
+
+  /**
+   * Filter on exact binary matches to the passed family
+   * @param family to compare against
+   */
+  public FamilyOnlyFilter(final byte[] family) {
+    this(new BinaryComparator(family));
+  }
+
+  public FamilyOnlyFilter(final WritableByteArrayComparable familyComparator) {
+    super(CompareOp.EQUAL, familyComparator);
+  }
+
+
+  @Override
+  public boolean filterAllRemaining() {
+    return done;
+  }
+
+  @Override
+  public void reset() {
+    done = false;
+    previousMatchFound = false;
+  }
+
+  @Override
+  public ReturnCode filterKeyValue(KeyValue v) {
+    if (done) {
+      return ReturnCode.SKIP;
+    }
+    ReturnCode code = super.filterKeyValue(v);
+    if (previousMatchFound) {
+      // we found a match before, and now we are skipping the key because of the family, therefore
+      // we are done (no more of the family).
+      if (code.equals(ReturnCode.SKIP)) {
+      done = true;
+      }
+    } else {
+      // if we haven't seen a match before, then it doesn't matter what we see now, except to mark
+      // if we've seen a match
+      if (code.equals(ReturnCode.INCLUDE)) {
+        previousMatchFound = true;
+      }
+    }
+    return code;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/filter/MaxTimestampFilter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/filter/MaxTimestampFilter.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/filter/MaxTimestampFilter.java
new file mode 100644
index 0000000..5b76579
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/filter/MaxTimestampFilter.java
@@ -0,0 +1,76 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index.covered.filter;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.filter.FilterBase;
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * Inclusive filter on the maximum timestamp allowed. Excludes all elements greater than (but not
+ * equal to) the given timestamp, so given ts = 5, a {@link KeyValue} with ts 6 is excluded, but not
+ * one with ts = 5.
+ */
+public class MaxTimestampFilter extends FilterBase {
+
+  private long ts;
+
+  public MaxTimestampFilter(long maxTime) {
+    this.ts = maxTime;
+  }
+
+  @Override
+  public KeyValue getNextKeyHint(KeyValue currentKV) {
+    // this might be a little excessive right now - better safe than sorry though, so we don't mess
+    // with other filters too much.
+    KeyValue kv = currentKV.deepCopy();
+    int offset =kv.getTimestampOffset();
+    //set the timestamp in the buffer
+    byte[] buffer = kv.getBuffer();
+    byte[] ts = Bytes.toBytes(this.ts);
+    System.arraycopy(ts, 0, buffer, offset, ts.length);
+
+    return kv;
+  }
+
+  @Override
+  public ReturnCode filterKeyValue(KeyValue v) {
+    long timestamp = v.getTimestamp();
+    if (timestamp > ts) {
+      return ReturnCode.SEEK_NEXT_USING_HINT;
+    }
+    return ReturnCode.INCLUDE;
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    throw new UnsupportedOperationException("Server-side only filter, cannot be serialized!");
+
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    throw new UnsupportedOperationException("Server-side only filter, cannot be deserialized!");
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/filter/NewerTimestampFilter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/filter/NewerTimestampFilter.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/filter/NewerTimestampFilter.java
new file mode 100644
index 0000000..560cdd8
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/filter/NewerTimestampFilter.java
@@ -0,0 +1,37 @@
+package org.apache.hadoop.hbase.index.covered.filter;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.filter.FilterBase;
+
+/**
+ * Server-side only class used in the indexer to filter out keyvalues newer than a given timestamp
+ * (so allows anything <code><=</code> timestamp through).
+ * <p>
+ * Note,<tt>this</tt> doesn't support {@link #write(DataOutput)} or {@link #readFields(DataInput)}.
+ */
+public class NewerTimestampFilter extends FilterBase {
+
+  private long timestamp;
+
+  public NewerTimestampFilter(long timestamp) {
+    this.timestamp = timestamp;
+  }
+
+  @Override
+  public ReturnCode filterKeyValue(KeyValue ignored) {
+    return ignored.getTimestamp() > timestamp ? ReturnCode.SKIP : ReturnCode.INCLUDE;
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    throw new UnsupportedOperationException("TimestampFilter is server-side only!");
+  }
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    throw new UnsupportedOperationException("TimestampFilter is server-side only!");
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/update/ColumnReference.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/update/ColumnReference.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/update/ColumnReference.java
new file mode 100644
index 0000000..0525028
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/update/ColumnReference.java
@@ -0,0 +1,168 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index.covered.update;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+
+/**
+ * 
+ */
+public class ColumnReference implements Comparable<ColumnReference> {
+    
+  public static final byte[] ALL_QUALIFIERS = new byte[0];
+  
+  private static int calcHashCode(byte[] family, byte[] qualifier) {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + Bytes.hashCode(family);
+    result = prime * result + Bytes.hashCode(qualifier);
+    return result;
+  }
+
+  private final int hashCode;
+  protected final byte[] family;
+  protected final byte[] qualifier;
+    private volatile ImmutableBytesWritable familyPtr;
+    private volatile ImmutableBytesWritable qualifierPtr;
+
+  public ColumnReference(byte[] family, byte[] qualifier) {
+    this.family = family;
+    this.qualifier = qualifier;
+    this.hashCode = calcHashCode(family, qualifier);
+  }
+
+  public byte[] getFamily() {
+    return this.family;
+  }
+
+  public byte[] getQualifier() {
+    return this.qualifier;
+  }
+  
+    public ImmutableBytesWritable getFamilyWritable() {
+        if (this.familyPtr == null) {
+            synchronized (this.family) {
+                if (this.familyPtr == null) {
+                    this.familyPtr = new ImmutableBytesPtr(this.family);
+                }
+            }
+        }
+        return this.familyPtr;
+    }
+
+    public ImmutableBytesWritable getQualifierWritable() {
+        if (this.qualifierPtr == null) {
+            synchronized (this.qualifier) {
+                if (this.qualifierPtr == null) {
+                    this.qualifierPtr = new ImmutableBytesPtr(this.qualifier);
+                }
+            }
+        }
+        return this.qualifierPtr;
+    }
+
+  public boolean matches(KeyValue kv) {
+    if (matchesFamily(kv.getBuffer(), kv.getFamilyOffset(), kv.getFamilyLength())) {
+      return matchesQualifier(kv.getBuffer(), kv.getQualifierOffset(), kv.getQualifierLength());
+    }
+    return false;
+  }
+
+  /**
+   * @param qual to check against
+   * @return <tt>true</tt> if this column covers the given qualifier.
+   */
+  public boolean matchesQualifier(byte[] qual) {
+    return matchesQualifier(qual, 0, qual.length);
+  }
+
+  public boolean matchesQualifier(byte[] bytes, int offset, int length) {
+    return allColumns() ? true : match(bytes, offset, length, qualifier);
+  }
+
+  /**
+   * @param family to check against
+   * @return <tt>true</tt> if this column covers the given family.
+   */
+  public boolean matchesFamily(byte[] family) {
+    return matchesFamily(family, 0, family.length);
+  }
+
+  public boolean matchesFamily(byte[] bytes, int offset, int length) {
+    return match(bytes, offset, length, family);
+  }
+
+  /**
+   * @return <tt>true</tt> if this should include all column qualifiers, <tt>false</tt> otherwise
+   */
+  public boolean allColumns() {
+    return this.qualifier == ALL_QUALIFIERS;
+  }
+
+  /**
+   * Check to see if the passed bytes match the stored bytes
+   * @param first
+   * @param storedKey the stored byte[], should never be <tt>null</tt>
+   * @return <tt>true</tt> if they are byte-equal
+   */
+  private boolean match(byte[] first, int offset, int length, byte[] storedKey) {
+    return first == null ? false : Bytes.equals(first, offset, length, storedKey, 0,
+      storedKey.length);
+  }
+
+  public KeyValue getFirstKeyValueForRow(byte[] row) {
+    return KeyValue.createFirstOnRow(row, family, qualifier == ALL_QUALIFIERS ? null : qualifier);
+  }
+
+  @Override
+  public int compareTo(ColumnReference o) {
+    int c = Bytes.compareTo(family, o.family);
+    if (c == 0) {
+      // matching families, compare qualifiers
+      c = Bytes.compareTo(qualifier, o.qualifier);
+    }
+    return c;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (o instanceof ColumnReference) {
+      ColumnReference other = (ColumnReference) o;
+      if (hashCode == other.hashCode && Bytes.equals(family, other.family)) {
+        return Bytes.equals(qualifier, other.qualifier);
+      }
+    }
+    return false;
+  }
+
+  @Override
+  public int hashCode() {
+    return hashCode;
+  }
+
+  @Override
+  public String toString() {
+    return "ColumnReference - " + Bytes.toString(family) + ":" + Bytes.toString(qualifier);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/update/ColumnTracker.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/update/ColumnTracker.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/update/ColumnTracker.java
new file mode 100644
index 0000000..e41c314
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/update/ColumnTracker.java
@@ -0,0 +1,117 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index.covered.update;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+
+/**
+ * Simple POJO for tracking a bunch of column references and the next-newest timestamp for those
+ * columns
+ * <p>
+ * Two {@link ColumnTracker}s are considered equal if they track the same columns, even if their
+ * timestamps are different.
+ */
+public class ColumnTracker implements IndexedColumnGroup {
+
+  public static final long NO_NEWER_PRIMARY_TABLE_ENTRY_TIMESTAMP = Long.MAX_VALUE;
+  public static final long GUARANTEED_NEWER_UPDATES = Long.MIN_VALUE;
+  private final List<ColumnReference> columns;
+  private long ts = NO_NEWER_PRIMARY_TABLE_ENTRY_TIMESTAMP;
+  private final int hashCode;
+
+  private static int calcHashCode(List<ColumnReference> columns) {
+      return columns.hashCode();
+    }
+
+  public ColumnTracker(Collection<? extends ColumnReference> columns) {
+    this.columns = new ArrayList<ColumnReference>(columns);
+    // sort the columns
+    Collections.sort(this.columns);
+    this.hashCode = calcHashCode(this.columns);
+  }
+
+  /**
+   * Set the current timestamp, only if the passed timestamp is strictly less than the currently
+   * stored timestamp
+   * @param ts the timestmap to potentially store.
+   * @return the currently stored timestamp.
+   */
+  public long setTs(long ts) {
+    this.ts = this.ts > ts ? ts : this.ts;
+    return this.ts;
+  }
+
+  public long getTS() {
+    return this.ts;
+  }
+
+  @Override
+  public int hashCode() {
+    return hashCode;
+  }
+
+  @Override
+  public boolean equals(Object o){
+    if(!(o instanceof ColumnTracker)){
+      return false;
+    }
+    ColumnTracker other = (ColumnTracker)o;
+    if (hashCode != other.hashCode) {
+        return false;
+    }
+    if (other.columns.size() != columns.size()) {
+      return false;
+    }
+
+    // check each column to see if they match
+    for (int i = 0; i < columns.size(); i++) {
+      if (!columns.get(i).equals(other.columns.get(i))) {
+        return false;
+      }
+    }
+
+    return true;
+  }
+
+  @Override
+  public List<ColumnReference> getColumns() {
+    return this.columns;
+  }
+
+  /**
+   * @return <tt>true</tt> if this set of columns has seen a column with a timestamp newer than the
+   *         requested timestamp, <tt>false</tt> otherwise.
+   */
+  public boolean hasNewerTimestamps() {
+    return !isNewestTime(this.ts);
+  }
+
+  /**
+   * @param ts timestamp to check
+   * @return <tt>true</tt> if the timestamp is at the most recent timestamp for a column
+   */
+  public static boolean isNewestTime(long ts) {
+    return ts == NO_NEWER_PRIMARY_TABLE_ENTRY_TIMESTAMP;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/update/IndexUpdateManager.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/update/IndexUpdateManager.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/update/IndexUpdateManager.java
new file mode 100644
index 0000000..52eb5dd
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/update/IndexUpdateManager.java
@@ -0,0 +1,240 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index.covered.update;
+
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+
+import com.google.common.collect.Lists;
+import com.google.common.primitives.Longs;
+import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+
+/**
+ * Keeps track of the index updates
+ */
+public class IndexUpdateManager {
+
+  public Comparator<Mutation> COMPARATOR = new MutationComparator();
+  class MutationComparator implements Comparator<Mutation> {
+
+    @Override
+    public int compare(Mutation o1, Mutation o2) {
+      // always sort rows first
+      int compare = o1.compareTo(o2);
+      if (compare != 0) {
+        return compare;
+      }
+
+      // if same row, sort by reverse timestamp (larger first)
+      compare = Longs.compare(o2.getTimeStamp(), o1.getTimeStamp());
+      if (compare != 0) {
+        return compare;
+      }
+      // deletes always sort before puts for the same row
+      if (o1 instanceof Delete) {
+        // same row, same ts == same delete since we only delete rows
+        if (o2 instanceof Delete) {
+          return 0;
+        } else {
+          // o2 has to be a put
+          return -1;
+        }
+      }
+      // o1 must be a put
+      if (o2 instanceof Delete) {
+        return 1;
+      } else if (o2 instanceof Put) {
+        return comparePuts((Put) o1, (Put) o2);
+      }
+
+      throw new RuntimeException(
+          "Got unexpected mutation types! Can only be Put or Delete, but got: " + o1 + ", and "
+              + o2);
+    }
+
+    private int comparePuts(Put p1, Put p2) {
+      int p1Size = p1.size();
+      int p2Size = p2.size();
+      int compare = p1Size - p2Size;
+      if (compare == 0) {
+        // TODO: make this a real comparison
+        // this is a little cheating, but we don't really need to worry too much about this being
+        // the same - chances are that exact matches here are really the same update.
+        return Longs.compare(p1.heapSize(), p2.heapSize());
+      }
+      return compare;
+    }
+
+  }
+
+  private static final String PHOENIX_HBASE_TEMP_DELETE_MARKER = "phoenix.hbase.temp.delete.marker";
+  private static final byte[] TRUE_MARKER = new byte[] { 1 };
+
+  protected final Map<ImmutableBytesPtr, Collection<Mutation>> map =
+      new HashMap<ImmutableBytesPtr, Collection<Mutation>>();
+
+  /**
+   * Add an index update. Keeps the latest {@link Put} for a given timestamp
+   * @param tableName
+   * @param m
+   */
+  public void addIndexUpdate(byte[] tableName, Mutation m) {
+    // we only keep the most recent update
+    ImmutableBytesPtr key = new ImmutableBytesPtr(tableName);
+    Collection<Mutation> updates = map.get(key);
+    if (updates == null) {
+      updates = new SortedCollection<Mutation>(COMPARATOR);
+      map.put(key, updates);
+    }
+    fixUpCurrentUpdates(updates, m);
+  }
+
+  /**
+   * Fix up the current updates, given the pending mutation.
+   * @param updates current updates
+   * @param pendingMutation
+   */
+  protected void fixUpCurrentUpdates(Collection<Mutation> updates, Mutation pendingMutation) {
+    // need to check for each entry to see if we have a duplicate
+    Mutation toRemove = null;
+    Delete pendingDelete = pendingMutation instanceof Delete ? (Delete) pendingMutation : null;
+    boolean sawRowMatch = false;
+    for (Mutation stored : updates) {
+      int compare = pendingMutation.compareTo(stored);
+      // skip to the right row
+      if (compare < 0) {
+        continue;
+      } else if (compare > 0) {
+        if (sawRowMatch) {
+          break;
+        }
+        continue;
+      }
+
+      // set that we saw a row match, so any greater row will necessarily be the wrong
+      sawRowMatch = true;
+
+      // skip until we hit the right timestamp
+      if (stored.getTimeStamp() < pendingMutation.getTimeStamp()) {
+        continue;
+      }
+
+      if (stored instanceof Delete) {
+        // we already have a delete for this row, so we are done.
+        if (pendingDelete != null) {
+          return;
+        }
+        // pending update must be a Put, so we ignore the Put.
+        // add a marker in the this delete that it has been canceled out already. We need to keep
+        // the delete around though so we can figure out if other Puts would also be canceled out.
+        markMutationForRemoval(stored);
+        return;
+      }
+
+      // otherwise, the stored mutation is a Put. Either way, we want to remove it. If the pending
+      // update is a delete, we need to remove the entry (no longer applies - covered by the
+      // delete), or its an older version of the row, so we cover it with the newer.
+      toRemove = stored;
+      if (pendingDelete != null) {
+        // the pending mutation, but we need to mark the mutation for removal later
+        markMutationForRemoval(pendingMutation);
+        break;
+      }
+    }
+    
+    updates.remove(toRemove);
+    updates.add(pendingMutation);
+  }
+
+  private void markMutationForRemoval(Mutation m) {
+    m.setAttribute(PHOENIX_HBASE_TEMP_DELETE_MARKER, TRUE_MARKER);
+  }
+
+  public List<Pair<Mutation, byte[]>> toMap() {
+    List<Pair<Mutation, byte[]>> updateMap = Lists.newArrayList();
+    for (Entry<ImmutableBytesPtr, Collection<Mutation>> updates : map.entrySet()) {
+      // get is ok because we always set with just the bytes
+      byte[] tableName = updates.getKey().get();
+      // TODO replace this as just storing a byte[], to avoid all the String <-> byte[] swapping
+      // HBase does
+      for (Mutation m : updates.getValue()) {
+        // skip elements that have been marked for delete
+        if (shouldBeRemoved(m)) {
+          continue;
+        }
+        updateMap.add(new Pair<Mutation, byte[]>(m, tableName));
+      }
+    }
+    return updateMap;
+  }
+
+  /**
+   * @param updates
+   */
+  public void addAll(Collection<Pair<Mutation, String>> updates) {
+    for (Pair<Mutation, String> update : updates) {
+      addIndexUpdate(Bytes.toBytes(update.getSecond()), update.getFirst());
+    }
+  }
+
+  private boolean shouldBeRemoved(Mutation m) {
+    return m.getAttribute(PHOENIX_HBASE_TEMP_DELETE_MARKER) != null;
+  }
+
+  @Override
+  public String toString() {
+    StringBuffer sb = new StringBuffer("Pending Index Updates:\n");
+    for (Entry<ImmutableBytesPtr, Collection<Mutation>> entry : map.entrySet()) {
+      String tableName = Bytes.toString(entry.getKey().get());
+      sb.append("   Table: '" + tableName + "'\n");
+      for (Mutation m : entry.getValue()) {
+        sb.append("\t");
+        if (shouldBeRemoved(m)) {
+          sb.append("[REMOVED]");
+        }
+        sb.append(m.getClass().getSimpleName() + ":"
+            + ((m instanceof Put) ? m.getTimeStamp() + " " : ""));
+        sb.append(" row=" + Bytes.toString(m.getRow()));
+        sb.append("\n");
+        if (m.getFamilyMap().isEmpty()) {
+          sb.append("\t\t=== EMPTY ===\n");
+        }
+        for (List<KeyValue> kvs : m.getFamilyMap().values()) {
+          for (KeyValue kv : kvs) {
+            sb.append("\t\t" + kv.toString() + "/value=" + Bytes.toStringBinary(kv.getValue()));
+            sb.append("\n");
+          }
+        }
+      }
+    }
+    return sb.toString();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/update/IndexedColumnGroup.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/update/IndexedColumnGroup.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/update/IndexedColumnGroup.java
new file mode 100644
index 0000000..535742b
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/update/IndexedColumnGroup.java
@@ -0,0 +1,30 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index.covered.update;
+
+import java.util.List;
+
+/**
+ * Group of columns that were requested to build an index
+ */
+public interface IndexedColumnGroup {
+
+  public List<ColumnReference> getColumns();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/update/SortedCollection.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/update/SortedCollection.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/update/SortedCollection.java
new file mode 100644
index 0000000..b228e63
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/update/SortedCollection.java
@@ -0,0 +1,130 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index.covered.update;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.PriorityQueue;
+
+import com.google.common.collect.Iterators;
+
+/**
+ * A collection whose elements are stored and returned sorted.
+ * <p>
+ * We can't just use something like a {@link PriorityQueue} because it doesn't return the
+ * underlying values in sorted order.
+ * @param <T>
+ */
+class SortedCollection<T> implements Collection<T>, Iterable<T> {
+
+  private PriorityQueue<T> queue;
+  private Comparator<T> comparator;
+
+  /**
+   * Use the given comparator to compare all keys for sorting
+   * @param comparator
+   */
+  public SortedCollection(Comparator<T> comparator) {
+    this.queue = new PriorityQueue<T>(1, comparator);
+    this.comparator = comparator;
+  }
+  
+  /**
+   * All passed elements are expected to be {@link Comparable}
+   */
+  public SortedCollection() {
+    this.queue = new PriorityQueue<T>();
+  }
+  
+  @Override
+  public int size() {
+    return this.queue.size();
+  }
+
+  @Override
+  public boolean isEmpty() {
+    return this.queue.isEmpty();
+  }
+
+  @Override
+  public boolean contains(Object o) {
+    return this.queue.contains(o);
+  }
+
+  @Override
+  public Iterator<T> iterator() {
+    @SuppressWarnings("unchecked")
+    T[] array = (T[]) this.queue.toArray();
+    if (this.comparator == null) {
+      Arrays.sort(array);
+    } else {
+      Arrays.sort(
+     array, this.comparator);}
+    return Iterators.forArray(array);
+  }
+
+  @Override
+  public Object[] toArray() {
+    return this.queue.toArray();
+  }
+
+  @SuppressWarnings("hiding")
+  @Override
+  public <T> T[] toArray(T[] a) {
+    return this.queue.toArray(a);
+  }
+
+  @Override
+  public boolean add(T e) {
+    return this.queue.add(e);
+  }
+
+  @Override
+  public boolean remove(Object o) {
+    return this.queue.remove(o);
+  }
+
+  @Override
+  public boolean containsAll(Collection<?> c) {
+    return this.queue.containsAll(c);
+  }
+
+  @Override
+  public boolean addAll(Collection<? extends T> c) {
+    return this.queue.addAll(c);
+  }
+
+  @Override
+  public boolean removeAll(Collection<?> c) {
+    return queue.removeAll(c);
+  }
+
+  @Override
+  public boolean retainAll(Collection<?> c) {
+    return this.queue.retainAll(c);
+  }
+
+  @Override
+  public void clear() {
+    this.queue.clear();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/exception/IndexWriteException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/exception/IndexWriteException.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/exception/IndexWriteException.java
new file mode 100644
index 0000000..16fa1bc
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/exception/IndexWriteException.java
@@ -0,0 +1,45 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index.exception;
+
+import org.apache.hadoop.hbase.HBaseIOException;
+
+/**
+ * Generic {@link Exception} that an index write has failed
+ */
+@SuppressWarnings("serial")
+public class IndexWriteException extends HBaseIOException {
+
+  public IndexWriteException() {
+    super();
+  }
+
+  public IndexWriteException(String message, Throwable cause) {
+    super(message, cause);
+  }
+
+  public IndexWriteException(String message) {
+    super(message);
+  }
+
+  public IndexWriteException(Throwable cause) {
+    super(cause);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/exception/MultiIndexWriteFailureException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/exception/MultiIndexWriteFailureException.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/exception/MultiIndexWriteFailureException.java
new file mode 100644
index 0000000..6af43bc
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/exception/MultiIndexWriteFailureException.java
@@ -0,0 +1,46 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index.exception;
+
+import java.util.List;
+
+import org.apache.hadoop.hbase.index.table.HTableInterfaceReference;
+
+/**
+ * Indicate a failure to write to multiple index tables.
+ */
+@SuppressWarnings("serial")
+public class MultiIndexWriteFailureException extends IndexWriteException {
+
+  private List<HTableInterfaceReference> failures;
+
+  /**
+   * @param failures the tables to which the index write did not succeed
+   */
+  public MultiIndexWriteFailureException(List<HTableInterfaceReference> failures) {
+    super("Failed to write to multiple index tables");
+    this.failures = failures;
+
+  }
+
+  public List<HTableInterfaceReference> getFailedTables() {
+    return this.failures;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/exception/SingleIndexWriteFailureException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/exception/SingleIndexWriteFailureException.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/exception/SingleIndexWriteFailureException.java
new file mode 100644
index 0000000..f0fcb4b
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/exception/SingleIndexWriteFailureException.java
@@ -0,0 +1,63 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index.exception;
+
+import java.util.List;
+
+import org.apache.hadoop.hbase.client.Mutation;
+
+/**
+ * Exception thrown if we cannot successfully write to an index table.
+ */
+@SuppressWarnings("serial")
+public class SingleIndexWriteFailureException extends IndexWriteException {
+
+  private String table;
+
+  /**
+   * Cannot reach the index, but not sure of the table or the mutations that caused the failure
+   * @param msg more description of what happened
+   * @param cause original cause
+   */
+  public SingleIndexWriteFailureException(String msg, Throwable cause) {
+    super(msg, cause);
+  }
+
+  /**
+   * Failed to write the passed mutations to an index table for some reason.
+   * @param targetTableName index table to which we attempted to write
+   * @param mutations mutations that were attempted
+   * @param cause underlying reason for the failure
+   */
+  public SingleIndexWriteFailureException(String targetTableName, List<Mutation> mutations,
+      Exception cause) {
+    super("Failed to make index update:\n\t table: " + targetTableName + "\n\t edits: " + mutations
+        + "\n\tcause: " + cause == null ? "UNKNOWN" : cause.getMessage(), cause);
+    this.table = targetTableName;
+  }
+
+  /**
+   * @return The table to which we failed to write the index updates. If unknown, returns
+   *         <tt>null</tt>
+   */
+  public String getTableName() {
+    return this.table;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/BaseTaskRunner.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/BaseTaskRunner.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/BaseTaskRunner.java
new file mode 100644
index 0000000..da6f795
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/BaseTaskRunner.java
@@ -0,0 +1,131 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index.parallel;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.Abortable;
+
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ListeningExecutorService;
+import com.google.common.util.concurrent.MoreExecutors;
+
+/**
+ * {@link TaskRunner} that just manages the underlying thread pool. On called to
+ * {@link #stop(String)}, the thread pool is shutdown immediately - all pending tasks are cancelled
+ * and running tasks receive and interrupt.
+ * <p>
+ * If we find a failure the failure is propagated to the {@link TaskBatch} so any {@link Task} that
+ * is interested can kill itself as well.
+ */
+public abstract class BaseTaskRunner implements TaskRunner {
+
+  private static final Log LOG = LogFactory.getLog(BaseTaskRunner.class);
+  protected ListeningExecutorService writerPool;
+  private boolean stopped;
+
+  public BaseTaskRunner(ExecutorService service) {
+    this.writerPool = MoreExecutors.listeningDecorator(service);
+  }
+
+  @Override
+  public <R> List<R> submit(TaskBatch<R> tasks) throws CancellationException, ExecutionException,
+      InterruptedException {
+    // submit each task to the pool and queue it up to be watched
+    List<ListenableFuture<R>> futures = new ArrayList<ListenableFuture<R>>(tasks.size());
+    for (Task<R> task : tasks.getTasks()) {
+      futures.add(this.writerPool.submit(task));
+    }
+    try {
+      // This logic is actually much more synchronized than the previous logic. Now we rely on a
+      // synchronization around the status to tell us when we are done. While this does have the
+      // advantage of being (1) less code, and (2) supported as part of a library, it is just that
+      // little bit slower. If push comes to shove, we can revert back to the previous
+      // implementation, but for right now, this works just fine.
+      return submitTasks(futures).get();
+    } catch (CancellationException e) {
+      // propagate the failure back out
+      logAndNotifyAbort(e, tasks);
+      throw e;
+    } catch (ExecutionException e) {
+      // propagate the failure back out
+      logAndNotifyAbort(e, tasks);
+      throw e;
+    }
+  }
+
+  private void logAndNotifyAbort(Exception e, Abortable abort) {
+    String msg = "Found a failed task because: " + e.getMessage();
+    LOG.error(msg, e);
+    abort.abort(msg, e.getCause());
+  }
+
+  /**
+   * Build a ListenableFuture for the tasks. Implementing classes can determine return behaviors on
+   * the given tasks
+   * @param futures to wait on
+   * @return a single {@link ListenableFuture} that completes based on the passes tasks.
+   */
+  protected abstract <R> ListenableFuture<List<R>> submitTasks(List<ListenableFuture<R>> futures);
+
+  @Override
+  public <R> List<R> submitUninterruptible(TaskBatch<R> tasks) throws EarlyExitFailure,
+      ExecutionException {
+    boolean interrupted = false;
+    try {
+      while (!this.isStopped()) {
+        try {
+          return this.submit(tasks);
+        } catch (InterruptedException e) {
+          interrupted = true;
+        }
+      }
+    } finally {
+      // restore the interrupted status
+      if (interrupted) {
+        Thread.currentThread().interrupt();
+      }
+    }
+
+    // should only get here if we are interrupted while waiting for a result and have been told to
+    // shutdown by an external source
+    throw new EarlyExitFailure("Interrupted and stopped before computation was complete!");
+  }
+
+  @Override
+  public void stop(String why) {
+    if (this.stopped) {
+      return;
+    }
+    LOG.info("Shutting down task runner because " + why);
+    this.writerPool.shutdownNow();
+  }
+
+  @Override
+  public boolean isStopped() {
+    return this.stopped;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/EarlyExitFailure.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/EarlyExitFailure.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/EarlyExitFailure.java
new file mode 100644
index 0000000..ab2ffd5
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/EarlyExitFailure.java
@@ -0,0 +1,36 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index.parallel;
+
+import java.io.IOException;
+
+/**
+ * Exception denoting a need to early-exit a task (or group of tasks) due to external notification
+ */
+@SuppressWarnings("serial")
+public class EarlyExitFailure extends IOException {
+
+  /**
+   * @param msg reason for the early exit
+   */
+  public EarlyExitFailure(String msg) {
+    super(msg);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/QuickFailingTaskRunner.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/QuickFailingTaskRunner.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/QuickFailingTaskRunner.java
new file mode 100644
index 0000000..6a194a6
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/QuickFailingTaskRunner.java
@@ -0,0 +1,51 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index.parallel;
+
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+
+/**
+ * {@link TaskRunner} that attempts to run all tasks passed, but quits early if any {@link Task}
+ * fails, not waiting for the remaining {@link Task}s to complete.
+ */
+public class QuickFailingTaskRunner extends BaseTaskRunner {
+
+  static final Log LOG = LogFactory.getLog(QuickFailingTaskRunner.class);
+
+  /**
+   * @param service thread pool to which {@link Task}s are submitted. This service is then 'owned'
+   *          by <tt>this</tt> and will be shutdown on calls to {@link #stop(String)}.
+   */
+  public QuickFailingTaskRunner(ExecutorService service) {
+    super(service);
+  }
+
+  @Override
+  protected <R> ListenableFuture<List<R>> submitTasks(List<ListenableFuture<R>> futures) {
+    return Futures.allAsList(futures);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/Task.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/Task.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/Task.java
new file mode 100644
index 0000000..a052759
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/Task.java
@@ -0,0 +1,42 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index.parallel;
+
+import java.util.concurrent.Callable;
+
+import org.apache.hadoop.hbase.Abortable;
+
+/**
+ * Like a {@link Callable}, but supports an internal {@link Abortable} that can be checked
+ * periodically to determine if the batch should abort
+ * @param <V> expected result of the task
+ */
+public abstract class Task<V> implements Callable<V> {
+
+  private Abortable batch;
+
+  void setBatchMonitor(Abortable abort) {
+    this.batch = abort;
+  }
+
+  protected boolean isBatchFailed() {
+    return this.batch.isAborted();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/TaskBatch.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/TaskBatch.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/TaskBatch.java
new file mode 100644
index 0000000..49d96bf
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/TaskBatch.java
@@ -0,0 +1,76 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index.parallel;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.Abortable;
+
+/**
+ * A group of {@link Task}s. The tasks are all bound together using the same {@link Abortable} (
+ * <tt>this</tt>) to ensure that all tasks are aware when any of the other tasks fails.
+ * @param <V> expected result type from all the tasks
+ */
+public class TaskBatch<V> implements Abortable {
+  private static final Log LOG = LogFactory.getLog(TaskBatch.class);
+  private AtomicBoolean aborted = new AtomicBoolean();
+  private List<Task<V>> tasks;
+
+  /**
+   * @param size expected number of tasks
+   */
+  public TaskBatch(int size) {
+    this.tasks = new ArrayList<Task<V>>(size);
+  }
+
+  public void add(Task<V> task) {
+    this.tasks.add(task);
+    task.setBatchMonitor(this);
+  }
+
+  public Collection<Task<V>> getTasks() {
+    return this.tasks;
+  }
+
+  @Override
+  public void abort(String why, Throwable e) {
+    if (this.aborted.getAndSet(true)) {
+      return;
+    }
+    LOG.info("Aborting batch of tasks because " + why);
+  }
+
+  @Override
+  public boolean isAborted() {
+    return this.aborted.get();
+  }
+
+  /**
+   * @return the number of tasks assigned to this batch
+   */
+  public int size() {
+    return this.tasks.size();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/TaskRunner.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/TaskRunner.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/TaskRunner.java
new file mode 100644
index 0000000..bc8749c
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/TaskRunner.java
@@ -0,0 +1,62 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index.parallel;
+
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+
+import org.apache.hadoop.hbase.Stoppable;
+
+/**
+ *
+ */
+public interface TaskRunner extends Stoppable {
+
+  /**
+   * Submit the given tasks to the pool and wait for them to complete. fail.
+   * <p>
+   * Non-interruptible method. To stop any running tasks call {@link #stop(String)} - this will
+   * shutdown the thread pool, causing any pending tasks to be failed early (whose failure will be
+   * ignored) and interrupt any running tasks. It is up to the passed tasks to respect the interrupt
+   * notification
+   * @param tasks to run
+   * @return the result from each task
+   * @throws ExecutionException if any of the tasks fails. Wraps the underyling failure, which can
+   *           be retrieved via {@link ExecutionException#getCause()}.
+   * @throws InterruptedException if the current thread is interrupted while waiting for the batch
+   *           to complete
+   */
+  public <R> List<R> submit(TaskBatch<R> tasks) throws
+      ExecutionException, InterruptedException;
+
+  /**
+   * Similar to {@link #submit(TaskBatch)}, but is not interruptible. If an interrupt is found while
+   * waiting for results, we ignore it and only stop is {@link #stop(String)} has been called. On
+   * return from the method, the interrupt status of the thread is restored.
+   * @param tasks to run
+   * @return the result from each task
+   * @throws EarlyExitFailure if there are still tasks to submit to the pool, but there is a stop
+   *           notification
+   * @throws ExecutionException if any of the tasks fails. Wraps the underyling failure, which can
+   *           be retrieved via {@link ExecutionException#getCause()}.
+   */
+  public <R> List<R> submitUninterruptible(TaskBatch<R> tasks) throws EarlyExitFailure,
+      ExecutionException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/ThreadPoolBuilder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/ThreadPoolBuilder.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/ThreadPoolBuilder.java
new file mode 100644
index 0000000..869ccb8
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/ThreadPoolBuilder.java
@@ -0,0 +1,92 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index.parallel;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.util.Pair;
+
+/**
+ * Helper utility to make a thread pool from a configuration based on reasonable defaults and passed
+ * configuration keys.
+ */
+public class ThreadPoolBuilder {
+
+  private static final Log LOG = LogFactory.getLog(ThreadPoolBuilder.class);
+  private static final long DEFAULT_TIMEOUT = 60;
+  private static final int DEFAULT_MAX_THREADS = 1;// is there a better default?
+  private Pair<String, Long> timeout;
+  private Pair<String, Integer> maxThreads;
+  private String name;
+  private Configuration conf;
+
+  public ThreadPoolBuilder(String poolName, Configuration conf) {
+    this.name = poolName;
+    this.conf = conf;
+  }
+
+  public ThreadPoolBuilder setCoreTimeout(String confkey, long defaultTime) {
+    if (defaultTime <= 0) {
+      defaultTime = DEFAULT_TIMEOUT;
+    }
+    this.timeout = new Pair<String, Long>(confkey, defaultTime);
+    return this;
+  }
+
+  public ThreadPoolBuilder setCoreTimeout(String confKey) {
+    return this.setCoreTimeout(confKey, DEFAULT_TIMEOUT);
+  }
+
+  public ThreadPoolBuilder setMaxThread(String confkey, int defaultThreads) {
+    if (defaultThreads <= 0) {
+      defaultThreads = DEFAULT_MAX_THREADS;
+    }
+    this.maxThreads = new Pair<String, Integer>(confkey, defaultThreads);
+    return this;
+  }
+
+  String getName() {
+   return this.name;
+  }
+
+  int getMaxThreads() {
+    int maxThreads = DEFAULT_MAX_THREADS;
+    if (this.maxThreads != null) {
+      String key = this.maxThreads.getFirst();
+      maxThreads =
+          key == null ? this.maxThreads.getSecond() : conf.getInt(key, this.maxThreads.getSecond());
+    }
+    LOG.trace("Creating pool builder with max " + maxThreads + " threads ");
+    return maxThreads;
+  }
+
+  long getKeepAliveTime() {
+    long timeout =DEFAULT_TIMEOUT;
+    if (this.timeout != null) {
+      String key = this.timeout.getFirst();
+      timeout =
+          key == null ? this.timeout.getSecond() : conf.getLong(key, this.timeout.getSecond());
+    }
+
+    LOG.trace("Creating pool builder with core thread timeout of " + timeout + " seconds ");
+    return timeout;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/ThreadPoolManager.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/ThreadPoolManager.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/ThreadPoolManager.java
new file mode 100644
index 0000000..7a08105
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/ThreadPoolManager.java
@@ -0,0 +1,148 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index.parallel;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.util.Threads;
+
+/**
+ * Manage access to thread pools
+ */
+public class ThreadPoolManager {
+
+  private static final Log LOG = LogFactory.getLog(ThreadPoolManager.class);
+
+  /**
+   * Get an executor for the given name, based on the passed {@link Configuration}. If a thread pool
+   * already exists with that name, it will be returned.
+   * @param builder
+   * @param env
+   * @return a {@link ThreadPoolExecutor} for the given name. Thread pool that only shuts down when
+   *         there are no more explicit references to it. You do not need to shutdown the threadpool
+   *         on your own - it is managed for you. When you are done, you merely need to release your
+   *         reference. If you do attempt to shutdown the pool, you should be careful to call
+   *         {@link ThreadPoolExecutor#shutdown()} XOR {@link ThreadPoolExecutor#shutdownNow()} - extra calls to either can lead to
+   *         early shutdown of the pool.
+   */
+  public static synchronized ThreadPoolExecutor getExecutor(ThreadPoolBuilder builder,
+      RegionCoprocessorEnvironment env) {
+    return getExecutor(builder, env.getSharedData());
+  }
+
+  static synchronized ThreadPoolExecutor getExecutor(ThreadPoolBuilder builder,
+      Map<String, Object> poolCache) {
+    ThreadPoolExecutor pool = (ThreadPoolExecutor) poolCache.get(builder.getName());
+    if (pool == null || pool.isTerminating() || pool.isShutdown()) {
+      pool = getDefaultExecutor(builder);
+      LOG.info("Creating new pool for " + builder.getName());
+      poolCache.put(builder.getName(), pool);
+    }
+    ((ShutdownOnUnusedThreadPoolExecutor) pool).addReference();
+
+    return pool;
+  }
+
+  /**
+   * @param conf
+   * @return
+   */
+  private static ShutdownOnUnusedThreadPoolExecutor getDefaultExecutor(ThreadPoolBuilder builder) {
+    int maxThreads = builder.getMaxThreads();
+    long keepAliveTime = builder.getKeepAliveTime();
+
+    // we prefer starting a new thread to queuing (the opposite of the usual ThreadPoolExecutor)
+    // since we are probably writing to a bunch of index tables in this case. Any pending requests
+    // are then queued up in an infinite (Integer.MAX_VALUE) queue. However, we allow core threads
+    // to timeout, to we tune up/down for bursty situations. We could be a bit smarter and more
+    // closely manage the core-thread pool size to handle the bursty traffic (so we can always keep
+    // some core threads on hand, rather than starting from scratch each time), but that would take
+    // even more time. If we shutdown the pool, but are still putting new tasks, we can just do the
+    // usual policy and throw a RejectedExecutionException because we are shutting down anyways and
+    // the worst thing is that this gets unloaded.
+    ShutdownOnUnusedThreadPoolExecutor pool =
+        new ShutdownOnUnusedThreadPoolExecutor(maxThreads, maxThreads, keepAliveTime,
+            TimeUnit.SECONDS, new LinkedBlockingQueue<Runnable>(),
+            Threads.newDaemonThreadFactory(builder.getName() + "-"), builder.getName());
+    pool.allowCoreThreadTimeOut(true);
+    return pool;
+  }
+
+  /**
+   * Thread pool that only shuts down when there are no more explicit references to it. A reference
+   * is when obtained and released on calls to {@link #shutdown()} or {@link #shutdownNow()}.
+   * Therefore, users should be careful to call {@link #shutdown()} XOR {@link #shutdownNow()} -
+   * extra calls to either can lead to early shutdown of the pool.
+   */
+  private static class ShutdownOnUnusedThreadPoolExecutor extends ThreadPoolExecutor {
+
+    private AtomicInteger references;
+    private String poolName;
+
+    public ShutdownOnUnusedThreadPoolExecutor(int coreThreads, int maxThreads, long keepAliveTime,
+        TimeUnit timeUnit, BlockingQueue<Runnable> workQueue, ThreadFactory threadFactory,
+        String poolName) {
+      super(coreThreads, maxThreads, keepAliveTime, timeUnit, workQueue, threadFactory);
+      this.references = new AtomicInteger();
+      this.poolName = poolName;
+    }
+
+    public void addReference() {
+      this.references.incrementAndGet();
+    }
+
+    @Override
+    protected void finalize() {
+      // override references counter if we go out of scope - ensures the pool gets cleaned up
+      LOG.info("Shutting down pool '" + poolName + "' because no more references");
+      super.finalize();
+    }
+
+    @Override
+    public void shutdown() {
+      if (references.decrementAndGet() <= 0) {
+        LOG.debug("Shutting down pool " + this.poolName);
+        super.shutdown();
+      }
+    }
+
+    @Override
+    public List<Runnable> shutdownNow() {
+      if (references.decrementAndGet() <= 0) {
+        LOG.debug("Shutting down pool " + this.poolName + " NOW!");
+        return super.shutdownNow();
+      }
+      return Collections.emptyList();
+    }
+
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/WaitForCompletionTaskRunner.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/WaitForCompletionTaskRunner.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/WaitForCompletionTaskRunner.java
new file mode 100644
index 0000000..fe9f3ac
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/WaitForCompletionTaskRunner.java
@@ -0,0 +1,53 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index.parallel;
+
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+
+/**
+ * A {@link TaskRunner} that ensures that all the tasks have been attempted before we return, even
+ * if some of the tasks cause failures.
+ * <p>
+ * Because we wait until the entire batch is complete to see the failure, checking for failure of
+ * the {@link TaskBatch} on the submitted tasks will not help - they will never see the failure of
+ * the other tasks. You will need to provide an external mechanism to propagate the error.
+ * <p>
+ * Does not throw an {@link ExecutionException} if any of the tasks fail.
+ */
+public class WaitForCompletionTaskRunner extends BaseTaskRunner {
+  
+  /**
+   * @param service thread pool to which {@link Task}s are submitted. This service is then 'owned'
+   *          by <tt>this</tt> and will be shutdown on calls to {@link #stop(String)}.
+   */
+  public WaitForCompletionTaskRunner(ExecutorService service) {
+    super(service);
+  }
+
+  @Override
+  public <R> ListenableFuture<List<R>> submitTasks(List<ListenableFuture<R>> futures) {
+    return Futures.successfulAsList(futures);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/scanner/EmptyScanner.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/scanner/EmptyScanner.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/scanner/EmptyScanner.java
new file mode 100644
index 0000000..491a895
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/scanner/EmptyScanner.java
@@ -0,0 +1,32 @@
+package org.apache.hadoop.hbase.index.scanner;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.KeyValue;
+
+
+/**
+ * {@link Scanner} that has no underlying data
+ */
+public class EmptyScanner implements Scanner {
+
+  @Override
+  public KeyValue next() throws IOException {
+    return null;
+  }
+
+  @Override
+  public boolean seek(KeyValue next) throws IOException {
+    return false;
+  }
+
+  @Override
+  public KeyValue peek() throws IOException {
+    return null;
+  }
+
+  @Override
+  public void close() throws IOException {
+    // noop
+  }
+}
\ No newline at end of file


[07/51] [partial] Initial commit of master branch from github

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/ValueBitSet.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/ValueBitSet.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/ValueBitSet.java
new file mode 100644
index 0000000..2b1937b
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/ValueBitSet.java
@@ -0,0 +1,200 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.util.Arrays;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import org.apache.phoenix.util.SizedUtil;
+
+
+/**
+ * 
+ * Class to track whether or not a value is null.
+ * The value is a zero-based position in the schema provided.
+ *
+ * @author jtaylor
+ * @since 0.1
+ * 
+ */
+public class ValueBitSet {
+    public final static ValueBitSet EMPTY_VALUE_BITSET = new ValueBitSet();
+    private static final int BITS_PER_LONG = 64;
+    private static final int BITS_PER_SHORT = 16;
+    private final long[] bits;
+    private final ValueSchema schema;
+    
+    private int maxSetBit = -1;
+    
+    public static ValueBitSet newInstance(ValueSchema schema) {
+        if (schema.getFieldCount() == schema.getMinNullable()) {
+            return EMPTY_VALUE_BITSET;
+        }
+        return new ValueBitSet(schema);
+    }
+    
+    private ValueBitSet() {
+        schema = null;
+        bits = new long[0];
+    }
+    
+    private ValueBitSet(ValueSchema schema) {
+        this.schema = schema;
+        bits = new long[Math.max(1,(schema.getFieldCount() - schema.getMinNullable() + BITS_PER_LONG -1) / BITS_PER_LONG)];
+    }
+    
+    public int getMaxSetBit() {
+        return maxSetBit;
+    }
+    
+    private boolean isVarLength() {
+        return schema == null ? false : schema.getFieldCount() - schema.getMinNullable() > BITS_PER_SHORT;
+    }
+    
+    public int getNullCount(int nBit, int nFields) {
+        if (schema == null) {
+            return 0;
+        }
+        int count = 0;
+        int index = nBit/BITS_PER_LONG;
+        // Shift right based on the bit index, because we aren't interested in the bits before this.
+        int shiftRight = nBit % BITS_PER_LONG;
+        int bitsToLeft = BITS_PER_LONG - shiftRight;
+        // Shift left based on the number of fields we're interested in counting.
+        int shiftLeft = Math.max(0, (BITS_PER_LONG - nFields));
+        // Mask off the bits of interest by shifting the bitset.
+        count += Math.min(nFields, bitsToLeft) - (Long.bitCount((bits[index] >>> shiftRight) << shiftLeft));
+        // Subtract from the number of fields the total number of possible fields we looked at
+        nFields -= bitsToLeft;
+        if (nFields > 0) {
+            // If more fields to count, then walk through the successive long bits
+            while (nFields > BITS_PER_LONG) {
+                count += BITS_PER_LONG - Long.bitCount(bits[++index]);
+                nFields -= BITS_PER_LONG;
+            }
+            // Count the final remaining fields
+            if (nFields > 0) {
+                count += nFields - Long.bitCount(bits[++index] << (BITS_PER_LONG - nFields));
+            }
+        }
+        return count;
+    }
+    
+    /**
+     * Serialize the value bit set into a byte array. The byte array
+     * is expected to have enough room (use {@link #getEstimatedLength()}
+     * to ensure enough room exists.
+     * @param b the byte array into which to put the serialized bit set
+     * @param offset the offset into the byte array
+     * @return the incremented offset
+     */
+    public int toBytes(byte[] b, int offset) {
+        if (schema == null) {
+            return offset;
+        }
+        // If the total number of possible values is bigger than 16 bits (the
+        // size of a short), then serialize the long array followed by the
+        // array length.
+        if (isVarLength()) {
+            short nLongs = (short)((maxSetBit + BITS_PER_LONG - 1) / BITS_PER_LONG);
+            for (int i = 0; i < nLongs; i++) {
+                offset = Bytes.putLong(b, offset, bits[i]);
+            }
+            offset = Bytes.putShort(b, offset, nLongs);
+        } else { 
+            // Else if the number of values is less than or equal to 16,
+            // serialize the bits directly into a short.
+            offset = Bytes.putShort(b, offset, (short)bits[0]);            
+        }
+        return offset;
+    }
+    
+    public void clear() {
+        Arrays.fill(bits, 0);
+        maxSetBit = -1;
+    }
+    
+    public boolean get(int nBit) {
+        int lIndex = nBit / BITS_PER_LONG;
+        int bIndex = nBit % BITS_PER_LONG;
+        return (bits[lIndex] & (1L << bIndex)) != 0;
+    }
+    
+    public void set(int nBit) {
+        int lIndex = nBit / BITS_PER_LONG;
+        int bIndex = nBit % BITS_PER_LONG;
+        bits[lIndex] |= (1L << bIndex);
+        maxSetBit = Math.max(maxSetBit, nBit);
+    }
+    
+    public void or(ImmutableBytesWritable ptr) {
+        if (schema == null) {
+            return;
+        }
+        if (isVarLength()) {
+            int offset = ptr.getOffset() + ptr.getLength() - Bytes.SIZEOF_SHORT;
+            short nLongs = Bytes.toShort(ptr.get(), offset);
+            offset -= nLongs * Bytes.SIZEOF_LONG;
+            for (int i = 0; i < nLongs; i++) {
+                bits[i] |= Bytes.toLong(ptr.get(), offset);
+                offset += Bytes.SIZEOF_LONG;
+            }
+            maxSetBit = Math.max(maxSetBit, nLongs * Bytes.SIZEOF_LONG - 1);
+        } else {
+            long l = Bytes.toShort(ptr.get(), ptr.getOffset() + ptr.getLength() - Bytes.SIZEOF_SHORT);
+            bits[0] |= l;
+            maxSetBit = Math.max(maxSetBit, BITS_PER_SHORT - 1);
+        }
+        
+    }
+    
+    /**
+     * @return Max serialization size
+     */
+    public int getEstimatedLength() {
+        if (schema == null) {
+            return 0;
+        }
+        return Bytes.SIZEOF_SHORT + (isVarLength() ? (maxSetBit + BITS_PER_LONG - 1) / BITS_PER_LONG * Bytes.SIZEOF_LONG : 0);
+    }
+    
+    public static int getSize(int nBits) {
+        return SizedUtil.OBJECT_SIZE + SizedUtil.POINTER_SIZE + SizedUtil.ARRAY_SIZE + SizedUtil.INT_SIZE + (nBits + BITS_PER_LONG - 1) / BITS_PER_LONG * Bytes.SIZEOF_LONG;
+    }
+    
+    /**
+     * @return Size of object in memory
+     */
+    public int getSize() {
+        if (schema == null) {
+            return 0;
+        }
+        return SizedUtil.OBJECT_SIZE + SizedUtil.POINTER_SIZE + SizedUtil.ARRAY_SIZE + SizedUtil.LONG_SIZE * bits.length + SizedUtil.INT_SIZE;
+    }
+
+    public void or(ValueBitSet isSet) {
+        for (int i = 0; i < bits.length; i++) {
+            bits[i] |= isSet.bits[i];
+        }
+        maxSetBit = Math.max(maxSetBit, isSet.maxSetBit);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/ValueRangeExcpetion.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/ValueRangeExcpetion.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/ValueRangeExcpetion.java
new file mode 100644
index 0000000..aff2ef7
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/ValueRangeExcpetion.java
@@ -0,0 +1,40 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.sql.SQLException;
+
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+
+/**
+ * Exception thrown when we try to use use an argument that has the wrong type. 
+ * 
+ * @author anoopsjohn
+ * @since 1.1.2
+ */
+public class ValueRangeExcpetion extends SQLException{
+    private static final long serialVersionUID = 1L;
+    private static SQLExceptionCode code = SQLExceptionCode.VALUE_OUTSIDE_RANGE;
+    
+    public ValueRangeExcpetion(Object minValue, Object maxValue, Object actualValue, String location){
+        super(new SQLExceptionInfo.Builder(code).setMessage("expected: [" + minValue + " , " + maxValue + "] but was: " + actualValue + " at " + location).build().toString(), code.getSQLState(), code.getErrorCode());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/ValueSchema.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/ValueSchema.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/ValueSchema.java
new file mode 100644
index 0000000..41daf2a
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/ValueSchema.java
@@ -0,0 +1,337 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableUtils;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+
+/**
+ * 
+ * Simple flat schema over a byte array where fields may be any of {@link PDataType}.
+ * Optimized for positional access by index.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public abstract class ValueSchema implements Writable {
+    public static final int ESTIMATED_VARIABLE_LENGTH_SIZE = 10;
+    private int[] fieldIndexByPosition;
+    private List<Field> fields;
+    private int estimatedLength;
+    private boolean isFixedLength;
+    private boolean isMaxLength;
+    private int minNullable;
+    
+    public ValueSchema() {
+    }
+    
+    protected ValueSchema(int minNullable, List<Field> fields) {
+        init(minNullable, fields);
+    }
+
+    private void init(int minNullable, List<Field> fields) {
+        this.minNullable = minNullable;
+        this.fields = ImmutableList.copyOf(fields);
+        int estimatedLength = 0;
+        boolean isMaxLength = true, isFixedLength = true;
+        int positions = 0;
+        for (Field field : fields) {
+            int fieldEstLength = 0;
+            PDataType type = field.getDataType();
+            Integer byteSize = type.getByteSize();
+            if (type.isFixedWidth()) {
+                fieldEstLength += field.getByteSize();
+            } else {
+                isFixedLength = false;
+                // Account for vint for length if not fixed
+                if (byteSize == null) {
+                    isMaxLength = false;
+                    fieldEstLength += ESTIMATED_VARIABLE_LENGTH_SIZE;
+                } else {
+                    fieldEstLength += WritableUtils.getVIntSize(byteSize);
+                    fieldEstLength = byteSize;
+                }
+            }
+            positions += field.getCount();
+            estimatedLength += fieldEstLength * field.getCount();
+        }
+        fieldIndexByPosition = new int[positions];
+        for (int i = 0, j= 0; i < fields.size(); i++) {
+            Field field = fields.get(i);
+            Arrays.fill(fieldIndexByPosition, j, j + field.getCount(), i);
+            j += field.getCount();
+        }
+        this.isFixedLength = isFixedLength;
+        this.isMaxLength = isMaxLength;
+        this.estimatedLength = estimatedLength;
+    }
+    
+    public int getFieldCount() {
+        return fieldIndexByPosition.length;
+    }
+    
+    public List<Field> getFields() {
+        return fields;
+    }
+    
+    /**
+     * @return true if all types are fixed width
+     */
+    public boolean isFixedLength() {
+        return isFixedLength;
+    }
+    
+    /**
+     * @return true if {@link #getEstimatedValueLength()} returns the maximum length
+     * of a serialized value for this schema
+     */
+    public boolean isMaxLength() {
+        return isMaxLength;
+    }
+    
+    /**
+     * @return estimated size in bytes of a serialized value for this schema
+     */
+    public int getEstimatedValueLength() {
+        return estimatedLength;
+    }
+    
+    /**
+     * Non-nullable fields packed to the left so that we do not need to store trailing nulls.
+     * Knowing the minimum position of a nullable field enables this.
+     * @return the minimum position of a nullable field
+     */
+    public int getMinNullable() {
+        return minNullable;
+    }
+    
+    public static final class Field implements Writable {
+        @Override
+        public int hashCode() {
+            final int prime = 31;
+            int result = 1;
+            result = prime * result + byteSize;
+            result = prime * result + type.hashCode();
+            result = prime * result + ((columnModifier == null) ? 0 : columnModifier.hashCode());
+            result = prime * result + (isNullable ? 1231 : 1237);
+            return result;
+        }
+
+        @Override
+        public boolean equals(Object obj) {
+            if (this == obj) return true;
+            if (obj == null) return false;
+            if (getClass() != obj.getClass()) return false;
+            Field other = (Field)obj;
+            if (byteSize != other.byteSize) return false;
+            if (columnModifier != other.columnModifier) return false;
+            if (isNullable != other.isNullable) return false;
+            if (type != other.type) return false;
+            return true;
+        }
+
+        private int count;
+        private PDataType type;
+        private int byteSize = 0;
+        private boolean isNullable;
+        private ColumnModifier columnModifier;
+        
+        public Field() {
+        }
+        
+        private Field(PDatum datum, boolean isNullable, int count, ColumnModifier columnModifier) {
+            this.type = datum.getDataType();
+            this.columnModifier = columnModifier;
+            this.count = count;
+            this.isNullable = isNullable;
+            if (this.type.isFixedWidth() && this.type.getByteSize() == null) {
+                this.byteSize = datum.getByteSize();
+            }
+        }
+        
+        private Field(Field field, int count) {
+            this.type = field.getDataType();
+            this.byteSize = field.byteSize;
+            this.count = count;
+        }
+        
+        public final ColumnModifier getColumnModifier() {
+            return columnModifier;
+        }
+        
+        public final PDataType getDataType() {
+            return type;
+        }
+        
+        public final boolean isNullable() {
+            return isNullable;
+        }
+        
+        public final int getByteSize() {
+            return type.getByteSize() == null ? byteSize : type.getByteSize();
+        }
+        
+        public final int getCount() {
+            return count;
+        }
+
+        @Override
+        public void readFields(DataInput input) throws IOException {
+            // Encode isNullable in sign bit of type ordinal (offset by 1, since ordinal could be 0)
+            int typeOrdinal = WritableUtils.readVInt(input);
+            if (typeOrdinal < 0) {
+                typeOrdinal *= -1;
+                this.isNullable = true;
+            }
+            this.type = PDataType.values()[typeOrdinal-1];
+            this.count = WritableUtils.readVInt(input);
+            if (this.count < 0) {
+                this.count *= -1;
+                this.columnModifier = ColumnModifier.SORT_DESC;
+            }
+            if (this.type.isFixedWidth() && this.type.getByteSize() == null) {
+                this.byteSize = WritableUtils.readVInt(input);
+            }
+        }
+
+        @Override
+        public void write(DataOutput output) throws IOException {
+            WritableUtils.writeVInt(output, (type.ordinal() + 1) * (this.isNullable ? -1 : 1));
+            WritableUtils.writeVInt(output, count * (columnModifier == null ? 1 : -1));
+            if (type.isFixedWidth() && type.getByteSize() == null) {
+                WritableUtils.writeVInt(output, byteSize);
+            }
+        }
+    }
+    
+    public abstract static class ValueSchemaBuilder {
+        protected List<Field> fields = new ArrayList<Field>();
+        protected int nFields = Integer.MAX_VALUE;
+        protected final int minNullable;
+        
+        public ValueSchemaBuilder(int minNullable) {
+            this.minNullable = minNullable;
+        }
+        
+        protected List<Field> buildFields() {
+            List<Field> condensedFields = new ArrayList<Field>(fields.size());
+            for (int i = 0; i < Math.min(nFields,fields.size()); ) {
+                Field field = fields.get(i);
+                int count = 1;
+                while ( ++i < fields.size() && field.equals(fields.get(i))) {
+                    count++;
+                }
+                condensedFields.add(count == 1 ? field : new Field(field,count));
+            }
+            return condensedFields;
+        }
+
+        abstract public ValueSchema build();
+
+        public ValueSchemaBuilder setMaxFields(int nFields) {
+            this.nFields = nFields;
+            return this;
+        }
+        
+        protected ValueSchemaBuilder addField(PDatum datum, boolean isNullable, ColumnModifier columnModifier) {
+            fields.add(new Field(datum, isNullable, 1, columnModifier));
+            return this;
+        }
+    }
+    
+    public int getEstimatedByteSize() {
+        int size = 0;
+        size += WritableUtils.getVIntSize(minNullable);
+        size += WritableUtils.getVIntSize(fields.size());
+        size += fields.size() * 3;
+        return size;
+    }
+    
+    public void serialize(DataOutput output) throws IOException {
+        WritableUtils.writeVInt(output, minNullable);
+        WritableUtils.writeVInt(output, fields.size());
+        for (int i = 0; i < fields.size(); i++) {
+            fields.get(i).write(output);
+        }
+    }
+    
+    public Field getField(int position) {
+        return fields.get(fieldIndexByPosition[position]);
+    }
+    
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + minNullable;
+        for (Field field : fields) {
+        	result = prime * result + field.hashCode();
+        }
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) return true;
+        if (obj == null) return false;
+        if (getClass() != obj.getClass()) return false;
+        ValueSchema other = (ValueSchema)obj;
+        if (minNullable != other.minNullable) return false;
+        if (fields.size() != other.fields.size()) return false;
+        for (int i = 0; i < fields.size(); i++) {
+        	if (!fields.get(i).equals(other.fields.get(i)))
+        		return false;
+        }
+        return true;
+    }
+    
+    @Override
+    public void readFields(DataInput in) throws IOException {
+        int minNullable = WritableUtils.readVInt(in);
+        int nFields = WritableUtils.readVInt(in);
+        List<Field> fields = Lists.newArrayListWithExpectedSize(nFields);
+        for (int i = 0; i < nFields; i++) {
+            Field field = new Field();
+            field.readFields(in);
+            fields.add(field);
+        }
+        init(minNullable, fields);
+    }
+         
+    @Override
+    public void write(DataOutput out) throws IOException {
+        WritableUtils.writeVInt(out, minNullable);
+        WritableUtils.writeVInt(out, fields.size());
+        for (int i = 0; i < fields.size(); i++) {
+            fields.get(i).write(out);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/stat/PTableStats.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/stat/PTableStats.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/stat/PTableStats.java
new file mode 100644
index 0000000..5e38a27
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/stat/PTableStats.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.stat;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.HRegionInfo;
+
+
+/**
+ * Interface for Phoenix table statistics. Statistics is collected on the server
+ * side and can be used for various purpose like splitting region for scanning, etc.
+ * 
+ * The table is defined on the client side, but it is populated on the server side. The client
+ * should not populate any data to the statistics object.
+ */
+public interface PTableStats {
+
+    /**
+     * Given the region info, returns an array of bytes that is the current estimate of key
+     * distribution inside that region. The keys should split that region into equal chunks.
+     * 
+     * @param region
+     * @return array of keys
+     */
+    byte[][] getRegionGuidePosts(HRegionInfo region);
+
+    void write(DataOutput output) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/stat/PTableStatsImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/stat/PTableStatsImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/stat/PTableStatsImpl.java
new file mode 100644
index 0000000..27bb665
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/stat/PTableStatsImpl.java
@@ -0,0 +1,70 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.stat;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.WritableUtils;
+
+import com.google.common.collect.ImmutableMap;
+
+
+/**
+ * Implementation for PTableStats.
+ */
+public class PTableStatsImpl implements PTableStats {
+
+    // The map for guide posts should be immutable. We only take the current snapshot from outside
+    // method call and store it.
+    private Map<String, byte[][]> regionGuidePosts;
+
+    public PTableStatsImpl() { }
+
+    public PTableStatsImpl(Map<String, byte[][]> stats) {
+        regionGuidePosts = ImmutableMap.copyOf(stats);
+    }
+
+    @Override
+    public byte[][] getRegionGuidePosts(HRegionInfo region) {
+        return regionGuidePosts.get(region.getRegionNameAsString());
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+        if (regionGuidePosts == null) {
+            WritableUtils.writeVInt(output, 0);
+            return;
+        }
+        WritableUtils.writeVInt(output, regionGuidePosts.size());
+        for (Entry<String, byte[][]> entry : regionGuidePosts.entrySet()) {
+            WritableUtils.writeString(output, entry.getKey());
+            byte[][] value = entry.getValue();
+            WritableUtils.writeVInt(output, value.length);
+            for (int i=0; i<value.length; i++) {
+                Bytes.writeByteArray(output, value[i]);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..ed69560
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/MultiKeyValueTuple.java
@@ -0,0 +1,85 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.util.List;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.phoenix.util.KeyValueUtil;
+
+
+public class MultiKeyValueTuple implements Tuple {
+    private List<KeyValue> values;
+    
+    public MultiKeyValueTuple(List<KeyValue> values) {
+        setKeyValues(values);
+    }
+    
+    public MultiKeyValueTuple() {
+    }
+
+    public void setKeyValues(List<KeyValue> values) {
+        this.values = ImmutableList.copyOf(values);
+    }
+    
+    @Override
+    public void getKey(ImmutableBytesWritable ptr) {
+        KeyValue value = values.get(0);
+        ptr.set(value.getBuffer(), value.getRowOffset(), value.getRowLength());
+    }
+
+    @Override
+    public boolean isImmutable() {
+        return true;
+    }
+
+    @Override
+    public KeyValue getValue(byte[] family, byte[] qualifier) {
+        return KeyValueUtil.getColumnLatest(values, family, qualifier);
+    }
+
+    @Override
+    public String toString() {
+        return values.toString();
+    }
+
+    @Override
+    public int size() {
+        return values.size();
+    }
+
+    @Override
+    public KeyValue getValue(int index) {
+        return values.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.getBuffer(), kv.getValueOffset(), kv.getValueLength());
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..c99a990
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/ResultTuple.java
@@ -0,0 +1,104 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import org.apache.phoenix.util.ResultUtil;
+
+
+public class ResultTuple implements Tuple {
+    private Result result;
+    
+    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) {
+        ResultUtil.getKey(result, ptr);
+    }
+
+    @Override
+    public boolean isImmutable() {
+        return true;
+    }
+
+    @Override
+    public KeyValue getValue(byte[] family, byte[] qualifier) {
+        return result.getColumnLatest(family, qualifier);
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder();
+      sb.append("keyvalues=");
+      if(this.result.isEmpty()) {
+        sb.append("NONE");
+        return sb.toString();
+      }
+      sb.append("{");
+      boolean moreThanOne = false;
+      for(KeyValue kv : this.result.list()) {
+        if(moreThanOne) {
+          sb.append(", \n");
+        } else {
+          moreThanOne = true;
+        }
+        sb.append(kv.toString()+"/value="+Bytes.toString(kv.getValue()));
+      }
+      sb.append("}\n");
+      return sb.toString();
+    }
+
+    @Override
+    public int size() {
+        return result.size();
+    }
+
+    @Override
+    public KeyValue getValue(int index) {
+        return result.raw()[index];
+    }
+
+    @Override
+    public boolean getValue(byte[] family, byte[] qualifier,
+            ImmutableBytesWritable ptr) {
+        KeyValue kv = getValue(family, qualifier);
+        if (kv == null)
+            return false;
+        ptr.set(kv.getBuffer(), kv.getValueOffset(), kv.getValueLength());
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/SingleKeyValueTuple.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/SingleKeyValueTuple.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/SingleKeyValueTuple.java
new file mode 100644
index 0000000..5d76604
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/SingleKeyValueTuple.java
@@ -0,0 +1,111 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+
+
+public class SingleKeyValueTuple implements Tuple {
+    private static final byte[] UNITIALIZED_KEY_BUFFER = new byte[0];
+    private KeyValue keyValue;
+    private ImmutableBytesWritable keyPtr = new ImmutableBytesWritable(UNITIALIZED_KEY_BUFFER);
+    
+    public SingleKeyValueTuple() {
+    }
+    
+    public SingleKeyValueTuple(KeyValue keyValue) {
+        if (keyValue == null) {
+            throw new NullPointerException();
+        }
+        setKeyValue(keyValue);
+    }
+    
+    public boolean hasKey() {
+        return keyPtr.get() != UNITIALIZED_KEY_BUFFER;
+    }
+    
+    public void reset() {
+        this.keyValue = null;
+        keyPtr.set(UNITIALIZED_KEY_BUFFER);
+    }
+    
+    public void setKeyValue(KeyValue keyValue) {
+        if (keyValue == null) {
+            throw new IllegalArgumentException();
+        }
+        this.keyValue = keyValue;
+        setKey(keyValue);
+    }
+    
+    public void setKey(ImmutableBytesWritable ptr) {
+        keyPtr.set(ptr.get(), ptr.getOffset(), ptr.getLength());
+    }
+    
+    public void setKey(KeyValue keyValue) {
+        if (keyValue == null) {
+            throw new IllegalArgumentException();
+        }
+        keyPtr.set(keyValue.getBuffer(), keyValue.getRowOffset(), keyValue.getRowLength());
+    }
+    
+    @Override
+    public void getKey(ImmutableBytesWritable ptr) {
+        ptr.set(keyPtr.get(), keyPtr.getOffset(), keyPtr.getLength());
+    }
+    
+    @Override
+    public KeyValue getValue(byte[] cf, byte[] cq) {
+        return keyValue;
+    }
+
+    @Override
+    public boolean isImmutable() {
+        return true;
+    }
+    
+    @Override
+    public String toString() {
+        return "SingleKeyValueTuple[" + keyValue == null ? keyPtr.get() == UNITIALIZED_KEY_BUFFER ? "null" : Bytes.toStringBinary(keyPtr.get(),keyPtr.getOffset(),keyPtr.getLength()) : keyValue.toString() + "]";
+    }
+
+    @Override
+    public int size() {
+        return keyValue == null ? 0 : 1;
+    }
+
+    @Override
+    public KeyValue getValue(int index) {
+        if (index != 0 || keyValue == null) {
+            throw new IndexOutOfBoundsException(Integer.toString(index));
+        }
+        return keyValue;
+    }
+
+    @Override
+    public boolean getValue(byte[] family, byte[] qualifier,
+            ImmutableBytesWritable ptr) {
+        if (keyValue == null)
+            return false;
+        ptr.set(keyValue.getBuffer(), keyValue.getValueOffset(), keyValue.getValueLength());
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..ffa3f72
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/Tuple.java
@@ -0,0 +1,84 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+/**
+ * 
+ * Interface representing an ordered list of KeyValues returned as the
+ * result of a query. Each tuple represents a row (i.e. all its KeyValues
+ * will have the same key), and each KeyValue represents a column value.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public interface Tuple {
+    /**
+     * @return Number of KeyValues contained by the Tuple.
+     */
+    public int size();
+    
+    /**
+     * Determines whether or not the Tuple is immutable (the typical case)
+     * or will potentially have additional KeyValues added to it (the case
+     * during filter evaluation when we see one KeyValue at a time).
+     * @return true if Tuple is immutable and false otherwise.
+     */
+    public boolean isImmutable();
+    
+    /**
+     * Get the row key for the Tuple
+     * @param ptr the bytes pointer that will be updated to point to
+     * the key buffer.
+     */
+    public void getKey(ImmutableBytesWritable ptr);
+    
+    /**
+     * Get the KeyValue at the given index.
+     * @param index the zero-based KeyValue index between 0 and {@link #size()} exclusive
+     * @return the KeyValue at the given index
+     * @throws IndexOutOfBoundsException if an invalid index is used
+     */
+    public KeyValue getValue(int index);
+    
+    /***
+     * Get the KeyValue contained by the Tuple with the given family and
+     * qualifier name.
+     * @param family the column family of the KeyValue being retrieved
+     * @param qualifier the column qualify of the KeyValue being retrieved
+     * @return the KeyValue with the given family and qualifier name or
+     * null if not found.
+     */
+    public KeyValue getValue(byte [] family, byte [] qualifier);
+    
+    /***
+     * Get the value byte array of the KeyValue contained by the Tuple with 
+     * the given family and qualifier name.
+     * @param family the column family of the KeyValue being retrieved
+     * @param qualifier the column qualify of the KeyValue being retrieved
+     * @param ptr the bytes pointer that will be updated to point to the 
+     * value buffer.
+     * @return true if the KeyValue with the given family and qualifier name
+     * exists; otherwise false.
+     */
+    public boolean getValue(byte [] family, byte [] qualifier, ImmutableBytesWritable ptr);
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/util/BigDecimalUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/BigDecimalUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/BigDecimalUtil.java
new file mode 100644
index 0000000..4b49f0f
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/BigDecimalUtil.java
@@ -0,0 +1,70 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 org.apache.hadoop.hbase.util.Pair;
+
+/**
+ * 
+ * @author anoopsjohn
+ * @since 1.2.1
+ */
+public class BigDecimalUtil {
+
+    /**
+     * Calculates the precision and scale for BigDecimal arithmetic operation results. It uses the algorithm mentioned
+     * <a href="http://db.apache.org/derby/docs/10.0/manuals/reference/sqlj124.html#HDRSII-SQLJ-36146">here</a>
+     * @param lp precision of the left operand
+     * @param ls scale of the left operand
+     * @param rp precision of the right operand
+     * @param rs scale of the right operand
+     * @param op The operation type
+     * @return {@link Pair} comprising of the precision and scale.
+     */
+    public static Pair<Integer, Integer> getResultPrecisionScale(int lp, int ls, int rp, int rs, Operation op) {
+        int resultPrec = 0, resultScale = 0;
+        switch (op) {
+        case MULTIPLY:
+            resultPrec = lp + rp;
+            resultScale = ls + rs;
+            break;
+        case DIVIDE:
+            resultPrec = lp - ls + rp + Math.max(ls + rp - rs + 1, 4);
+            resultScale = 31 - lp + ls - rs;
+            break;
+        case ADD:
+            resultPrec = 2 * (lp - ls) + ls; // Is this correct? The page says addition -> 2 * (p - s) + s.
+            resultScale = Math.max(ls, rs);
+            break;
+        case AVG:
+            resultPrec = Math.max(lp - ls, rp - rs) + 1 + Math.max(ls, rs);
+            resultScale = Math.max(Math.max(ls, rs), 4);
+            break;
+        case OTHERS:
+            resultPrec = Math.max(lp - ls, rp - rs) + 1 + Math.max(ls, rs);
+            resultScale = Math.max(ls, rs);
+        }
+        return new Pair<Integer, Integer>(resultPrec, resultScale);
+    }
+    
+    public static enum Operation {
+        MULTIPLY, DIVIDE, ADD, AVG, OTHERS;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/util/BitSet.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/BitSet.java b/phoenix-core/src/main/java/org/apache/phoenix/util/BitSet.java
new file mode 100644
index 0000000..7cdbe6a
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/BitSet.java
@@ -0,0 +1,106 @@
+package org.apache.phoenix.util;
+
+import java.io.*;
+
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * 
+ * BitSet that can be initialized with primitive types, which
+ * is only available in Java 7 or above.
+ *
+ * @author jtaylor
+ * @since 2.1.0
+ */
+public class BitSet {
+    public static final int BITS_PER_LONG = 64;
+    public static final int BITS_PER_INT = 32;
+    public static final int BITS_PER_SHORT = 16;
+    public static final int BITS_PER_BYTE = 8;
+    private final long[] bits;
+    
+    public static int getByteSize(int capacity) {
+        if (capacity <= BitSet.BITS_PER_BYTE) {
+            return Bytes.SIZEOF_BYTE;
+        } else if (capacity <= BitSet.BITS_PER_SHORT) {
+            return Bytes.SIZEOF_SHORT;
+        } else if (capacity <= BitSet.BITS_PER_INT) {
+            return Bytes.SIZEOF_INT;
+        } else if (capacity <= BitSet.BITS_PER_LONG) {
+            return Bytes.SIZEOF_LONG;
+        } else {
+            int nLongs = (capacity-1) / BitSet.BITS_PER_LONG + 1;
+            return nLongs * Bytes.SIZEOF_LONG;
+        }
+    }
+
+    public static BitSet read(DataInput input, int capacity) throws IOException {
+        if (capacity <= BitSet.BITS_PER_BYTE) {
+            return fromPrimitive(input.readByte());
+        } else if (capacity <= BitSet.BITS_PER_SHORT) {
+            return fromPrimitive(input.readShort());
+        } else if (capacity <= BitSet.BITS_PER_INT) {
+            return fromPrimitive(input.readInt());
+        } else if (capacity <= BitSet.BITS_PER_LONG) {
+            return fromPrimitive(input.readLong());
+        } else {
+            int nLongs = (capacity-1) / BitSet.BITS_PER_LONG + 1;
+            return fromArray(ByteUtil.readFixedLengthLongArray(input, nLongs));
+        }
+    }
+    
+    public static void write(DataOutput output, BitSet bitSet, int capacity) throws IOException {
+        if (capacity <= BitSet.BITS_PER_BYTE) {
+            output.writeByte((byte)bitSet.bits[0]);
+        } else if (capacity <= BitSet.BITS_PER_SHORT) {
+            output.writeShort((short)bitSet.bits[0]);
+        } else if (capacity <= BitSet.BITS_PER_INT) {
+            output.writeInt((int)bitSet.bits[0]);
+        } else if (capacity <= BitSet.BITS_PER_LONG) {
+            output.writeLong(bitSet.bits[0]);
+        } else {
+            ByteUtil.writeFixedLengthLongArray(output, bitSet.bits);
+        }
+    }
+    
+    public static BitSet fromPrimitive(byte bits) {
+        return new BitSet(new long[] { bits });
+    }
+
+    public static BitSet fromPrimitive(short bits) {
+        return new BitSet(new long[] { bits });
+    }
+
+    public static BitSet fromPrimitive(int bits) {
+        return new BitSet(new long[] { bits });
+    }
+
+    public static BitSet fromPrimitive(long bits) {
+        return new BitSet(new long[] { bits });
+    }
+
+    public static BitSet fromArray(long[] bits) {
+        return new BitSet(bits);
+    }
+
+    public static BitSet withCapacity(int maxBits) {
+        int size = Math.max(1,(maxBits + BITS_PER_LONG -1) / BITS_PER_LONG);
+        return new BitSet(new long[size]);
+    }
+
+    public BitSet(long[] bits) {
+        this.bits = bits;
+    }
+
+    public boolean get(int nBit) {
+        int lIndex = nBit / BITS_PER_LONG;
+        int bIndex = nBit % BITS_PER_LONG;
+        return (bits[lIndex] & (1L << bIndex)) != 0;
+    }
+    
+    public void set(int nBit) {
+        int lIndex = nBit / BITS_PER_LONG;
+        int bIndex = nBit % BITS_PER_LONG;
+        bits[lIndex] |= (1L << bIndex);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/util/ByteUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/ByteUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/ByteUtil.java
new file mode 100644
index 0000000..93c4df2
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/ByteUtil.java
@@ -0,0 +1,561 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.io.ByteArrayInputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Comparator;
+
+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.io.WritableUtils;
+
+import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.query.KeyRange;
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.PDataType;
+
+
+/**
+ * 
+ * Byte utilities
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class ByteUtil {
+    public static final byte[] EMPTY_BYTE_ARRAY = new byte[0];
+    public static final ImmutableBytesPtr EMPTY_BYTE_ARRAY_PTR = new ImmutableBytesPtr(
+            EMPTY_BYTE_ARRAY);
+    
+    public static final Comparator<ImmutableBytesPtr> BYTES_PTR_COMPARATOR = new Comparator<ImmutableBytesPtr>() {
+
+        @Override
+        public int compare(ImmutableBytesPtr o1, ImmutableBytesPtr o2) {
+            return Bytes.compareTo(o1.get(), o1.getOffset(), o1.getLength(), o2.get(), o2.getOffset(), o2.getLength());
+        }
+        
+    };
+
+    /**
+     * Serialize an array of byte arrays into a single byte array.  Used
+     * to pass through a set of bytes arrays as an attribute of a Scan.
+     * Use {@link #toByteArrays(byte[], int)} to convert the serialized
+     * byte array back to the array of byte arrays.
+     * @param byteArrays the array of byte arrays to serialize
+     * @return the byte array
+     */
+    public static byte[] toBytes(byte[][] byteArrays) {
+        int size = 0;
+        for (byte[] b : byteArrays) {
+            if (b == null) {
+                size++;
+            } else {
+                size += b.length;
+                size += WritableUtils.getVIntSize(b.length);
+            }
+        }
+        TrustedByteArrayOutputStream bytesOut = new TrustedByteArrayOutputStream(size);
+        DataOutputStream out = new DataOutputStream(bytesOut);
+        try {
+            for (byte[] b : byteArrays) {
+                if (b == null) {
+                    WritableUtils.writeVInt(out, 0);
+                } else {
+                    WritableUtils.writeVInt(out, b.length);
+                    out.write(b);
+                }
+            }
+        } catch (IOException e) {
+            throw new RuntimeException(e); // not possible
+        } finally {
+            try {
+                out.close();
+            } catch (IOException e) {
+                throw new RuntimeException(e); // not possible
+            }
+        }
+        return bytesOut.getBuffer();
+    }
+
+    /**
+     * Deserialize a byte array into a set of byte arrays.  Used in
+     * coprocessor to reconstruct byte arrays from attribute value
+     * passed through the Scan.
+     * @param b byte array containing serialized byte arrays (created by {@link #toBytes(byte[][])}).
+     * @param length number of byte arrays that were serialized
+     * @return array of now deserialized byte arrays
+     * @throws IllegalStateException if there are more than length number of byte arrays that were serialized
+     */
+    public static byte[][] toByteArrays(byte[] b, int length) {
+        return toByteArrays(b, 0, length);
+    }
+
+    public static byte[][] toByteArrays(byte[] b, int offset, int length) {
+        ByteArrayInputStream bytesIn = new ByteArrayInputStream(b, offset, b.length - offset);
+        DataInputStream in = new DataInputStream(bytesIn);
+        byte[][] byteArrays = new byte[length][];
+        try {
+            for (int i = 0; i < length; i++) {
+                int bLength = WritableUtils.readVInt(in);
+                if (bLength == 0) {
+                    byteArrays[i] = null;
+                } else {
+                    byteArrays[i] = new byte[bLength];
+                    int rLength = in.read(byteArrays[i], 0, bLength);
+                    assert (rLength == bLength); // For find bugs
+                }
+            }
+            if (in.read() != -1) {
+                throw new IllegalStateException("Expected only " + length + " byte arrays, but found more");
+            }
+            return byteArrays;
+        } catch (IOException e) {
+            throw new RuntimeException(e); // not possible
+        } finally {
+            try {
+                in.close();
+            } catch (IOException e) {
+                throw new RuntimeException(e); // not possible
+            }
+        }
+    }
+
+    public static byte[] serializeVIntArray(int[] intArray) {
+        return serializeVIntArray(intArray,intArray.length);
+    }
+
+    public static byte[] serializeVIntArray(int[] intArray, int encodedLength) {
+        int size = WritableUtils.getVIntSize(encodedLength);
+        for (int i = 0; i < intArray.length; i++) {
+            size += WritableUtils.getVIntSize(intArray[i]);
+        }
+        int offset = 0;
+        byte[] out = new byte[size];
+        offset += ByteUtil.vintToBytes(out, offset, size);
+        for (int i = 0; i < intArray.length; i++) {
+            offset += ByteUtil.vintToBytes(out, offset, intArray[i]);
+        }
+        return out;
+    }
+
+    public static void serializeVIntArray(DataOutput output, int[] intArray) throws IOException {
+        serializeVIntArray(output, intArray, intArray.length);
+    }
+
+    /**
+     * Allows additional stuff to be encoded in length
+     * @param output
+     * @param intArray
+     * @param encodedLength
+     * @throws IOException
+     */
+    public static void serializeVIntArray(DataOutput output, int[] intArray, int encodedLength) throws IOException {
+        WritableUtils.writeVInt(output, encodedLength);
+        for (int i = 0; i < intArray.length; i++) {
+            WritableUtils.writeVInt(output, intArray[i]);
+        }
+    }
+
+    public static long[] readFixedLengthLongArray(DataInput input, int length) throws IOException {
+        long[] longArray = new long[length];
+        for (int i = 0; i < length; i++) {
+            longArray[i] = input.readLong();
+        }
+        return longArray;
+    }
+
+    public static void writeFixedLengthLongArray(DataOutput output, long[] longArray) throws IOException {
+        for (int i = 0; i < longArray.length; i++) {
+            output.writeLong(longArray[i]);
+        }
+    }
+
+    /**
+     * Deserialize a byte array into a int array.  
+     * @param b byte array storing serialized vints
+     * @return int array
+     */
+    public static int[] deserializeVIntArray(byte[] b) {
+        ByteArrayInputStream bytesIn = new ByteArrayInputStream(b);
+        DataInputStream in = new DataInputStream(bytesIn);
+        try {
+            int length = WritableUtils.readVInt(in);
+            return deserializeVIntArray(in, length);
+        } catch (IOException e) {
+            throw new RuntimeException(e); // not possible
+        } finally {
+            try {
+                in.close();
+            } catch (IOException e) {
+                throw new RuntimeException(e); // not possible
+            }
+        }
+    }
+
+    public static int[] deserializeVIntArray(DataInput in) throws IOException {
+        return deserializeVIntArray(in, WritableUtils.readVInt(in));
+    }
+
+    public static int[] deserializeVIntArray(DataInput in, int length) throws IOException {
+        int i = 0;
+        int[] intArray = new int[length];
+        while (i < length) {
+            intArray[i++] = WritableUtils.readVInt(in);
+        }
+        return intArray;
+    }
+
+    /**
+     * Deserialize a byte array into a int array.  
+     * @param b byte array storing serialized vints
+     * @param length number of serialized vints
+     * @return int array
+     */
+    public static int[] deserializeVIntArray(byte[] b, int length) {
+        ByteArrayInputStream bytesIn = new ByteArrayInputStream(b);
+        DataInputStream in = new DataInputStream(bytesIn);
+        try {
+            return deserializeVIntArray(in,length);
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    /**
+     * Concatenate together one or more byte arrays
+     * @param first first byte array
+     * @param rest rest of byte arrays
+     * @return newly allocated byte array that is a concatenation of all the byte arrays passed in
+     */
+    public static byte[] concat(byte[] first, byte[]... rest) {
+        int totalLength = first.length;
+        for (byte[] array : rest) {
+            totalLength += array.length;
+        }
+        byte[] result = Arrays.copyOf(first, totalLength);
+        int offset = first.length;
+        for (byte[] array : rest) {
+            System.arraycopy(array, 0, result, offset, array.length);
+            offset += array.length;
+        }
+        return result;
+    }
+
+    public static <T> T[] concat(T[] first, T[]... rest) {
+        int totalLength = first.length;
+        for (T[] array : rest) {
+          totalLength += array.length;
+        }
+        T[] result = Arrays.copyOf(first, totalLength);
+        int offset = first.length;
+        for (T[] array : rest) {
+            System.arraycopy(array, 0, result, offset, array.length);
+            offset += array.length;
+        }
+        return result;
+    }
+
+    public static byte[] concat(ColumnModifier columnModifier, ImmutableBytesWritable... writables) {
+        int totalLength = 0;
+        for (ImmutableBytesWritable writable : writables) {
+            totalLength += writable.getLength();
+        }
+        byte[] result = new byte[totalLength];
+        int offset = 0;
+        for (ImmutableBytesWritable array : writables) {
+            byte[] bytes = array.get();
+            if (columnModifier != null) {
+                bytes = columnModifier.apply(bytes, array.getOffset(), new byte[array.getLength()], 0, array.getLength());
+            }
+            System.arraycopy(bytes, array.getOffset(), result, offset, array.getLength());
+            offset += array.getLength();
+        }
+        return result;
+    }
+
+    public static int vintFromBytes(byte[] buffer, int offset) {
+        try {
+            return (int)Bytes.readVLong(buffer, offset);
+        } catch (IOException e) { // Impossible
+            throw new RuntimeException(e);
+        }
+    }
+
+    /**
+     * Decode a vint from the buffer pointed at to by ptr and
+     * increment the offset of the ptr by the length of the
+     * vint.
+     * @param ptr a pointer to a byte array buffer
+     * @return the decoded vint value as an int
+     */
+    public static int vintFromBytes(ImmutableBytesWritable ptr) {
+        return (int) vlongFromBytes(ptr);
+    }
+
+    /**
+     * Decode a vint from the buffer pointed at to by ptr and
+     * increment the offset of the ptr by the length of the
+     * vint.
+     * @param ptr a pointer to a byte array buffer
+     * @return the decoded vint value as a long
+     */
+    public static long vlongFromBytes(ImmutableBytesWritable ptr) {
+        final byte [] buffer = ptr.get();
+        final int offset = ptr.getOffset();
+        byte firstByte = buffer[offset];
+        int len = WritableUtils.decodeVIntSize(firstByte);
+        if (len == 1) {
+            ptr.set(buffer, offset+1, ptr.getLength());
+            return firstByte;
+        }
+        long i = 0;
+        for (int idx = 0; idx < len-1; idx++) {
+            byte b = buffer[offset + 1 + idx];
+            i = i << 8;
+            i = i | (b & 0xFF);
+        }
+        ptr.set(buffer, offset+len, ptr.getLength());
+        return (WritableUtils.isNegativeVInt(firstByte) ? ~i : i);
+    }
+
+    
+    /**
+     * Put long as variable length encoded number at the offset in the result byte array
+     * @param vint Integer to make a vint of.
+     * @param result buffer to put vint into
+     * @return Vint length in bytes of vint
+     */
+    public static int vintToBytes(byte[] result, int offset, final long vint) {
+      long i = vint;
+      if (i >= -112 && i <= 127) {
+        result[offset] = (byte) i;
+        return 1;
+      }
+
+      int len = -112;
+      if (i < 0) {
+        i ^= -1L; // take one's complement'
+        len = -120;
+      }
+
+      long tmp = i;
+      while (tmp != 0) {
+        tmp = tmp >> 8;
+        len--;
+      }
+
+      result[offset++] = (byte) len;
+
+      len = (len < -120) ? -(len + 120) : -(len + 112);
+
+      for (int idx = len; idx != 0; idx--) {
+        int shiftbits = (idx - 1) * 8;
+        long mask = 0xFFL << shiftbits;
+        result[offset++] = (byte)((i & mask) >> shiftbits);
+      }
+      return len + 1;
+    }
+
+    /**
+     * Increment the key to the next key
+     * @param key the key to increment
+     * @return a new byte array with the next key or null
+     *  if the key could not be incremented because it's
+     *  already at its max value.
+     */
+    public static byte[] nextKey(byte[] key) {
+        byte[] nextStartRow = new byte[key.length];
+        System.arraycopy(key, 0, nextStartRow, 0, key.length);
+        if (!nextKey(nextStartRow, nextStartRow.length)) {
+            return null;
+        }
+        return nextStartRow;
+    }
+
+    /**
+     * Increment the key in-place to the next key
+     * @param key the key to increment
+     * @param length the length of the key
+     * @return true if the key can be incremented and
+     *  false otherwise if the key is at its max
+     *  value.
+     */
+    public static boolean nextKey(byte[] key, int length) {
+        return nextKey(key, 0, length);
+    }
+    
+    public static boolean nextKey(byte[] key, int offset, int length) {
+        if (length == 0) {
+            return false;
+        }
+        int i = offset + length - 1;
+        while (key[i] == -1) {
+            key[i] = 0;
+            i--;
+            if (i < offset) {
+                // Change bytes back to the way they were
+                do {
+                    key[++i] = -1;
+                } while (i < offset + length - 1);
+                return false;
+            }
+         }
+        key[i] = (byte)(key[i] + 1);
+        return true;
+    }
+
+    public static byte[] previousKey(byte[] key) {
+        byte[] previousKey = new byte[key.length];
+        System.arraycopy(key, 0, previousKey, 0, key.length);
+        if (!previousKey(previousKey, previousKey.length)) {
+            return null;
+        }
+        return previousKey;
+    }
+
+    public static boolean previousKey(byte[] key, int length) {
+        return previousKey(key, 0, length);
+    }
+    
+    public static boolean previousKey(byte[] key, int offset, int length) {
+        if (length == 0) {
+            return false;
+        }
+        int i = offset + length - 1;
+        while (key[i] == 0) {
+            key[i] = -1;
+            i--;
+            if (i < offset) {
+                // Change bytes back to the way they were
+                do {
+                    key[++i] = 0;
+                } while (i < offset + length - 1);
+                return false;
+            }
+         }
+        key[i] = (byte)(key[i] - 1);
+        return true;
+    }
+
+    /**
+     * Expand the key to length bytes using the fillByte to fill the
+     * bytes beyond the current key length.
+     */
+    public static byte[] fillKey(byte[] key, int length) {
+        if(key.length > length) {
+            throw new IllegalStateException();
+        }
+        if (key.length == length) {
+            return key;
+        }
+        byte[] newBound = new byte[length];
+        System.arraycopy(key, 0, newBound, 0, key.length);
+        return newBound;
+    }
+
+    /**
+     * Get the size in bytes of the UTF-8 encoded CharSequence
+     * @param sequence the CharSequence
+     */
+    public static int getSize(CharSequence sequence) {
+        int count = 0;
+        for (int i = 0, len = sequence.length(); i < len; i++) {
+          char ch = sequence.charAt(i);
+          if (ch <= 0x7F) {
+            count++;
+          } else if (ch <= 0x7FF) {
+            count += 2;
+          } else if (Character.isHighSurrogate(ch)) {
+            count += 4;
+            ++i;
+          } else {
+            count += 3;
+          }
+        }
+        return count;
+    }
+
+    public static boolean isInclusive(CompareOp op) {
+        switch (op) {
+            case LESS:
+            case GREATER:
+                return false;
+            case EQUAL:
+            case NOT_EQUAL:
+            case LESS_OR_EQUAL:
+            case GREATER_OR_EQUAL:
+                return true;
+            default:
+              throw new RuntimeException("Unknown Compare op " + op.name());
+        }
+    }
+    public static boolean compare(CompareOp op, int compareResult) {
+        switch (op) {
+            case LESS:
+              return compareResult < 0;
+            case LESS_OR_EQUAL:
+              return compareResult <= 0;
+            case EQUAL:
+              return compareResult == 0;
+            case NOT_EQUAL:
+              return compareResult != 0;
+            case GREATER_OR_EQUAL:
+              return compareResult >= 0;
+            case GREATER:
+              return compareResult > 0;
+            default:
+              throw new RuntimeException("Unknown Compare op " + op.name());
+        }
+    }
+
+    /**
+     * Given an ImmutableBytesWritable, returns the payload part of the argument as an byte array. 
+     */
+    public static byte[] copyKeyBytesIfNecessary(ImmutableBytesWritable ptr) {
+        if (ptr.getOffset() == 0 && ptr.getLength() == ptr.get().length) {
+            return ptr.get();
+        }
+        return ptr.copyBytes();
+    }
+    
+    public static KeyRange getKeyRange(byte[] key, CompareOp op, PDataType type) {
+        switch (op) {
+        case EQUAL:
+            return type.getKeyRange(key, true, key, true);
+        case GREATER:
+            return type.getKeyRange(key, false, KeyRange.UNBOUND, false);
+        case GREATER_OR_EQUAL:
+            return type.getKeyRange(key, true, KeyRange.UNBOUND, false);
+        case LESS:
+            return type.getKeyRange(KeyRange.UNBOUND, false, key, false);
+        case LESS_OR_EQUAL:
+            return type.getKeyRange(KeyRange.UNBOUND, false, key, true);
+        default:
+            throw new IllegalArgumentException("Unknown operator " + op);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/util/CSVLoader.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/CSVLoader.java b/phoenix-core/src/main/java/org/apache/phoenix/util/CSVLoader.java
new file mode 100644
index 0000000..508818d
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/CSVLoader.java
@@ -0,0 +1,250 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 au.com.bytecode.opencsv.CSVReader;
+import com.google.common.collect.Maps;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.schema.PDataType;
+
+import java.io.FileReader;
+import java.sql.DatabaseMetaData;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/***
+ * Upserts CSV data using Phoenix JDBC connection
+ * 
+ * @author mchohan
+ * 
+ */
+public class CSVLoader {
+
+	private final PhoenixConnection conn;
+	private final String tableName;
+    private final List<String> columns;
+    private final boolean isStrict;
+    private final List<String> delimiter;
+    private final Map<String,Character> ctrlTable = new HashMap<String,Character>() {
+        {   put("1",'\u0001');
+            put("2",'\u0002');
+            put("3",'\u0003');
+            put("4",'\u0004');
+            put("5",'\u0005');
+            put("6",'\u0006');
+            put("7",'\u0007');
+            put("8",'\u0008');
+            put("9",'\u0009');}};
+    
+    private int unfoundColumnCount;
+
+    public CSVLoader(PhoenixConnection conn, String tableName, List<String> columns, boolean isStrict,List<String> delimiter) {
+        this.conn = conn;
+        this.tableName = tableName;
+        this.columns = columns;
+        this.isStrict = isStrict;
+        this.delimiter = delimiter;
+    }
+
+	public CSVLoader(PhoenixConnection conn, String tableName, List<String> columns, boolean isStrict) {
+       this(conn,tableName,columns,isStrict,null);
+    }
+
+
+    /**
+	 * Upserts data from CSV file. Data is batched up based on connection batch
+	 * size. Column PDataType is read from metadata and is used to convert
+	 * column value to correct type before upsert. Note: Column Names are
+	 * expected as first line of CSV file.
+	 * 
+	 * @param fileName
+	 * @throws Exception
+	 */
+	public void upsert(String fileName) throws Exception {
+        List<String> delimiter = this.delimiter;
+        CSVReader reader;
+        if ((delimiter != null) && (delimiter.size() == 3)) {
+            reader = new CSVReader(new FileReader(fileName),
+                getCSVCustomField(this.delimiter.get(0)),
+                getCSVCustomField(this.delimiter.get(1)),
+                getCSVCustomField(this.delimiter.get(2)));
+        } else {
+            reader = new CSVReader(new FileReader(fileName));
+        }
+        upsert(reader);
+	}
+
+
+    public char getCSVCustomField(String field) {
+        if(this.ctrlTable.containsKey(field)) {
+            return this.ctrlTable.get(field);
+        } else {
+            return field.charAt(0);
+        }
+    }
+
+	/**
+	 * Upserts data from CSV file. Data is batched up based on connection batch
+	 * size. Column PDataType is read from metadata and is used to convert
+	 * column value to correct type before upsert. Note: Column Names are
+	 * expected as first line of CSV file.
+	 * 
+	 * @param reader CSVReader instance
+	 * @throws Exception
+	 */
+	public void upsert(CSVReader reader) throws Exception {
+	    List<String> columns = this.columns;
+	    if (columns != null && columns.isEmpty()) {
+	        columns = Arrays.asList(reader.readNext());
+	    }
+		ColumnInfo[] columnInfo = generateColumnInfo(columns);
+        PreparedStatement stmt = null;
+        PreparedStatement[] stmtCache = null;
+		if (columns == null) {
+		    stmtCache = new PreparedStatement[columnInfo.length];
+		} else {
+		    String upsertStatement = QueryUtil.constructUpsertStatement(columnInfo, tableName, columnInfo.length - unfoundColumnCount);
+		    stmt = conn.prepareStatement(upsertStatement);
+		}
+		String[] nextLine;
+		int rowCount = 0;
+		int upsertBatchSize = conn.getMutateBatchSize();
+		boolean wasAutoCommit = conn.getAutoCommit();
+		try {
+    		conn.setAutoCommit(false);
+    		Object upsertValue = null;
+    		long start = System.currentTimeMillis();
+    
+    		// Upsert data based on SqlType of each column
+    		while ((nextLine = reader.readNext()) != null) {
+    		    if (columns == null) {
+    		        stmt = stmtCache[nextLine.length-1];
+    		        if (stmt == null) {
+    	                String upsertStatement = QueryUtil.constructUpsertStatement(columnInfo, tableName, nextLine.length);
+    	                stmt = conn.prepareStatement(upsertStatement);
+    	                stmtCache[nextLine.length-1] = stmt;
+    		        }
+    		    }
+    			for (int index = 0; index < columnInfo.length; index++) {
+    			    if (columnInfo[index] == null) {
+    			        continue;
+    			    }
+                    String line = nextLine[index];
+                    Integer info = columnInfo[index].getSqlType();
+                    upsertValue = convertTypeSpecificValue(line, info);
+    				if (upsertValue != null) {
+    					stmt.setObject(index + 1, upsertValue, columnInfo[index].getSqlType());
+    				} else {
+    					stmt.setNull(index + 1, columnInfo[index].getSqlType());
+    				}
+    			}
+    			stmt.execute();
+    
+    			// Commit when batch size is reached
+    			if (++rowCount % upsertBatchSize == 0) {
+    				conn.commit();
+    				System.out.println("Rows upserted: " + rowCount);
+    			}
+    		}
+    		conn.commit();
+    		double elapsedDuration = ((System.currentTimeMillis() - start) / 1000.0);
+    		System.out.println("CSV Upsert complete. " + rowCount + " rows upserted");
+    		System.out.println("Time: " + elapsedDuration + " sec(s)\n");
+		} finally {
+		    if(stmt != null) {
+		        stmt.close();
+		    }
+		    if (wasAutoCommit) conn.setAutoCommit(true);
+		}
+	}
+	
+	/**
+	 * Gets CSV string input converted to correct type 
+	 */
+	private Object convertTypeSpecificValue(String s, Integer sqlType) throws Exception {
+	    return PDataType.fromTypeId(sqlType).toObject(s);
+	}
+
+	/**
+	 * Get array of ColumnInfos that contain Column Name and its associated
+	 * PDataType
+	 * 
+	 * @param columns
+	 * @return
+	 * @throws SQLException
+	 */
+	private ColumnInfo[] generateColumnInfo(List<String> columns)
+			throws SQLException {
+	    Map<String,Integer> columnNameToTypeMap = Maps.newLinkedHashMap();
+        DatabaseMetaData dbmd = conn.getMetaData();
+        // TODO: escape wildcard characters here because we don't want that behavior here
+        String escapedTableName = StringUtil.escapeLike(tableName);
+        String[] schemaAndTable = escapedTableName.split("\\.");
+        ResultSet rs = null;
+        try {
+            rs = dbmd.getColumns(null, (schemaAndTable.length == 1 ? "" : schemaAndTable[0]),
+                    (schemaAndTable.length == 1 ? escapedTableName : schemaAndTable[1]),
+                    null);
+            while (rs.next()) {
+                columnNameToTypeMap.put(rs.getString(QueryUtil.COLUMN_NAME_POSITION), rs.getInt(QueryUtil.DATA_TYPE_POSITION));
+            }
+        } finally {
+            if(rs != null) {
+                rs.close();
+            }
+        }
+        ColumnInfo[] columnType;
+	    if (columns == null) {
+            int i = 0;
+            columnType = new ColumnInfo[columnNameToTypeMap.size()];
+            for (Map.Entry<String, Integer> entry : columnNameToTypeMap.entrySet()) {
+                columnType[i++] = new ColumnInfo(entry.getKey(),entry.getValue());
+            }
+	    } else {
+            // Leave "null" as indication to skip b/c it doesn't exist
+            columnType = new ColumnInfo[columns.size()];
+            for (int i = 0; i < columns.size(); i++) {
+                String columnName = SchemaUtil.normalizeIdentifier(columns.get(i).trim());
+                Integer sqlType = columnNameToTypeMap.get(columnName);
+                if (sqlType == null) {
+                    if (isStrict) {
+                        throw new SQLExceptionInfo.Builder(SQLExceptionCode.COLUMN_NOT_FOUND)
+                            .setColumnName(columnName).setTableName(tableName).build().buildException();
+                    }
+                    unfoundColumnCount++;
+                } else {
+                    columnType[i] = new ColumnInfo(columnName, sqlType);
+                }
+            }
+            if (unfoundColumnCount == columns.size()) {
+                throw new SQLExceptionInfo.Builder(SQLExceptionCode.COLUMN_NOT_FOUND)
+                    .setColumnName(Arrays.toString(columns.toArray(new String[0]))).setTableName(tableName).build().buildException();
+            }
+	    }
+		return columnType;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/util/Closeables.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/Closeables.java b/phoenix-core/src/main/java/org/apache/phoenix/util/Closeables.java
new file mode 100644
index 0000000..b09ebe0
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/Closeables.java
@@ -0,0 +1,124 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.io.Closeable;
+import java.io.IOException;
+import java.util.*;
+
+import com.google.common.collect.Iterables;
+
+
+/**
+ * Utilities for operating on {@link Closeable}s.
+ * 
+ */
+public class Closeables {
+    /** Not constructed */
+    private Closeables() { }
+    
+    /**
+     * Allows you to close as many of the {@link Closeable}s as possible.
+     * 
+     * If any of the close's fail with an IOException, those exception(s) will
+     * be thrown after attempting to close all of the inputs.
+     */
+    public static void closeAll(Iterable<? extends Closeable> iterable) throws IOException {
+        IOException ex = closeAllQuietly(iterable);
+        if (ex != null) throw ex;
+    }
+ 
+    public static IOException closeAllQuietly(Iterable<? extends Closeable> iterable) {
+        if (iterable == null) return null;
+        
+        LinkedList<IOException> exceptions = null;
+        for (Closeable closeable : iterable) {
+            try {
+                closeable.close();
+            } catch (IOException x) {
+                if (exceptions == null) exceptions = new LinkedList<IOException>();
+                exceptions.add(x);
+            }
+        }
+        
+        IOException ex = MultipleCausesIOException.fromIOExceptions(exceptions);
+        return ex;
+    }
+
+    static private class MultipleCausesIOException extends IOException {
+    	private static final long serialVersionUID = 1L;
+
+        static IOException fromIOExceptions(Collection<? extends IOException> exceptions) {
+            if (exceptions == null || exceptions.isEmpty()) return null;
+            if (exceptions.size() == 1) return Iterables.getOnlyElement(exceptions);
+            
+            return new MultipleCausesIOException(exceptions);
+        }
+        
+        private final Collection<? extends IOException> exceptions;
+        private boolean hasSetStackTrace;
+        
+        /**
+         * Use the {@link #fromIOExceptions(Collection) factory}.
+         */
+        private MultipleCausesIOException(Collection<? extends IOException> exceptions) {
+            this.exceptions = exceptions;
+        }
+
+        @Override
+        public String getMessage() {
+            StringBuilder sb = new StringBuilder(this.exceptions.size() * 50);
+            int exceptionNum = 0;
+            for (IOException ex : this.exceptions) {
+                sb.append("Cause Number " + exceptionNum + ": " + ex.getMessage() + "\n");
+                exceptionNum++;
+            }
+            return sb.toString();
+        }
+        
+        @Override
+        public StackTraceElement[] getStackTrace() {
+            if (!this.hasSetStackTrace) {
+                ArrayList<StackTraceElement> frames = new ArrayList<StackTraceElement>(this.exceptions.size() * 20);
+                
+                int exceptionNum = 0;
+                for (IOException exception : this.exceptions) {
+                    StackTraceElement header = new StackTraceElement(MultipleCausesIOException.class.getName(), 
+                            "Exception Number " + exceptionNum, 
+                            "<no file>",
+                            0);
+                    
+                    frames.add(header);
+                    for (StackTraceElement ste : exception.getStackTrace()) {
+                        frames.add(ste);
+                    }
+                    exceptionNum++;
+                }
+                
+                setStackTrace(frames.toArray(new StackTraceElement[frames.size()]));
+                this.hasSetStackTrace = true;
+            }        
+            
+            return super.getStackTrace();
+        }
+
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/util/ColumnInfo.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/ColumnInfo.java b/phoenix-core/src/main/java/org/apache/phoenix/util/ColumnInfo.java
new file mode 100644
index 0000000..9ca05e7
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/ColumnInfo.java
@@ -0,0 +1,22 @@
+package org.apache.phoenix.util;
+
+/**
+ * ColumnInfo used to store Column Name and its associated PDataType
+ */
+public class ColumnInfo {
+	private String columnName;
+	private Integer sqlType;
+
+	public ColumnInfo(String columnName, Integer sqlType) {
+		this.columnName = columnName;
+		this.sqlType = sqlType;
+	}
+
+	public String getColumnName() {
+		return columnName;
+	}
+
+	public Integer getSqlType() {
+		return sqlType;
+	}
+}	


[13/51] [partial] Initial commit of master branch from github

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/query/StatsManagerImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/StatsManagerImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/StatsManagerImpl.java
new file mode 100644
index 0000000..a36f275
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/StatsManagerImpl.java
@@ -0,0 +1,221 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.io.IOException;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.hadoop.hbase.HConstants;
+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.filter.KeyOnlyFilter;
+
+import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.ServerUtil;
+
+
+/**
+ * 
+ * Implementation of StatsManager. Table stats are updated asynchronously when they're
+ * accessed and past time-to-live. In this case, future calls (after the asynchronous
+ * call has completed), will have the updated stats.
+ * 
+ * All tables share the same HBase connection for a given connection and each connection
+ * will have it's own cache for these stats. This isn't ideal and will get reworked when
+ * the schema is kept on the server side. It's ok for now because:
+ * 1) we only ask the server for these stats when the start/end region is queried against
+ * 2) the query to get the stats pulls a single row so it's very cheap
+ * 3) it's async and if it takes too long it won't lead to anything except less optimal
+ *  parallelization.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class StatsManagerImpl implements StatsManager {
+    private final ConnectionQueryServices services;
+    private final int statsUpdateFrequencyMs;
+    private final int maxStatsAgeMs;
+    private final TimeKeeper timeKeeper;
+    private final ConcurrentMap<String,PTableStats> tableStatsMap = new ConcurrentHashMap<String,PTableStats>();
+
+    public StatsManagerImpl(ConnectionQueryServices services, int statsUpdateFrequencyMs, int maxStatsAgeMs) {
+        this(services, statsUpdateFrequencyMs, maxStatsAgeMs, TimeKeeper.SYSTEM);
+    }
+    
+    public StatsManagerImpl(ConnectionQueryServices services, int statsUpdateFrequencyMs, int maxStatsAgeMs, TimeKeeper timeKeeper) {
+        this.services = services;
+        this.statsUpdateFrequencyMs = statsUpdateFrequencyMs;
+        this.maxStatsAgeMs = maxStatsAgeMs;
+        this.timeKeeper = timeKeeper;
+    }
+    
+    public static interface TimeKeeper {
+        static final TimeKeeper SYSTEM = new TimeKeeper() {
+            @Override
+            public long currentTimeMillis() {
+                return System.currentTimeMillis();
+            }
+        };
+        
+        long currentTimeMillis();
+    }
+    public long getStatsUpdateFrequency() {
+        return statsUpdateFrequencyMs;
+    }
+    
+    @Override
+    public void updateStats(TableRef tableRef) throws SQLException {
+        SQLException sqlE = null;
+        HTableInterface hTable = services.getTable(tableRef.getTable().getPhysicalName().getBytes());
+        try {
+            byte[] minKey = null, maxKey = null;
+            // Do a key-only scan to get the first row of a table. This is the min
+            // key for the table.
+            Scan scan = new Scan(HConstants.EMPTY_START_ROW, new KeyOnlyFilter());
+            ResultScanner scanner = hTable.getScanner(scan);
+            try {
+                Result r = scanner.next(); 
+                if (r != null) {
+                    minKey = r.getRow();
+                }
+            } finally {
+                scanner.close();
+            }
+            int maxPossibleKeyLength = SchemaUtil.estimateKeyLength(tableRef.getTable());
+            byte[] maxPossibleKey = new byte[maxPossibleKeyLength];
+            Arrays.fill(maxPossibleKey, (byte)255);
+            // Use this deprecated method to get the key "before" the max possible key value,
+            // which is the max key for a table.
+            @SuppressWarnings("deprecation")
+            Result r = hTable.getRowOrBefore(maxPossibleKey, tableRef.getTable().getColumnFamilies().iterator().next().getName().getBytes());
+            if (r != null) {
+                maxKey = r.getRow();
+            }
+            tableStatsMap.put(tableRef.getTable().getName().getString(), new PTableStats(timeKeeper.currentTimeMillis(),minKey,maxKey));
+        } catch (IOException e) {
+            sqlE = ServerUtil.parseServerException(e);
+        } finally {
+            try {
+                hTable.close();
+            } catch (IOException e) {
+                if (sqlE == null) {
+                    sqlE = ServerUtil.parseServerException(e);
+                } else {
+                    sqlE.setNextException(ServerUtil.parseServerException(e));
+                }
+            } finally {
+                if (sqlE != null) {
+                    throw sqlE;
+                }
+            }
+        }
+    }
+    
+    private PTableStats getStats(final TableRef table) {
+        PTableStats stats = tableStatsMap.get(table);
+        if (stats == null) {
+            PTableStats newStats = new PTableStats();
+            stats = tableStatsMap.putIfAbsent(table.getTable().getName().getString(), newStats);
+            stats = stats == null ? newStats : stats;
+        }
+        // Synchronize on the current stats for a table to prevent
+        // multiple attempts to update the stats.
+        synchronized (stats) {
+            long initiatedTime = stats.getInitiatedTime();
+            long currentTime = timeKeeper.currentTimeMillis();
+            // Update stats asynchronously if they haven't been updated within the specified frequency.
+            // We update asynchronously because we don't ever want to block the caller - instead we'll continue
+            // to use the old one.
+            if ( currentTime - initiatedTime >= getStatsUpdateFrequency()) {
+                stats.setInitiatedTime(currentTime);
+                services.getExecutor().submit(new Callable<Void>() {
+
+                    @Override
+                    public Void call() throws Exception { // TODO: will exceptions be logged?
+                        updateStats(table);
+                        return null;
+                    }
+                    
+                });
+            }
+            // If the stats are older than the max age, use an empty stats
+            if (currentTime - stats.getCompletedTime() >= maxStatsAgeMs) {
+                return PTableStats.NO_STATS;
+            }
+        }
+        return stats;
+    }
+    
+    @Override
+    public byte[] getMinKey(TableRef table) {
+        PTableStats stats = getStats(table);
+        return stats.getMinKey();
+    }
+
+    @Override
+    public byte[] getMaxKey(TableRef table) {
+        PTableStats stats = getStats(table);
+        return stats.getMaxKey();
+    }
+
+    private static class PTableStats {
+        private static final PTableStats NO_STATS = new PTableStats();
+        private long initiatedTime;
+        private final long completedTime;
+        private final byte[] minKey;
+        private final byte[] maxKey;
+        
+        public PTableStats() {
+            this(-1,null,null);
+        }
+        public PTableStats(long completedTime, byte[] minKey, byte[] maxKey) {
+            this.minKey = minKey;
+            this.maxKey = maxKey;
+            this.completedTime = this.initiatedTime = completedTime;
+        }
+
+        private byte[] getMinKey() {
+            return minKey;
+        }
+
+        private byte[] getMaxKey() {
+            return maxKey;
+        }
+
+        private long getCompletedTime() {
+            return completedTime;
+        }
+
+        private void setInitiatedTime(long initiatedTime) {
+            this.initiatedTime = initiatedTime;
+        }
+
+        private long getInitiatedTime() {
+            return initiatedTime;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/AmbiguousColumnException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/AmbiguousColumnException.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/AmbiguousColumnException.java
new file mode 100644
index 0000000..f460d63
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/AmbiguousColumnException.java
@@ -0,0 +1,52 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.sql.SQLException;
+
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+
+/**
+ * 
+ * Exception thrown when a column name is used without being qualified with an alias
+ * and more than one table contains that column.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class AmbiguousColumnException extends SQLException {
+    private static final long serialVersionUID = 1L;
+    private static SQLExceptionCode code = SQLExceptionCode.AMBIGUOUS_COLUMN;
+
+    public AmbiguousColumnException() {
+        super(new SQLExceptionInfo.Builder(code).build().toString(), code.getSQLState(), code.getErrorCode());
+    }
+
+    public AmbiguousColumnException(String columnName) {
+        super(new SQLExceptionInfo.Builder(code).setColumnName(columnName).build().toString(),
+                code.getSQLState(), code.getErrorCode());
+    }
+
+    public AmbiguousColumnException(String columnName, Throwable cause) {
+        super(new SQLExceptionInfo.Builder(code).setColumnName(columnName).build().toString(),
+                code.getSQLState(), code.getErrorCode(), cause);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/AmbiguousTableException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/AmbiguousTableException.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/AmbiguousTableException.java
new file mode 100644
index 0000000..56815f9
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/AmbiguousTableException.java
@@ -0,0 +1,52 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.sql.SQLException;
+
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+
+/**
+ * 
+ * Exception thrown when a table name is used without being qualified with an alias
+ * and more than one schema contains that table.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class AmbiguousTableException extends SQLException {
+    private static final long serialVersionUID = 1L;
+    private static SQLExceptionCode code = SQLExceptionCode.AMBIGUOUS_TABLE;
+
+    public AmbiguousTableException() {
+        super(new SQLExceptionInfo.Builder(code).build().toString(), code.getSQLState(), code.getErrorCode(), null);
+    }
+
+    public AmbiguousTableException(String tableName) {
+        super(new SQLExceptionInfo.Builder(code).setTableName(tableName).toString(),
+                code.getSQLState(), code.getErrorCode(), null);
+    }
+
+    public AmbiguousTableException(String tableName, Throwable cause) {
+        super(new SQLExceptionInfo.Builder(code).setTableName(tableName).toString(),
+                code.getSQLState(), code.getErrorCode(), cause);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/ArgumentTypeMismatchException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/ArgumentTypeMismatchException.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/ArgumentTypeMismatchException.java
new file mode 100644
index 0000000..51cea07
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/ArgumentTypeMismatchException.java
@@ -0,0 +1,44 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.sql.SQLException;
+
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+
+/**
+ * Exception thrown when we try to use use an argument that has the wrong type. 
+ * 
+ * @author zhuang
+ * @since 1.0
+ */
+public class ArgumentTypeMismatchException extends SQLException {
+    private static final long serialVersionUID = 1L;
+    private static SQLExceptionCode code = SQLExceptionCode.TYPE_MISMATCH;
+
+    public ArgumentTypeMismatchException(PDataType expected, PDataType actual, String location) {
+        super(new SQLExceptionInfo.Builder(code).setMessage("expected: " + expected + " but was: " + actual + " at " + location).build().toString(), code.getSQLState(), code.getErrorCode());
+    }
+
+    public ArgumentTypeMismatchException(String expected, String actual, String location) {
+        super(new SQLExceptionInfo.Builder(code).setMessage("expected: " + expected + " but was: " + actual + " at " + location).build().toString(), code.getSQLState(), code.getErrorCode());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnAlreadyExistsException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnAlreadyExistsException.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnAlreadyExistsException.java
new file mode 100644
index 0000000..6c270c1
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnAlreadyExistsException.java
@@ -0,0 +1,62 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.sql.SQLException;
+
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+
+
+/**
+ * 
+ * Exception thrown when a column already exists.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class ColumnAlreadyExistsException extends SQLException {
+    private static final long serialVersionUID = 1L;
+    private static SQLExceptionCode code = SQLExceptionCode.COLUMN_EXIST_IN_DEF;
+    private final String schemaName;
+    private final String tableName;
+    private final String columnName;
+
+    public ColumnAlreadyExistsException(String schemaName, String tableName, String columnName) {
+        super(new SQLExceptionInfo.Builder(code).setColumnName(columnName)
+                .setSchemaName(schemaName).setTableName(tableName).build().toString(),
+                code.getSQLState(), code.getErrorCode(), null);
+        this.schemaName = schemaName;
+        this.tableName = tableName;
+        this.columnName = columnName;
+    }
+
+    public String getTableName() {
+        return tableName;
+    }
+
+    public String getSchemaName() {
+        return schemaName;
+    }
+
+    public String getColumnName() {
+        return columnName;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnFamilyNotFoundException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnFamilyNotFoundException.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnFamilyNotFoundException.java
new file mode 100644
index 0000000..a450b71
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnFamilyNotFoundException.java
@@ -0,0 +1,46 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+
+/**
+ * 
+ * Exception thrown when a family name could not be found in the schema
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class ColumnFamilyNotFoundException extends MetaDataEntityNotFoundException {
+    private static final long serialVersionUID = 1L;
+    private static SQLExceptionCode code = SQLExceptionCode.COLUMN_FAMILY_NOT_FOUND;
+    private final String familyName;
+
+    public ColumnFamilyNotFoundException(String familyName) {
+        super(new SQLExceptionInfo.Builder(code).setFamilyName(familyName).build().toString(),
+                code.getSQLState(), code.getErrorCode(), null);
+        this.familyName = familyName;
+    }
+
+    public String getFamilyName() {
+        return familyName;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnModifier.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnModifier.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnModifier.java
new file mode 100644
index 0000000..7b4bc3d
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnModifier.java
@@ -0,0 +1,149 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.filter.CompareFilter.CompareOp;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * A ColumnModifier implementation modifies how bytes are stored in a primary key column.</p>  
+ * The {@link ColumnModifier#apply apply} method is called when the bytes for a specific column are first written to HBase and again
+ * when they are read back.  Phoenix attemps to minimize calls to apply when bytes are read out of HBase.   
+ * 
+ * @author simontoens
+ * @since 1.2
+ */
+public enum ColumnModifier {
+    /**
+     * Invert the bytes in the src byte array to support descending ordering of row keys.
+     */
+    SORT_DESC(1) {
+        @Override
+        public byte[] apply(byte[] src, int srcOffset, byte[] dest, int dstOffset, int length) {
+            Preconditions.checkNotNull(src);            
+            Preconditions.checkNotNull(dest);            
+            for (int i = 0; i < length; i++) {
+                dest[dstOffset+i] = (byte)(src[srcOffset+i] ^ 0xFF);
+            }                       
+            return dest;
+        }
+
+        @Override
+        public byte apply(byte b) {
+            return (byte)(b ^ 0xFF);
+        }
+
+        @Override
+        public CompareOp transform(CompareOp op) {
+            switch (op) {
+                case EQUAL:
+                    return op;
+                case GREATER:
+                    return CompareOp.LESS;
+                case GREATER_OR_EQUAL:
+                    return CompareOp.LESS_OR_EQUAL;
+                case LESS:
+                    return CompareOp.GREATER;
+                case LESS_OR_EQUAL:
+                    return CompareOp.GREATER_OR_EQUAL;
+                default:
+                    throw new IllegalArgumentException("Unknown operator " + op);
+            }
+        }
+
+        @Override
+        public byte[] apply(byte[] src, int srcOffset, int length) {
+            return apply(src, srcOffset, new byte[length], 0, length);
+        }
+    };
+        
+    private final int serializationId;
+    
+    ColumnModifier(int serializationId) {
+        this.serializationId = serializationId;
+    }
+    
+    public int getSerializationId() {
+        return serializationId;
+    }
+    /**
+     * Returns the ColumnModifier for the specified DDL stmt keyword.
+     */
+    public static ColumnModifier fromDDLValue(String modifier) {
+        if (modifier == null) {
+            return null;
+        } else if (modifier.equalsIgnoreCase("ASC")) {
+            return null;
+        } else if (modifier.equalsIgnoreCase("DESC")) {
+            return SORT_DESC;
+        } else {
+            return null;
+        }                       
+    }
+
+   /**
+    * Returns the ColumnModifier for the specified internal value.
+    */
+    public static ColumnModifier fromSystemValue(int value) {
+        for (ColumnModifier mod : ColumnModifier.values()) {
+            if (mod.getSerializationId() == value) {
+                return mod;
+            }
+        }
+        return null;
+    }
+
+    /**
+     * Returns an internal value representing the specified ColumnModifier.
+     */
+    public static int toSystemValue(ColumnModifier columnModifier) {
+        if (columnModifier == null) {
+            return 0;
+        }
+        return columnModifier.getSerializationId();
+    }
+
+    /**
+     * Copies the bytes from source array to destination array and applies the column modifier operation on the bytes
+     * starting at the specified offsets.  The column modifier is applied to the number of bytes matching the 
+     * specified length.
+     * @param src  the source byte array to copy from, cannot be null
+     * @param srcOffset the offset into the source byte array at which to begin.
+     * @param dest the destination byte array into which to transfer the modified bytes.
+     * @param dstOffset the offset into the destination byte array at which to begin
+     * @param length the number of bytes for which to apply the modification
+     * @return the destination byte array
+     */
+    public abstract byte[] apply(byte[] src, int srcOffset, byte[] dest, int dstOffset, int length);
+    /**
+     * Copies the bytes from source array to a newly allocated destination array and applies the column
+     * modifier operation on the bytes starting at the specified offsets.  The column modifier is applied
+     * to the number of bytes matching the specified length.
+     * @param src  the source byte array to copy from, cannot be null
+     * @param srcOffset the offset into the source byte array at which to begin.
+     * @param length the number of bytes for which to apply the modification
+     * @return the newly allocated destination byte array
+     */
+    public abstract byte[] apply(byte[] src, int srcOffset, int length);
+    public abstract byte apply(byte b);
+    
+    public abstract CompareOp transform(CompareOp op);
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnNotFoundException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnNotFoundException.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnNotFoundException.java
new file mode 100644
index 0000000..c3b331b
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnNotFoundException.java
@@ -0,0 +1,65 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+
+
+/**
+ * 
+ * Exception thrown when a column name referenced in a select
+ * statement cannot be found in any table.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class ColumnNotFoundException extends MetaDataEntityNotFoundException {
+    private static final long serialVersionUID = 1L;
+    private static SQLExceptionCode code = SQLExceptionCode.COLUMN_NOT_FOUND;
+    private final String schemaName;
+    private final String tableName;
+    private final String columnName;
+
+    public ColumnNotFoundException(String columnName) {
+        this(null, null, null, columnName);
+    }
+
+    public ColumnNotFoundException(String schemaName, String tableName, String familyName, String columnName) {
+        super(new SQLExceptionInfo.Builder(code).setSchemaName(schemaName).setTableName(tableName)
+                .setFamilyName(familyName).setColumnName(columnName).build().toString(),
+                code.getSQLState(), code.getErrorCode(), null);
+        this.schemaName = schemaName;
+        this.tableName = tableName;
+        this.columnName = columnName;
+    }
+
+    public String getTableName() {
+        return tableName;
+    }
+
+    public String getSchemaName() {
+        return schemaName;
+    }
+
+    public String getColumnName() {
+        return columnName;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..ed033b7
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnRef.java
@@ -0,0 +1,135 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.http.annotation.Immutable;
+
+import org.apache.phoenix.expression.ColumnExpression;
+import org.apache.phoenix.expression.IndexKeyValueColumnExpression;
+import org.apache.phoenix.expression.KeyValueColumnExpression;
+import org.apache.phoenix.expression.ProjectedColumnExpression;
+import org.apache.phoenix.expression.RowKeyColumnExpression;
+import org.apache.phoenix.util.IndexUtil;
+import org.apache.phoenix.util.SchemaUtil;
+
+
+/**
+ * 
+ * Class that represents a reference to a PColumn in a PTable
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+@Immutable
+public final class ColumnRef {
+    private final TableRef tableRef;
+    private final int columnPosition;
+    private final int pkSlotPosition;
+    
+    public ColumnRef(ColumnRef columnRef, long timeStamp) {
+        this.tableRef = new TableRef(columnRef.tableRef, timeStamp);
+        this.columnPosition = columnRef.columnPosition;
+        this.pkSlotPosition = columnRef.pkSlotPosition;
+    }
+
+    public ColumnRef(TableRef tableRef, int columnPosition) {
+        if (tableRef == null) {
+            throw new NullPointerException();
+        }
+        if (columnPosition < 0 || columnPosition >= tableRef.getTable().getColumns().size()) {
+            throw new IllegalArgumentException("Column position of " + columnPosition + " must be between 0 and " + tableRef.getTable().getColumns().size() + " for table " + tableRef.getTable().getName().getString());
+        }
+        this.tableRef = tableRef;
+        this.columnPosition = columnPosition;
+        PColumn column = getColumn();
+        int i = -1;
+        if (SchemaUtil.isPKColumn(column)) {
+            for (PColumn pkColumn : tableRef.getTable().getPKColumns()) {
+                i++;
+                if (pkColumn == column) {
+                    break;
+                }
+            }
+        }
+        pkSlotPosition = i;
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + columnPosition;
+        result = prime * result + tableRef.hashCode();
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) return true;
+        if (obj == null) return false;
+        if (getClass() != obj.getClass()) return false;
+        ColumnRef other = (ColumnRef)obj;
+        if (columnPosition != other.columnPosition) return false;
+        if (!tableRef.equals(other.tableRef)) return false;
+        return true;
+    }
+
+    public ColumnExpression newColumnExpression() {
+        boolean isIndex = tableRef.getTable().getType() == PTableType.INDEX;
+        if (SchemaUtil.isPKColumn(this.getColumn())) {
+            String name = this.getColumn().getName().getString();
+            if (isIndex) {
+                name = IndexUtil.getDataColumnName(name);
+            }
+            return new RowKeyColumnExpression(
+                    getColumn(), 
+                    new RowKeyValueAccessor(this.getTable().getPKColumns(), pkSlotPosition),
+                    name);
+        }
+        
+        if (isIndex)
+        	return new IndexKeyValueColumnExpression(getColumn());
+        
+        if (tableRef.getTable().getType() == PTableType.JOIN)
+        	return new ProjectedColumnExpression(getColumn(), tableRef.getTable());
+       
+        return new KeyValueColumnExpression(getColumn());
+    }
+
+    public int getColumnPosition() {
+        return columnPosition;
+    }
+    
+    public int getPKSlotPosition() {
+        return pkSlotPosition;
+    }
+    
+    public PColumn getColumn() {
+        return tableRef.getTable().getColumns().get(columnPosition);
+    }
+
+    public PTable getTable() {
+        return tableRef.getTable();
+    }
+    
+    public TableRef getTableRef() {
+        return tableRef;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/ConcurrentTableMutationException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/ConcurrentTableMutationException.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/ConcurrentTableMutationException.java
new file mode 100644
index 0000000..c3b540b
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/ConcurrentTableMutationException.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.sql.SQLException;
+
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+
+
+public class ConcurrentTableMutationException extends SQLException {
+    private static final long serialVersionUID = 1L;
+    private static SQLExceptionCode code = SQLExceptionCode.CONCURRENT_TABLE_MUTATION;
+    private final String schemaName;
+    private final String tableName;
+
+    public ConcurrentTableMutationException(String schemaName, String tableName) {
+        super(new SQLExceptionInfo.Builder(code).setSchemaName(schemaName).setTableName(tableName).build().toString(), 
+                code.getSQLState(), code.getErrorCode());
+        this.schemaName = schemaName;
+        this.tableName = tableName;
+    }
+
+    public String getTableName() {
+        return tableName;
+    }
+
+    public String getSchemaName() {
+        return schemaName;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/ConstraintViolationException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/ConstraintViolationException.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/ConstraintViolationException.java
new file mode 100644
index 0000000..5a8b260
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/ConstraintViolationException.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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;
+
+/**
+ * 
+ * Exception thrown when a schema constraint is violated at the
+ * time of data insertion.
+ *
+ * @author jtaylor
+ * @since 180
+ */
+public class ConstraintViolationException extends RuntimeException {
+	private static final long serialVersionUID = 1L;
+
+    public ConstraintViolationException() {
+    }
+
+    public ConstraintViolationException(String message) {
+        super(message);
+    }
+
+    public ConstraintViolationException(Throwable cause) {
+        super(cause);
+    }
+
+    public ConstraintViolationException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..af6b77e
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateColumn.java
@@ -0,0 +1,71 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+public class DelegateColumn extends DelegateDatum implements PColumn {
+    
+    public DelegateColumn(PColumn delegate) {
+        super(delegate);
+    }
+    
+    @Override
+    protected PColumn getDelegate() {
+        return (PColumn)super.getDelegate();
+    }
+    
+    @Override
+    public PName getName() {
+        return getDelegate().getName();
+    }
+    
+    @Override
+    public ColumnModifier getColumnModifier() {
+    	return getDelegate().getColumnModifier();
+    }
+
+    @Override
+    public PName getFamilyName() {
+        return getDelegate().getFamilyName();
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        getDelegate().readFields(input);
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+        getDelegate().write(output);
+    }
+
+    @Override
+    public int getPosition() {
+        return getDelegate().getPosition();
+    }
+
+    @Override
+    public Integer getArraySize() {
+        return getDelegate().getArraySize();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateDatum.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateDatum.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateDatum.java
new file mode 100644
index 0000000..86e4cc9
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateDatum.java
@@ -0,0 +1,62 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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;
+
+public class DelegateDatum implements PDatum {
+    private final PDatum delegate;
+    
+    public DelegateDatum(PDatum delegate) {
+        this.delegate = delegate;
+    }
+    
+    @Override
+    public boolean isNullable() {
+        return delegate.isNullable();
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return delegate.getDataType();
+    }
+
+    @Override
+    public Integer getByteSize() {
+        return delegate.getByteSize();
+    }
+
+    @Override
+    public Integer getMaxLength() {
+        return delegate.getByteSize();
+    }
+
+    @Override
+    public Integer getScale() {
+        return delegate.getScale();
+    }
+    
+	@Override
+	public ColumnModifier getColumnModifier() {
+		return delegate.getColumnModifier();
+	}
+
+    protected PDatum getDelegate() {
+        return delegate;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/EmptySequenceCacheException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/EmptySequenceCacheException.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/EmptySequenceCacheException.java
new file mode 100644
index 0000000..1964983
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/EmptySequenceCacheException.java
@@ -0,0 +1,21 @@
+package org.apache.phoenix.schema;
+
+import java.sql.SQLException;
+
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+
+public class EmptySequenceCacheException extends SQLException {
+    private static final long serialVersionUID = 1L;
+    private static SQLExceptionCode ERROR_CODE = SQLExceptionCode.EMPTY_SEQUENCE_CACHE;
+
+    public EmptySequenceCacheException() {
+        this(null,null);
+    }
+
+    public EmptySequenceCacheException(String schemaName, String tableName) {
+        super(new SQLExceptionInfo.Builder(ERROR_CODE).setSchemaName(schemaName).setTableName(tableName).build().toString(),
+                ERROR_CODE.getSQLState(), ERROR_CODE.getErrorCode(), null);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/ExecuteQueryNotApplicableException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/ExecuteQueryNotApplicableException.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/ExecuteQueryNotApplicableException.java
new file mode 100644
index 0000000..8812bf8
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/ExecuteQueryNotApplicableException.java
@@ -0,0 +1,39 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.sql.SQLException;
+
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+
+public class ExecuteQueryNotApplicableException extends SQLException {
+    private static final long serialVersionUID = 1L;
+    private static SQLExceptionCode code = SQLExceptionCode.EXECUTE_QUERY_NOT_APPLICABLE;
+
+    public ExecuteQueryNotApplicableException(String query) {
+        super(new SQLExceptionInfo.Builder(code).setMessage("Query: " + query).build().toString(), code.getSQLState(), code.getErrorCode());
+    }
+
+    public ExecuteQueryNotApplicableException(String command, String statement) {
+        super(new SQLExceptionInfo.Builder(code).setMessage("Command: " + command + ". Statement: " + statement).build().toString(), code.getSQLState(), code.getErrorCode());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/ExecuteUpdateNotApplicableException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/ExecuteUpdateNotApplicableException.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/ExecuteUpdateNotApplicableException.java
new file mode 100644
index 0000000..b99d87b
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/ExecuteUpdateNotApplicableException.java
@@ -0,0 +1,38 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.sql.SQLException;
+
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+
+public class ExecuteUpdateNotApplicableException extends SQLException {
+    private static final long serialVersionUID = 1L;
+    private static SQLExceptionCode code = SQLExceptionCode.EXECUTE_QUERY_NOT_APPLICABLE;
+
+    public ExecuteUpdateNotApplicableException(String query) {
+        super(new SQLExceptionInfo.Builder(code).setMessage("Query: " + query).build().toString(), code.getSQLState(), code.getErrorCode());
+    }
+
+    public ExecuteUpdateNotApplicableException(String command, String statement) {
+        super(new SQLExceptionInfo.Builder(code).setMessage("Command: " + command + ". Statement: " + statement).build().toString(), code.getSQLState(), code.getErrorCode());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/IllegalDataException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/IllegalDataException.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/IllegalDataException.java
new file mode 100644
index 0000000..83ae565
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/IllegalDataException.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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;
+
+/**
+ * 
+ * Exception thrown when an invalid or illegal data value is found
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class IllegalDataException extends ConstraintViolationException {
+	private static final long serialVersionUID = 1L;
+
+    public IllegalDataException() {
+    }
+
+    public IllegalDataException(String message) {
+        super(message);
+    }
+
+    public IllegalDataException(Throwable cause) {
+        super(cause);
+    }
+
+    public IllegalDataException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/KeyValueSchema.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/KeyValueSchema.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/KeyValueSchema.java
new file mode 100644
index 0000000..bca09a9
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/KeyValueSchema.java
@@ -0,0 +1,218 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.http.annotation.Immutable;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.ByteUtil;
+
+
+/**
+ * 
+ * Simple flat schema over a byte array where fields may be any of {@link PDataType}.
+ * Optimized for positional access by index.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+@Immutable
+public class KeyValueSchema extends ValueSchema {
+	
+	public KeyValueSchema() {
+	}
+    
+    protected KeyValueSchema(int minNullable, List<Field> fields) {
+        super(minNullable, fields);
+    }
+
+    public static class KeyValueSchemaBuilder extends ValueSchemaBuilder {
+
+        public KeyValueSchemaBuilder(int minNullable) {
+            super(minNullable);
+        }
+        
+        @Override
+        public KeyValueSchema build() {
+            List<Field> condensedFields = buildFields();
+            return new KeyValueSchema(this.minNullable, condensedFields);
+        }
+
+        @Override
+        public KeyValueSchemaBuilder setMaxFields(int nFields) {
+            super.setMaxFields(nFields);
+            return this;
+        }
+        
+        public KeyValueSchemaBuilder addField(PDatum datum) {
+            super.addField(datum, fields.size() <  this.minNullable, null);
+            return this;
+        }
+    }
+    
+    public boolean isNull(int position, ValueBitSet bitSet) {
+        int nBit = position - getMinNullable();
+        return (nBit >= 0 && !bitSet.get(nBit));
+    }
+    
+    private static byte[] ensureSize(byte[] b, int offset, int size) {
+        if (size > b.length) {
+            byte[] bBigger = new byte[Math.max(b.length * 2, size)];
+            System.arraycopy(b, 0, bBigger, 0, b.length);
+            return bBigger;
+        }
+        return b;
+    }
+
+    /**
+     * @return byte representation of the KeyValueSchema
+     */
+    public byte[] toBytes(Expression[] expressions, ValueBitSet valueSet, ImmutableBytesWritable ptr) {
+    	return toBytes(null, expressions, valueSet, ptr);
+    }
+    
+    /**
+     * @return byte representation of the KeyValueSchema
+     */
+    public byte[] toBytes(Tuple tuple, Expression[] expressions, ValueBitSet valueSet, ImmutableBytesWritable ptr) {
+        int offset = 0;
+        int index = 0;
+        valueSet.clear();
+        int minNullableIndex = getMinNullable();
+        byte[] b = new byte[getEstimatedValueLength() + valueSet.getEstimatedLength()];
+        List<Field> fields = getFields();
+        // We can get away with checking if only nulls are left in the outer loop,
+        // since repeating fields will not span the non-null/null boundary.
+        for (int i = 0; i < fields.size(); i++) {
+            Field field = fields.get(i);
+            PDataType type = field.getDataType();
+            for (int j = 0; j < field.getCount(); j++) {
+                if (expressions[index].evaluate(tuple, ptr)) { // Skip null values
+                    if (index >= minNullableIndex) {
+                        valueSet.set(index - minNullableIndex);
+                    }
+                    if (!type.isFixedWidth()) {
+                        b = ensureSize(b, offset, offset + getVarLengthBytes(ptr.getLength()));
+                        offset = writeVarLengthField(ptr, b, offset);
+                    } else {
+                        int nBytes = ptr.getLength();
+                        b = ensureSize(b, offset, offset + nBytes);
+                        System.arraycopy(ptr.get(), ptr.getOffset(), b, offset, nBytes);
+                        offset += nBytes;
+                    }
+                }
+                index++;
+            }
+        }
+        // Add information about which values were set at end of value,
+        // so that we can quickly access them without needing to walk
+        // through the values using the schema.
+        // TODO: if there aren't any non null values, don't serialize anything
+        b = ensureSize(b, offset, offset + valueSet.getEstimatedLength());
+        offset = valueSet.toBytes(b, offset);
+
+        if (offset == b.length) {
+            return b;
+        } else {
+            byte[] bExact = new byte[offset];
+            System.arraycopy(b, 0, bExact, 0, offset);
+            return bExact;
+        }
+    }
+
+    private int getVarLengthBytes(int length) {
+        return length + WritableUtils.getVIntSize(length);
+    }
+    
+    private int writeVarLengthField(ImmutableBytesWritable ptr, byte[] b, int offset) {
+        int length = ptr.getLength();
+        offset += ByteUtil.vintToBytes(b, offset, length);
+        System.arraycopy(ptr.get(), ptr.getOffset(), b, offset, length);                        
+        offset += length;
+        return offset;
+    }
+
+    @edu.umd.cs.findbugs.annotations.SuppressWarnings(
+            value="NP_BOOLEAN_RETURN_NULL", 
+            justification="Designed to return null.")
+    public Boolean iterator(byte[] src, int srcOffset, int srcLength, ImmutableBytesWritable ptr, int position, ValueBitSet valueBitSet) {
+        ptr.set(src, srcOffset, 0);
+        int maxOffset = srcOffset + srcLength;
+        Boolean hasValue = null;
+        for (int i = 0; i < position; i++) {
+            hasValue = next(ptr, i, maxOffset, valueBitSet);
+        }
+        return hasValue;
+    }
+    
+    public Boolean iterator(ImmutableBytesWritable srcPtr, ImmutableBytesWritable ptr, int position, ValueBitSet valueSet) {
+        return iterator(srcPtr.get(),srcPtr.getOffset(),srcPtr.getLength(), ptr, position, valueSet);
+    }
+    
+    public Boolean iterator(ImmutableBytesWritable ptr, int position, ValueBitSet valueSet) {
+        return iterator(ptr, ptr, position, valueSet);
+    }
+    
+    public Boolean iterator(ImmutableBytesWritable ptr) {
+        return iterator(ptr, ptr, 0, ValueBitSet.EMPTY_VALUE_BITSET);
+    }
+    
+    /**
+     * Move the bytes ptr to the next position relative to the current ptr
+     * @param ptr bytes pointer pointing to the value at the positional index
+     * provided.
+     * @param position zero-based index of the next field in the value schema
+     * @param maxOffset max possible offset value when iterating
+     * @return true if a value was found and ptr was set, false if the value is null and ptr was not
+     * set, and null if the value is null and there are no more values
+      */
+    @edu.umd.cs.findbugs.annotations.SuppressWarnings(
+            value="NP_BOOLEAN_RETURN_NULL", 
+            justification="Designed to return null.")
+    public Boolean next(ImmutableBytesWritable ptr, int position, int maxOffset, ValueBitSet valueSet) {
+        if (ptr.getOffset() + ptr.getLength() >= maxOffset) {
+            ptr.set(ptr.get(), maxOffset, 0);
+            return null;
+        }
+        if (position >= getFieldCount()) {
+            return null;
+        }
+        // Move the pointer past the current value and set length
+        // to 0 to ensure you never set the ptr past the end of the
+        // backing byte array.
+        ptr.set(ptr.get(), ptr.getOffset() + ptr.getLength(), 0);
+        if (!isNull(position, valueSet)) {
+            Field field = this.getField(position);
+            if (field.getDataType().isFixedWidth()) {
+                ptr.set(ptr.get(),ptr.getOffset(), field.getByteSize());
+            } else {
+                int length = ByteUtil.vintFromBytes(ptr);
+                ptr.set(ptr.get(),ptr.getOffset(),length);
+            }
+            return ptr.getLength() > 0;
+        }
+        return false;
+    }
+}


[30/51] [partial] Initial commit of master branch from github

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/DistinctValueWithCountServerAggregator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/DistinctValueWithCountServerAggregator.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/DistinctValueWithCountServerAggregator.java
new file mode 100644
index 0000000..ef0a9c2
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/DistinctValueWithCountServerAggregator.java
@@ -0,0 +1,181 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.aggregator;
+
+import java.io.ByteArrayOutputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.io.hfile.Compression;
+import org.apache.hadoop.hbase.io.hfile.Compression.Algorithm;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.SizedUtil;
+
+/**
+ * Server side Aggregator which will aggregate data and find distinct values with number of occurrences for each.
+ * 
+ * @author anoopsjohn
+ * @since 1.2.1
+ */
+public class DistinctValueWithCountServerAggregator extends BaseAggregator {
+    private static final Logger LOG = LoggerFactory.getLogger(DistinctValueWithCountServerAggregator.class);
+    public static final int DEFAULT_ESTIMATED_DISTINCT_VALUES = 10000;
+    public static final byte[] COMPRESS_MARKER = new byte[] { (byte)1 };
+    public static final Algorithm COMPRESS_ALGO = Compression.Algorithm.SNAPPY;
+
+    private int compressThreshold;
+    private byte[] buffer = null;
+    private Map<ImmutableBytesPtr, Integer> valueVsCount = new HashMap<ImmutableBytesPtr, Integer>();
+
+    public DistinctValueWithCountServerAggregator(Configuration conf) {
+        super(null);
+        compressThreshold = conf.getInt(QueryServices.DISTINCT_VALUE_COMPRESS_THRESHOLD_ATTRIB,
+                QueryServicesOptions.DEFAULT_DISTINCT_VALUE_COMPRESS_THRESHOLD);
+    }
+
+    public DistinctValueWithCountServerAggregator(Configuration conf, DistinctValueWithCountClientAggregator clientAgg) {
+        this(conf);
+        valueVsCount = clientAgg.valueVsCount;
+    }
+
+    @Override
+    public void aggregate(Tuple tuple, ImmutableBytesWritable ptr) {
+        ImmutableBytesPtr key = new ImmutableBytesPtr(ptr.get(), ptr.getOffset(), ptr.getLength());
+        Integer count = this.valueVsCount.get(key);
+        if (count == null) {
+            this.valueVsCount.put(key, 1);
+        } else {
+            this.valueVsCount.put(key, ++count);
+        }
+    }
+
+    @Override
+    public boolean isNullable() {
+        return false;
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        // This serializes the Map. The format is as follows
+        // Map size(VInt ie. 1 to 5 bytes) +
+        // ( key length [VInt ie. 1 to 5 bytes] + key bytes + value [VInt ie. 1 to 5 bytes] )*
+        int serializationSize = countMapSerializationSize();
+        buffer = new byte[serializationSize];
+        int offset = 1;
+        offset += ByteUtil.vintToBytes(buffer, offset, this.valueVsCount.size());
+        for (Entry<ImmutableBytesPtr, Integer> entry : this.valueVsCount.entrySet()) {
+            ImmutableBytesPtr key = entry.getKey();
+            offset += ByteUtil.vintToBytes(buffer, offset, key.getLength());
+            System.arraycopy(key.get(), key.getOffset(), buffer, offset, key.getLength());
+            offset += key.getLength();
+            offset += ByteUtil.vintToBytes(buffer, offset, entry.getValue().intValue());
+        }
+        if (serializationSize > compressThreshold) {
+            // The size for the map serialization is above the threshold. We will do the Snappy compression here.
+            ByteArrayOutputStream compressedByteStream = new ByteArrayOutputStream();
+            try {
+                compressedByteStream.write(COMPRESS_MARKER);
+                OutputStream compressionStream = COMPRESS_ALGO.createCompressionStream(compressedByteStream,
+                        COMPRESS_ALGO.getCompressor(), 0);
+                compressionStream.write(buffer, 1, buffer.length - 1);
+                compressionStream.flush();
+                ptr.set(compressedByteStream.toByteArray(), 0, compressedByteStream.size());
+                return true;
+            } catch (Exception e) {
+                LOG.error("Exception while Snappy compression of data.", e);
+            }
+        }
+        ptr.set(buffer, 0, offset);
+        return true;
+    }
+
+    // The #bytes required to serialize the count map.
+    // Here let us assume to use 4 bytes for each of the int items. Normally it will consume lesser
+    // bytes as we will use vints.
+    // TODO Do we need to consider 5 as the number of bytes for each of the int field? Else there is
+    // a chance of ArrayIndexOutOfBoundsException when all the int fields are having very large
+    // values. Will that ever occur?
+    private int countMapSerializationSize() {
+        int size = Bytes.SIZEOF_INT;// Write the number of entries in the Map
+        for (ImmutableBytesPtr key : this.valueVsCount.keySet()) {
+            // Add up the key and key's lengths (Int) and the value
+            size += key.getLength() + Bytes.SIZEOF_INT + Bytes.SIZEOF_INT;
+        }
+        return size;
+    }
+
+    // The heap size which will be taken by the count map.
+    private int countMapHeapSize() {
+        int size = 0;
+        if (this.valueVsCount.size() > 0) {
+            for (ImmutableBytesPtr key : this.valueVsCount.keySet()) {
+                size += SizedUtil.MAP_ENTRY_SIZE + // entry
+                        Bytes.SIZEOF_INT + // key size
+                        key.getLength() + SizedUtil.ARRAY_SIZE; // value size
+            }
+        } else {
+            // Initially when the getSize() is called, we dont have any entries in the map so as to
+            // tell the exact heap need. Let us approximate the #entries
+            SizedUtil.sizeOfMap(DEFAULT_ESTIMATED_DISTINCT_VALUES,
+                    SizedUtil.IMMUTABLE_BYTES_PTR_SIZE, Bytes.SIZEOF_INT);
+        }
+        return size;
+    }
+
+    @Override
+    public final PDataType getDataType() {
+        return PDataType.VARBINARY;
+    }
+
+    @Override
+    public void reset() {
+        valueVsCount = new HashMap<ImmutableBytesPtr, Integer>();
+        buffer = null;
+        super.reset();
+    }
+
+    @Override
+    public String toString() {
+        return "DISTINCT VALUE vs COUNT";
+    }
+
+    @Override
+    public int getSize() {
+        // TODO make this size correct.??
+        // This size is being called initially at the begin of the scanner open. At that time we any
+        // way can not tell the exact size of the Map. The Aggregators get size from all Aggregator
+        // and stores in a variable for future use. This size of the Aggregators is being used in
+        // Grouped unordered scan. Do we need some changes there in that calculation?
+        return super.getSize() + SizedUtil.ARRAY_SIZE + countMapHeapSize();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/DoubleSumAggregator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/DoubleSumAggregator.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/DoubleSumAggregator.java
new file mode 100644
index 0000000..fa1dda7
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/DoubleSumAggregator.java
@@ -0,0 +1,94 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.aggregator;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.SizedUtil;
+
+public class DoubleSumAggregator extends BaseAggregator {
+    
+    private double sum = 0;
+    private byte[] buffer;
+
+    public DoubleSumAggregator(ColumnModifier columnModifier, ImmutableBytesWritable ptr) {
+        super(columnModifier);
+        if (ptr != null) {
+            initBuffer();
+            sum = PDataType.DOUBLE.getCodec().decodeDouble(ptr, columnModifier);
+        }
+    }
+    
+    protected PDataType getInputDataType() {
+        return PDataType.DOUBLE;
+    }
+    
+    private void initBuffer() {
+        buffer = new byte[getDataType().getByteSize()];
+    }
+
+    @Override
+    public void aggregate(Tuple tuple, ImmutableBytesWritable ptr) {
+        double value = getInputDataType().getCodec().decodeDouble(ptr, columnModifier);
+        sum += value;
+        if (buffer == null) {
+            initBuffer();
+        }
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        if (buffer == null) {
+            if (isNullable()) {
+                return false;
+            }
+            initBuffer();
+        }
+        ptr.set(buffer);
+        getDataType().getCodec().encodeDouble(sum, ptr);
+        return true;
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return PDataType.DOUBLE;
+    }
+    
+    @Override
+    public String toString() {
+        return "SUM [sum=" + sum + "]";
+    }
+    
+    @Override
+    public void reset() {
+        sum = 0;
+        buffer = null;
+        super.reset();
+    }
+    
+    @Override
+    public int getSize() {
+        return super.getSize() + SizedUtil.LONG_SIZE + SizedUtil.ARRAY_SIZE + getDataType().getByteSize();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/IntSumAggregator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/IntSumAggregator.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/IntSumAggregator.java
new file mode 100644
index 0000000..e96a993
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/IntSumAggregator.java
@@ -0,0 +1,43 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.aggregator;
+
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.PDataType;
+
+/**
+ * 
+ * Aggregator that sums integer values
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class IntSumAggregator extends NumberSumAggregator {
+    
+    public IntSumAggregator(ColumnModifier columnModifier) {
+        super(columnModifier);
+    }
+    
+    @Override
+    protected PDataType getInputDataType() {
+        return PDataType.INTEGER;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/LongSumAggregator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/LongSumAggregator.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/LongSumAggregator.java
new file mode 100644
index 0000000..bfdadc9
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/LongSumAggregator.java
@@ -0,0 +1,42 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.aggregator;
+
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.PDataType;
+
+/**
+ * 
+ * Aggregator that sums long values
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class LongSumAggregator extends NumberSumAggregator {
+    
+    public LongSumAggregator(ColumnModifier columnModifier) {
+        super(columnModifier);
+    }
+    
+    @Override
+    protected PDataType getInputDataType() {
+        return PDataType.LONG;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/MaxAggregator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/MaxAggregator.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/MaxAggregator.java
new file mode 100644
index 0000000..890e14a
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/MaxAggregator.java
@@ -0,0 +1,50 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.aggregator;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import org.apache.phoenix.schema.ColumnModifier;
+
+
+/**
+ * Aggregator that finds the max of values. Inverse of {@link MinAggregator}.
+ *
+ * @author syyang
+ * @since 0.1
+ */
+abstract public class MaxAggregator extends MinAggregator {
+    
+    public MaxAggregator(ColumnModifier columnModifier) {
+        super(columnModifier);
+    }
+    
+    @Override
+    protected boolean keepFirst(ImmutableBytesWritable ibw1, ImmutableBytesWritable ibw2) {
+        return !super.keepFirst(ibw1, ibw2);
+    }
+    
+    @Override
+    public String toString() {
+        return "MAX [value=" + Bytes.toStringBinary(value.get(),value.getOffset(),value.getLength()) + "]";
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/MinAggregator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/MinAggregator.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/MinAggregator.java
new file mode 100644
index 0000000..8954de2
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/MinAggregator.java
@@ -0,0 +1,98 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.aggregator;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.SizedUtil;
+
+
+/**
+ * Aggregator that finds the min of values. Inverse of {@link MaxAggregator}.
+ *
+ * @author syyang
+ * @since 0.1
+ */
+abstract public class MinAggregator extends BaseAggregator {
+    /** Used to store the accumulate the results of the MIN function */
+    protected final ImmutableBytesWritable value = new ImmutableBytesWritable(ByteUtil.EMPTY_BYTE_ARRAY);
+    
+    public MinAggregator(ColumnModifier columnModifier) {
+        super(columnModifier);
+    }
+
+    @Override
+    public void reset() {
+        value.set(ByteUtil.EMPTY_BYTE_ARRAY);
+        super.reset();
+    }
+
+    @Override
+    public int getSize() {
+        return super.getSize() + /*value*/ SizedUtil.IMMUTABLE_BYTES_WRITABLE_SIZE;
+    }
+
+    /**
+     * Compares two bytes writables, and returns true if the first one should be
+     * kept, and false otherwise. For the MIN function, this method will return
+     * true if the first bytes writable is less than the second.
+     * 
+     * @param ibw1 the first bytes writable
+     * @param ibw2 the second bytes writable
+     * @return true if the first bytes writable should be kept
+     */
+    protected boolean keepFirst(ImmutableBytesWritable ibw1, ImmutableBytesWritable ibw2) {
+        return 0 >= getDataType().compareTo(ibw1, columnModifier, ibw2, columnModifier, getDataType());
+    }
+
+    private boolean isNull() {
+        return value.get() == ByteUtil.EMPTY_BYTE_ARRAY;
+    }
+    
+    @Override
+    public void aggregate(Tuple tuple, ImmutableBytesWritable ptr) {
+        if (isNull()) {
+            value.set(ptr.get(), ptr.getOffset(), ptr.getLength());
+        } else {
+            if (!keepFirst(value, ptr)) {
+                // replace the value with the new value
+                value.set(ptr.get(), ptr.getOffset(), ptr.getLength());
+            }
+        }
+    }
+    
+    @Override
+    public String toString() {
+        return "MIN [value=" + Bytes.toStringBinary(value.get(),value.getOffset(),value.getLength()) + "]";
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        if (isNull()) {
+            return false;
+        }
+        ptr.set(value.get(), value.getOffset(), value.getLength());
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/NumberSumAggregator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/NumberSumAggregator.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/NumberSumAggregator.java
new file mode 100644
index 0000000..a7cfcd1
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/NumberSumAggregator.java
@@ -0,0 +1,114 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.aggregator;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.SizedUtil;
+
+/**
+ * 
+ * Aggregator that sums integral number values
+ * 
+ * @author jtaylor
+ * @since 0.1
+ */
+abstract public class NumberSumAggregator extends BaseAggregator {
+    private long sum = 0;
+    private byte[] buffer;
+
+    public NumberSumAggregator(ColumnModifier columnModifier) {
+        super(columnModifier);
+    }
+
+    public NumberSumAggregator(ColumnModifier columnModifier,
+            ImmutableBytesWritable ptr) {
+        this(columnModifier);
+        if (ptr != null) {
+            initBuffer();
+            sum = PDataType.LONG.getCodec().decodeLong(ptr, columnModifier);
+        }
+    }
+
+    public long getSum() {
+        return sum;
+    }
+
+    abstract protected PDataType getInputDataType();
+
+    private int getBufferLength() {
+        return getDataType().getByteSize();
+    }
+
+    private void initBuffer() {
+        buffer = new byte[getBufferLength()];
+    }
+
+    @Override
+    public void aggregate(Tuple tuple, ImmutableBytesWritable ptr) {
+        // Get either IntNative or LongNative depending on input type
+        long value = getInputDataType().getCodec().decodeLong(ptr,
+                columnModifier);
+        sum += value;
+        if (buffer == null) {
+            initBuffer();
+        }
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        if (buffer == null) {
+            if (isNullable()) {
+                return false;
+            }
+            initBuffer();
+        }
+        ptr.set(buffer);
+        getDataType().getCodec().encodeLong(sum, ptr);
+        return true;
+    }
+
+    @Override
+    public final PDataType getDataType() {
+        return PDataType.LONG;
+    }
+
+    @Override
+    public void reset() {
+        sum = 0;
+        buffer = null;
+        super.reset();
+    }
+
+    @Override
+    public String toString() {
+        return "SUM [sum=" + sum + "]";
+    }
+
+    @Override
+    public int getSize() {
+        return super.getSize() + SizedUtil.LONG_SIZE + SizedUtil.ARRAY_SIZE
+                + getBufferLength();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/PercentRankClientAggregator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/PercentRankClientAggregator.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/PercentRankClientAggregator.java
new file mode 100644
index 0000000..42ca267
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/PercentRankClientAggregator.java
@@ -0,0 +1,92 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.aggregator;
+
+import java.math.BigDecimal;
+import java.util.*;
+import java.util.Map.Entry;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.expression.*;
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+/**
+ * Client side Aggregator for PERCENT_RANK aggregations
+ * 
+ * @author anoopsjohn
+ * @since 1.2.1
+ */
+public class PercentRankClientAggregator extends DistinctValueWithCountClientAggregator {
+
+    private final List<Expression> exps;
+    private BigDecimal cachedResult = null;
+
+    public PercentRankClientAggregator(List<Expression> exps, ColumnModifier columnModifier) {
+        super(columnModifier);
+        this.exps = exps;
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        if (cachedResult == null) {
+            ColumnExpression columnExp = (ColumnExpression)exps.get(0);
+            // Second exp will be a LiteralExpression of Boolean type indicating whether the ordering to
+            // be ASC/DESC
+            LiteralExpression isAscendingExpression = (LiteralExpression)exps.get(1);
+            boolean isAscending = (Boolean)isAscendingExpression.getValue();
+
+            // Third expression will be LiteralExpression
+            LiteralExpression valueExp = (LiteralExpression)exps.get(2);
+            Map<Object, Integer> sorted = getSortedValueVsCount(isAscending, columnExp.getDataType());
+            long distinctCountsSum = 0;
+            Object value = valueExp.getValue();
+            for (Entry<Object, Integer> entry : sorted.entrySet()) {
+                Object colValue = entry.getKey();
+                int compareResult = columnExp.getDataType().compareTo(colValue, value, valueExp.getDataType());
+                boolean done = isAscending ? compareResult > 0 : compareResult <= 0;
+                if (done) break;
+                distinctCountsSum += entry.getValue();
+            }
+
+            float result = (float)distinctCountsSum / totalCount;
+            this.cachedResult = new BigDecimal(result);
+        }
+        if (buffer == null) {
+            initBuffer();
+        }
+        buffer = PDataType.DECIMAL.toBytes(this.cachedResult);
+        ptr.set(buffer);
+        return true;
+    }
+
+    @Override
+    protected int getBufferLength() {
+        return PDataType.DECIMAL.getByteSize();
+    }
+    
+    @Override
+    public void reset() {
+        super.reset();
+        this.cachedResult = null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/PercentileClientAggregator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/PercentileClientAggregator.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/PercentileClientAggregator.java
new file mode 100644
index 0000000..095842a
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/PercentileClientAggregator.java
@@ -0,0 +1,110 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.aggregator;
+
+import java.math.BigDecimal;
+import java.util.*;
+import java.util.Map.Entry;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.expression.*;
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+/**
+ * Client side Aggregator for PERCENTILE_CONT aggregations
+ * 
+ * @author anoopsjohn
+ * @since 1.2.1
+ */
+public class PercentileClientAggregator extends DistinctValueWithCountClientAggregator {
+
+    private final List<Expression> exps;
+    private BigDecimal cachedResult = null;
+
+    public PercentileClientAggregator(List<Expression> exps, ColumnModifier columnModifier) {
+        super(columnModifier);
+        this.exps = exps;
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        if (cachedResult == null) {
+            ColumnExpression columnExp = (ColumnExpression)exps.get(0);
+            // Second exp will be a LiteralExpression of Boolean type indicating whether the ordering to
+            // be ASC/DESC
+            LiteralExpression isAscendingExpression = (LiteralExpression)exps.get(1);
+            boolean isAscending = (Boolean)isAscendingExpression.getValue();
+
+            // Third expression will be LiteralExpression
+            LiteralExpression percentileExp = (LiteralExpression)exps.get(2);
+            float p = ((Number)percentileExp.getValue()).floatValue();
+            Map<Object, Integer> sorted = getSortedValueVsCount(isAscending, columnExp.getDataType());
+            float i = (p * this.totalCount) + 0.5F;
+            long k = (long)i;
+            float f = i - k;
+            Object o1 = null;
+            Object o2 = null;
+            long distinctCountsSum = 0;
+            for (Entry<Object, Integer> entry : sorted.entrySet()) {
+                if (o1 != null) {
+                    o2 = entry.getKey();
+                    break;
+                }
+                distinctCountsSum += entry.getValue();
+                if (distinctCountsSum == k) {
+                    o1 = entry.getKey();
+                } else if (distinctCountsSum > k) {
+                    o1 = o2 = entry.getKey();
+                    break;
+                }
+            }
+
+            double result = 0.0;
+            Number n1 = (Number)o1;
+            if (o2 == null || o1 == o2) {
+                result = n1.doubleValue();
+            } else {
+                Number n2 = (Number)o2;
+                result = (n1.doubleValue() * (1.0F - f)) + (n2.doubleValue() * f);
+            }
+            this.cachedResult = new BigDecimal(result);
+        }
+        if (buffer == null) {
+            initBuffer();
+        }
+        buffer = PDataType.DECIMAL.toBytes(this.cachedResult);
+        ptr.set(buffer);
+        return true;
+    }
+
+    @Override
+    protected int getBufferLength() {
+        return PDataType.DECIMAL.getByteSize();
+    }
+    
+    @Override
+    public void reset() {
+        super.reset();
+        this.cachedResult = null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/PercentileDiscClientAggregator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/PercentileDiscClientAggregator.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/PercentileDiscClientAggregator.java
new file mode 100644
index 0000000..206ed1d
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/PercentileDiscClientAggregator.java
@@ -0,0 +1,106 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.aggregator;
+
+import java.util.*;
+import java.util.Map.Entry;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.expression.*;
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+/**
+ * 
+ * Built-in function for PERCENTILE_DISC(<expression>) WITHIN GROUP (ORDER BY <expression> ASC/DESC) aggregate function
+ *
+ * @author ramkrishna
+ * @since 1.2.1
+ */
+public class PercentileDiscClientAggregator extends
+		DistinctValueWithCountClientAggregator {
+
+	private final List<Expression> exps;
+	private Object cachedResult = null;
+	ColumnExpression columnExp = null;
+
+	public PercentileDiscClientAggregator(List<Expression> exps, ColumnModifier columnModifier) {
+	    super(columnModifier);
+		this.exps = exps;
+	}
+
+	@Override
+	public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+		// Reset buffer so that it gets initialized with the current datatype of the column
+		buffer = null;
+		if (cachedResult == null) {
+			columnExp = (ColumnExpression)exps.get(0);
+			// Second exp will be a LiteralExpression of Boolean type indicating
+			// whether the ordering to be ASC/DESC
+			LiteralExpression isAscendingExpression = (LiteralExpression) exps
+					.get(1);
+			boolean isAscending = (Boolean) isAscendingExpression.getValue();
+
+			// Third expression will be LiteralExpression
+			LiteralExpression percentileExp = (LiteralExpression) exps.get(2);
+			float p = ((Number) percentileExp.getValue()).floatValue();
+			Map<Object, Integer> sorted = getSortedValueVsCount(isAscending, columnExp.getDataType());
+			int currValue = 0;
+			Object result = null;
+			// Here the Percentile_disc returns the cum_dist() that is greater or equal to the
+			// Percentile (p) specified in the query.  So the result set will be of that of the
+			// datatype of the column being selected
+			for (Entry<Object, Integer> entry : sorted.entrySet()) {
+				result = entry.getKey();
+				Integer value = entry.getValue();
+				currValue += value;
+				float cum_dist = (float) currValue / (float) totalCount;
+				if (cum_dist >= p) {
+					break;
+				}
+			}
+			this.cachedResult = result;
+		}
+		if (buffer == null) {
+			// Initialize based on the datatype
+			// columnExp cannot be null
+			buffer = new byte[columnExp.getDataType().getByteSize()];
+		}
+		// Copy the result to the buffer.
+		System.arraycopy(columnExp.getDataType().toBytes(this.cachedResult), 0, buffer, 0, buffer.length);
+		ptr.set(buffer);
+		return true;
+	}
+
+	@Override
+	public void reset() {
+		super.reset();
+		this.cachedResult = null;
+	}
+
+	@Override
+	protected int getBufferLength() {
+		// Will be used in the aggregate() call
+		return PDataType.DECIMAL.getByteSize();
+	}
+	
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/ServerAggregators.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/ServerAggregators.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/ServerAggregators.java
new file mode 100644
index 0000000..6457793
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/ServerAggregators.java
@@ -0,0 +1,145 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.aggregator;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.WritableUtils;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.ExpressionType;
+import org.apache.phoenix.expression.function.SingleAggregateFunction;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+
+/**
+ * 
+ * Aggregators that execute on the server-side
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class ServerAggregators extends Aggregators {
+    public static final ServerAggregators EMPTY_AGGREGATORS = new ServerAggregators(new SingleAggregateFunction[0], new Aggregator[0], new Expression[0], 0);
+    private final Expression[] expressions;
+    
+    private ServerAggregators(SingleAggregateFunction[] functions, Aggregator[] aggregators, Expression[] expressions, int minNullableIndex) {
+        super(functions, aggregators, minNullableIndex);
+        if (aggregators.length != expressions.length) {
+            throw new IllegalArgumentException("Number of aggregators (" + aggregators.length 
+                    + ") must match the number of expressions (" + Arrays.toString(expressions) + ")");
+        }
+        this.expressions = expressions;
+    }
+    
+    @Override
+    public void aggregate(Aggregator[] aggregators, Tuple result) {
+        for (int i = 0; i < expressions.length; i++) {
+            if (expressions[i].evaluate(result, ptr)) {
+                aggregators[i].aggregate(result, ptr);
+            }
+        }
+    }
+    
+    /**
+     * Serialize an Aggregator into a byte array
+     * @param aggFuncs list of aggregator to serialize
+     * @return serialized byte array respresentation of aggregator
+     */
+    public static byte[] serialize(List<SingleAggregateFunction> aggFuncs, int minNullableIndex) {
+        ByteArrayOutputStream stream = new ByteArrayOutputStream();
+        try {
+            DataOutputStream output = new DataOutputStream(stream);
+            WritableUtils.writeVInt(output, minNullableIndex);
+            WritableUtils.writeVInt(output, aggFuncs.size());
+            for (int i = 0; i < aggFuncs.size(); i++) {
+                SingleAggregateFunction aggFunc = aggFuncs.get(i);
+                WritableUtils.writeVInt(output, ExpressionType.valueOf(aggFunc).ordinal());
+                aggFunc.write(output);
+            }
+            return stream.toByteArray();
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        } finally {
+            try {
+                stream.close();
+            } catch (IOException e) {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+
+    @Override
+    public Aggregator[] newAggregators() {
+        return newAggregators(null);
+    }
+
+    public Aggregator[] newAggregators(Configuration conf) {
+        Aggregator[] aggregators = new Aggregator[functions.length];
+        for (int i = 0; i < functions.length; i++) {
+            aggregators[i] = functions[i].newServerAggregator(conf);
+        }
+        return aggregators;
+    }
+
+    /**
+     * Deserialize aggregators from the serialized byte array representation
+     * @param b byte array representation of a list of Aggregators
+     * @param conf Server side configuration used by HBase
+     * @return newly instantiated Aggregators instance
+     */
+    public static ServerAggregators deserialize(byte[] b, Configuration conf) {
+        if (b == null) {
+            return ServerAggregators.EMPTY_AGGREGATORS;
+        }
+        ByteArrayInputStream stream = new ByteArrayInputStream(b);
+        try {
+            DataInputStream input = new DataInputStream(stream);
+            int minNullableIndex = WritableUtils.readVInt(input);
+            int len = WritableUtils.readVInt(input);
+            Aggregator[] aggregators = new Aggregator[len];
+            Expression[] expressions = new Expression[len];
+            SingleAggregateFunction[] functions = new SingleAggregateFunction[len];
+            for (int i = 0; i < aggregators.length; i++) {
+                SingleAggregateFunction aggFunc = (SingleAggregateFunction)ExpressionType.values()[WritableUtils.readVInt(input)].newInstance();
+                aggFunc.readFields(input, conf);
+                functions[i] = aggFunc;
+                aggregators[i] = aggFunc.getAggregator();
+                expressions[i] = aggFunc.getAggregatorExpression();
+            }
+            return new ServerAggregators(functions, aggregators,expressions, minNullableIndex);
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        } finally {
+            try {
+                stream.close();
+            } catch (IOException e) {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/StddevPopAggregator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/StddevPopAggregator.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/StddevPopAggregator.java
new file mode 100644
index 0000000..30276e1
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/StddevPopAggregator.java
@@ -0,0 +1,43 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.aggregator;
+
+import java.util.List;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.schema.ColumnModifier;
+
+/**
+ * Client side Aggregator for STDDEV_POP aggregations
+ * 
+ * @author anoopsjohn
+ * @since 1.2.1
+ */
+public class StddevPopAggregator extends BaseStddevAggregator {
+
+    public StddevPopAggregator(List<Expression> exps, ColumnModifier columnModifier) {
+        super(exps, columnModifier);
+    }
+
+    @Override
+    protected long getDataPointsCount() {
+        return totalCount;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/StddevSampAggregator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/StddevSampAggregator.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/StddevSampAggregator.java
new file mode 100644
index 0000000..49f52d2
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/StddevSampAggregator.java
@@ -0,0 +1,43 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.aggregator;
+
+import java.util.List;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.schema.ColumnModifier;
+
+/**
+ * Client side Aggregator for STDDEV_SAMP aggregations
+ * 
+ * @author anoopsjohn
+ * @since 1.2.1
+ */
+public class StddevSampAggregator extends BaseStddevAggregator {
+
+    public StddevSampAggregator(List<Expression> exps, ColumnModifier columnModifier) {
+        super(exps, columnModifier);
+    }
+
+    @Override
+    protected long getDataPointsCount() {
+        return totalCount - 1;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/UnsignedIntSumAggregator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/UnsignedIntSumAggregator.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/UnsignedIntSumAggregator.java
new file mode 100644
index 0000000..c8befca
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/UnsignedIntSumAggregator.java
@@ -0,0 +1,43 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.aggregator;
+
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.PDataType;
+
+/**
+ * 
+ * Aggregator that sums unsigned integer values
+ *
+ * @author jtaylor
+ * @since 0.12
+ */
+public class UnsignedIntSumAggregator extends NumberSumAggregator {
+    
+    public UnsignedIntSumAggregator(ColumnModifier columnModifier) {
+        super(columnModifier);
+    }
+    
+    @Override
+    protected PDataType getInputDataType() {
+        return PDataType.UNSIGNED_INT;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/UnsignedLongSumAggregator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/UnsignedLongSumAggregator.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/UnsignedLongSumAggregator.java
new file mode 100644
index 0000000..b91a934
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/UnsignedLongSumAggregator.java
@@ -0,0 +1,44 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.aggregator;
+
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.PDataType;
+
+/**
+ * 
+ * Aggregator that sums unsigned long values
+ * TODO: create these classes dynamically based on the type passed through
+ *
+ * @author jtaylor
+ * @since 0.12
+ */
+public class UnsignedLongSumAggregator extends NumberSumAggregator {
+    
+    public UnsignedLongSumAggregator(ColumnModifier columnModifier) {
+        super(columnModifier);
+    }
+    
+    @Override
+    protected PDataType getInputDataType() {
+        return PDataType.UNSIGNED_LONG;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/AggregateFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/AggregateFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/AggregateFunction.java
new file mode 100644
index 0000000..72809f5
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/AggregateFunction.java
@@ -0,0 +1,54 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import java.util.List;
+
+import org.apache.phoenix.expression.Expression;
+
+
+
+
+/**
+ * 
+ * Compiled representation of a built-in aggregate function 
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+abstract public class AggregateFunction extends FunctionExpression {
+
+    public AggregateFunction() {
+    }
+
+    public AggregateFunction(List<Expression> children) {
+        super(children);
+    }
+    
+    @Override
+    public boolean isStateless() {
+        return false;
+    }
+
+    @Override
+    public boolean isDeterministic() {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..a4185e6
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayIndexFunction.java
@@ -0,0 +1,88 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import java.sql.Types;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+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.PArrayDataType;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+@BuiltInFunction(name = ArrayIndexFunction.NAME, args = {
+		@Argument(allowedTypes = { PDataType.BINARY_ARRAY,
+				PDataType.VARBINARY_ARRAY }),
+		@Argument(allowedTypes = { PDataType.INTEGER }) })
+public class ArrayIndexFunction extends ScalarFunction {
+
+	public static final String NAME = "ARRAY_ELEM";
+
+	public ArrayIndexFunction() {
+	}
+
+	public ArrayIndexFunction(List<Expression> children) {
+		super(children);
+	}
+
+	@Override
+	public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+		Expression indexExpr = children.get(1);
+		if (!indexExpr.evaluate(tuple, ptr)) {
+		  return false;
+		} else if (ptr.getLength() == 0) {
+		  return true;
+		}
+		// Use Codec to prevent Integer object allocation
+		int index = PDataType.INTEGER.getCodec().decodeInt(ptr, indexExpr.getColumnModifier());
+		if(index < 0) {
+			throw new ParseException("Index cannot be negative :" + index);
+		}
+		Expression arrayExpr = children.get(0);
+		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)
+		PArrayDataType.positionAtArrayElement(ptr, index, getDataType());
+		return true;
+		
+	}
+
+	@Override
+	public PDataType getDataType() {
+		return PDataType.fromTypeId(children.get(0).getDataType().getSqlType()
+				- Types.ARRAY);
+	}
+
+	@Override
+	public String getName() {
+		return NAME;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayLengthFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayLengthFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayLengthFunction.java
new file mode 100644
index 0000000..9a94129
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayLengthFunction.java
@@ -0,0 +1,75 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import java.sql.Types;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.parse.FunctionParseNode.Argument;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.schema.PArrayDataType;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+@BuiltInFunction(name = ArrayLengthFunction.NAME, args = { @Argument(allowedTypes = {
+		PDataType.BINARY_ARRAY, PDataType.VARBINARY_ARRAY }) })
+public class ArrayLengthFunction extends ScalarFunction {
+	public static final String NAME = "ARRAY_LENGTH";
+
+	public ArrayLengthFunction() {
+	}
+
+	public ArrayLengthFunction(List<Expression> children) {
+		super(children);
+	}
+
+	@Override
+	public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+		Expression arrayExpr = children.get(0);
+		if (!arrayExpr.evaluate(tuple, ptr)) {
+			return false;
+		} else if (ptr.getLength() == 0) {
+			return true;
+		}
+		PDataType baseType = PDataType.fromTypeId(children.get(0).getDataType()
+				.getSqlType()
+				- Types.ARRAY);
+		int length = PArrayDataType.getArrayLength(ptr, baseType);
+		byte[] lengthBuf = new byte[PDataType.INTEGER.getByteSize()];
+		PDataType.INTEGER.getCodec().encodeInt(length, lengthBuf, 0);
+		ptr.set(lengthBuf);
+		return true;
+	}
+
+	@Override
+	public PDataType getDataType() {
+		// Array length will return an Integer
+		return PDataType.INTEGER;
+	}
+
+	@Override
+	public String getName() {
+		return NAME;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/AvgAggregateFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/AvgAggregateFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/AvgAggregateFunction.java
new file mode 100644
index 0000000..62bb565
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/AvgAggregateFunction.java
@@ -0,0 +1,113 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import java.math.BigDecimal;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.LiteralExpression;
+import org.apache.phoenix.parse.*;
+import org.apache.phoenix.parse.FunctionParseNode.Argument;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+
+@BuiltInFunction(name=AvgAggregateFunction.NAME, nodeClass=AvgAggregateParseNode.class, args= {@Argument(allowedTypes={PDataType.DECIMAL})} )
+public class AvgAggregateFunction extends CompositeAggregateFunction {
+    public static final String NAME = "AVG";
+    private final CountAggregateFunction countFunc;
+    private final SumAggregateFunction sumFunc;
+    private Integer scale;
+
+    // TODO: remove when not required at built-in func register time
+    public AvgAggregateFunction(List<Expression> children) {
+        super(children);
+        this.countFunc = null;
+        this.sumFunc = null;
+        setScale(children);
+    }
+
+    public AvgAggregateFunction(List<Expression> children, CountAggregateFunction countFunc, SumAggregateFunction sumFunc) {
+        super(children);
+        this.countFunc = countFunc;
+        this.sumFunc = sumFunc;
+        setScale(children);
+    }
+
+    private void setScale(List<Expression> children) {
+        scale = PDataType.MIN_DECIMAL_AVG_SCALE; // At least 4;
+        for (Expression child: children) {
+            if (child.getScale() != null) {
+                scale = Math.max(scale, child.getScale());
+            }
+        }
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return PDataType.DECIMAL;
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        if (!countFunc.evaluate(tuple, ptr)) {
+            return false;
+        }
+        long count = countFunc.getDataType().getCodec().decodeLong(ptr, null);
+        if (count == 0) {
+            return false;
+        }
+        
+        // Normal case where a column reference was used as the argument to AVG
+        if (!countFunc.isConstantExpression()) {
+            sumFunc.evaluate(tuple, ptr);
+            BigDecimal sum = (BigDecimal)PDataType.DECIMAL.toObject(ptr, sumFunc.getDataType());
+            // For the final column projection, we divide the sum by the count, both coerced to BigDecimal.
+            // TODO: base the precision on column metadata instead of constant
+            BigDecimal avg = sum.divide(BigDecimal.valueOf(count), PDataType.DEFAULT_MATH_CONTEXT);
+            avg = avg.setScale(scale, BigDecimal.ROUND_DOWN);
+            ptr.set(PDataType.DECIMAL.toBytes(avg));
+            return true;
+        }
+        BigDecimal value = (BigDecimal) ((LiteralExpression)countFunc.getChildren().get(0)).getValue();
+        value = value.setScale(scale, BigDecimal.ROUND_DOWN);
+        ptr.set(PDataType.DECIMAL.toBytes(value));
+        return true;
+    }
+
+    @Override
+    public boolean isNullable() {
+        return sumFunc != null && sumFunc.isNullable();
+    }
+
+    @Override
+    public String getName() {
+        return NAME;
+    }
+
+    @Override
+    public Integer getScale() {
+        return scale;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CeilDateExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CeilDateExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CeilDateExpression.java
new file mode 100644
index 0000000..d0951cf
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CeilDateExpression.java
@@ -0,0 +1,77 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import com.google.common.collect.Lists;
+import org.apache.phoenix.expression.Expression;
+
+/**
+ * 
+ * Class encapsulating ceil operation on {@link org.apache.phoenix.schema.PDataType#DATE}.
+ *
+ * @author samarth.jain
+ * @since 3.0.0
+ */
+public class CeilDateExpression extends RoundDateExpression {
+    
+    public CeilDateExpression() {}
+    
+    /**
+     * @param timeUnit - unit of time to round up to.
+     * Creates a {@link CeilDateExpression} with default multiplier of 1.
+     */
+    public static Expression create(Expression expr, TimeUnit timeUnit) throws SQLException {
+        return create(expr, timeUnit, 1);
+    }
+    
+    /**
+     * @param timeUnit - unit of time to round up to
+     * @param multiplier - determines the roll up window size.
+     * Create a {@link CeilDateExpression}. 
+     */
+    public static Expression create(Expression expr, TimeUnit timeUnit, int multiplier) throws SQLException {
+        Expression timeUnitExpr = getTimeUnitExpr(timeUnit);
+        Expression defaultMultiplierExpr = getMultiplierExpr(multiplier);
+        List<Expression> expressions = Lists.newArrayList(expr, timeUnitExpr, defaultMultiplierExpr);
+        return CeilDateExpression.create(expressions);
+    }
+    
+    public static Expression create(List<Expression> children) throws SQLException {
+        return new CeilDateExpression(children);
+    }
+    
+    CeilDateExpression(List<Expression> children) {
+        super(children);
+    }
+    
+    @Override
+    protected long getRoundUpAmount() {
+        return divBy - 1;
+    }
+    
+    @Override
+    public String getName() {
+        return CeilFunction.NAME;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CeilDecimalExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CeilDecimalExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CeilDecimalExpression.java
new file mode 100644
index 0000000..dd2c22a
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CeilDecimalExpression.java
@@ -0,0 +1,77 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import java.math.RoundingMode;
+import java.sql.SQLException;
+import java.util.List;
+
+import com.google.common.collect.Lists;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.LiteralExpression;
+import org.apache.phoenix.schema.PDataType;
+
+/**
+ * 
+ * Class encapsulating the CEIL operation on a {@link org.apache.phoenix.schema.PDataType#DECIMAL}
+ *
+ * @author samarth.jain
+ * @since 3.0.0
+ */
+public class CeilDecimalExpression extends RoundDecimalExpression {
+    
+    public CeilDecimalExpression() {}
+    
+    public CeilDecimalExpression(List<Expression> children) {
+        super(children);
+    }
+    
+  /**
+   * Creates a {@link CeilDecimalExpression} with rounding scale given by @param scale.
+   *
+   */
+   public static Expression create(Expression expr, int scale) throws SQLException {
+       if (expr.getDataType().isCoercibleTo(PDataType.LONG)) {
+           return expr;
+       }
+       Expression scaleExpr = LiteralExpression.newConstant(scale, PDataType.INTEGER, true);
+       List<Expression> expressions = Lists.newArrayList(expr, scaleExpr);
+       return new CeilDecimalExpression(expressions);
+   }
+   
+   /**
+    * Creates a {@link CeilDecimalExpression} with a default scale of 0 used for rounding. 
+    *
+    */
+   public static Expression create(Expression expr) throws SQLException {
+       return create(expr, 0);
+   }
+    
+    @Override
+    protected RoundingMode getRoundingMode() {
+        return RoundingMode.CEILING;
+    }
+    
+    @Override
+    public String getName() {
+        return CeilFunction.NAME;
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CeilFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CeilFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CeilFunction.java
new file mode 100644
index 0000000..60ad329
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CeilFunction.java
@@ -0,0 +1,57 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import java.util.List;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.parse.CeilParseNode;
+import org.apache.phoenix.parse.FunctionParseNode.Argument;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.schema.PDataType;
+/**
+ * 
+ * Base class for built-in CEIL function.
+ *
+ * @author samarth.jain
+ * @since 3.0.0
+ */
+@BuiltInFunction(name = CeilFunction.NAME,
+                 nodeClass = CeilParseNode.class,
+                 args = {
+                        @Argument(allowedTypes={PDataType.TIMESTAMP, PDataType.DECIMAL}),
+                        @Argument(allowedTypes={PDataType.VARCHAR, PDataType.INTEGER}, defaultValue = "null", isConstant=true),
+                        @Argument(allowedTypes={PDataType.INTEGER}, defaultValue="1", isConstant=true)
+                        } 
+                )
+public abstract class CeilFunction extends ScalarFunction {
+    
+    public static final String NAME = "CEIL";
+    
+    public CeilFunction(List<Expression> children) {
+        super(children);
+    }
+    
+    @Override
+    public String getName() {
+        return NAME;
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CeilTimestampExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CeilTimestampExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CeilTimestampExpression.java
new file mode 100644
index 0000000..7f54cbc
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CeilTimestampExpression.java
@@ -0,0 +1,111 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import java.sql.SQLException;
+import java.sql.Timestamp;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import com.google.common.collect.Lists;
+import org.apache.phoenix.expression.CoerceExpression;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.LiteralExpression;
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.PDataType.PDataCodec;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+/**
+ * 
+ * Class encapsulating the CEIL operation on {@link org.apache.phoenix.schema.PDataType#TIMESTAMP}
+ * This class only supports CEIL {@link TimeUnit#MILLISECOND}. If you want more options of CEIL like 
+ * using {@link TimeUnit#HOUR} use {@link CeilDateExpression}
+ * 
+ * @author samarth.jain
+ * @since 3.0.0
+ */
+public class CeilTimestampExpression extends CeilDateExpression {
+    
+    public CeilTimestampExpression() {}
+    
+    private CeilTimestampExpression(List<Expression> children) {
+        super(children);
+    }
+    
+    /**
+     * Creates a {@link CeilTimestampExpression} that uses {@link TimeUnit#MILLISECOND} 
+     * as the time unit for rounding. 
+     */
+    public static CeilTimestampExpression create(Expression expr, int multiplier) throws SQLException {
+        List<Expression> childExprs = Lists.newArrayList(expr, getTimeUnitExpr(TimeUnit.MILLISECOND), getMultiplierExpr(multiplier));
+        return new CeilTimestampExpression(childExprs); 
+    }
+    
+    public static Expression create(List<Expression> children) throws SQLException {
+        Expression firstChild = children.get(0);
+        PDataType firstChildDataType = firstChild.getDataType();
+        String timeUnit = (String)((LiteralExpression)children.get(1)).getValue();
+        if(TimeUnit.MILLISECOND.toString().equalsIgnoreCase(timeUnit)) {
+            return new CeilTimestampExpression(children);
+        }
+        // Coerce TIMESTAMP to DATE, as the nanos has no affect
+        List<Expression> newChildren = Lists.newArrayListWithExpectedSize(children.size());
+        newChildren.add(CoerceExpression.create(firstChild, firstChildDataType == PDataType.TIMESTAMP ? PDataType.DATE : PDataType.UNSIGNED_DATE));
+        newChildren.addAll(children.subList(1, children.size()));
+        return CeilDateExpression.create(newChildren);
+    }
+    
+    /**
+     * Creates a {@link CeilTimestampExpression} that uses {@link TimeUnit#MILLISECOND} 
+     * as the time unit for rounding. 
+     */
+    public static CeilTimestampExpression create (Expression expr) throws SQLException {
+        return create(expr, 1);
+    }
+
+    @Override
+    protected PDataCodec getKeyRangeCodec(PDataType columnDataType) {
+        return columnDataType == PDataType.TIMESTAMP 
+                ? PDataType.DATE.getCodec() 
+                : columnDataType == PDataType.UNSIGNED_TIMESTAMP 
+                    ? PDataType.UNSIGNED_DATE.getCodec() 
+                    : super.getKeyRangeCodec(columnDataType);
+    }
+    
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        if (children.get(0).evaluate(tuple, ptr)) {
+            ColumnModifier columnModifier = children.get(0).getColumnModifier();
+            PDataType dataType = getDataType();
+            int nanos = dataType.getNanos(ptr, columnModifier);
+            if (nanos > 0) {
+                long millis = dataType.getMillis(ptr, columnModifier); 
+                Timestamp roundedTs = new Timestamp(millis + 1);
+                byte[] byteValue = dataType.toBytes(roundedTs);
+                ptr.set(byteValue);
+            }
+            return true; // for timestamp we only support rounding up the milliseconds.
+        }
+        return false;
+    }   
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CoalesceFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CoalesceFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CoalesceFunction.java
new file mode 100644
index 0000000..05f6bd8
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CoalesceFunction.java
@@ -0,0 +1,100 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.parse.FunctionParseNode.Argument;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+
+/**
+ * 
+ * Function used to provide an alternative value when the first argument is null.
+ * Usage:
+ * COALESCE(expr1,expr2)
+ * If expr1 is not null, then it is returned, otherwise expr2 is returned. 
+ *
+ * TODO: better bind parameter type matching, since arg2 must be coercible
+ * to arg1. consider allowing a common base type?
+ * @author jtaylor
+ * @since 0.1
+ */
+@BuiltInFunction(name=CoalesceFunction.NAME, args= {
+    @Argument(),
+    @Argument()} )
+public class CoalesceFunction extends ScalarFunction {
+    public static final String NAME = "COALESCE";
+
+    public CoalesceFunction() {
+    }
+
+    public CoalesceFunction(List<Expression> children) throws SQLException {
+        super(children);
+        if (!children.get(1).getDataType().isCoercibleTo(children.get(0).getDataType())) {
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_CONVERT_TYPE)
+                .setMessage(getName() + " expected " + children.get(0).getDataType() + ", but got " + children.get(1).getDataType())
+                .build().buildException();
+        }
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        if (children.get(0).evaluate(tuple, ptr)) {
+            return true;
+        }
+        return children.get(1).evaluate(tuple, ptr);
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return children.get(0).getDataType();
+    }
+
+    @Override
+    public Integer getByteSize() {
+        Integer maxByteSize1 = children.get(0).getByteSize();
+        if (maxByteSize1 != null) {
+            Integer maxByteSize2 = children.get(1).getByteSize();
+            if (maxByteSize2 != null) {
+                return maxByteSize1 > maxByteSize2 ? maxByteSize1 : maxByteSize2;
+            }
+        }
+        return null;
+    }
+
+    @Override
+    public boolean isNullable() {
+        return children.get(0).isNullable() && children.get(1).isNullable();
+    }
+
+    @Override
+    public String getName() {
+        return NAME;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CompositeAggregateFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CompositeAggregateFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CompositeAggregateFunction.java
new file mode 100644
index 0000000..2c8da08
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CompositeAggregateFunction.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import java.util.List;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.visitor.ExpressionVisitor;
+
+
+/**
+ * 
+ * Base class for aggregation functions which are composed of other
+ * aggregation functions (for example, AVG is modeled as a SUM aggregate
+ * function and a COUNT aggregate function).
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+abstract public class CompositeAggregateFunction extends AggregateFunction {
+
+    public CompositeAggregateFunction(List<Expression> children) {
+        super(children);
+    }
+    
+    @Override
+    public final <T> T accept(ExpressionVisitor<T> visitor) {
+        return null;
+    }
+}


[39/51] [partial] Initial commit of master branch from github

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/compile/PostIndexDDLCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/PostIndexDDLCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/PostIndexDDLCompiler.java
new file mode 100644
index 0000000..69108d3
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/PostIndexDDLCompiler.java
@@ -0,0 +1,119 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.compile;
+
+import java.sql.ParameterMetaData;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+
+import org.apache.phoenix.execute.MutationState;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixParameterMetaData;
+import org.apache.phoenix.schema.ColumnNotFoundException;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.util.IndexUtil;
+
+
+/**
+ * Class that compiles plan to generate initial data values after a DDL command for
+ * index table.
+ */
+public class PostIndexDDLCompiler {
+    private final PhoenixConnection connection;
+    private final TableRef dataTableRef;
+
+    public PostIndexDDLCompiler(PhoenixConnection connection, TableRef dataTableRef) {
+        this.connection = connection;
+        this.dataTableRef = dataTableRef;
+    }
+
+    public MutationPlan compile(final PTable indexTable) throws SQLException {
+        return new MutationPlan() {
+
+            @Override
+            public PhoenixConnection getConnection() {
+                return connection;
+            }
+
+            @Override
+            public ParameterMetaData getParameterMetaData() {
+                return PhoenixParameterMetaData.EMPTY_PARAMETER_META_DATA;
+            }
+
+            @Override
+            public ExplainPlan getExplainPlan() throws SQLException {
+                return ExplainPlan.EMPTY_PLAN;
+            }
+
+            @Override
+            public MutationState execute() throws SQLException {
+                boolean wasAutoCommit = connection.getAutoCommit();
+                try {
+                    connection.setAutoCommit(true);
+                    /*
+                     * Handles:
+                     * 1) Populate a newly created table with contents.
+                     * 2) Activate the index by setting the INDEX_STATE to 
+                     */
+                    // NOTE: For first version, we would use a upsert/select to populate the new index table and
+                    //   returns synchronously. Creating an index on an existing table with large amount of data
+                    //   will as a result take a very very long time.
+                    //   In the long term, we should change this to an asynchronous process to populate the index
+                    //   that would allow the user to easily monitor the process of index creation.
+                    StringBuilder indexColumns = new StringBuilder();
+                    StringBuilder dataColumns = new StringBuilder();
+                    for (PColumn col: dataTableRef.getTable().getColumns()) {
+                        String indexColName = IndexUtil.getIndexColumnName(col);
+                        try {
+                            indexTable.getColumn(indexColName);
+                            if (col.getFamilyName() != null) {
+                                dataColumns.append('"').append(col.getFamilyName()).append("\".");
+                            }
+                            dataColumns.append('"').append(col.getName()).append("\",");
+                            indexColumns.append('"').append(indexColName).append("\",");
+                        } catch (ColumnNotFoundException e) {
+                            // Catch and ignore - means that this data column is not in the index
+                        }
+                    }
+                    dataColumns.setLength(dataColumns.length()-1);
+                    indexColumns.setLength(indexColumns.length()-1);
+                    String schemaName = dataTableRef.getTable().getSchemaName().getString();
+                    String tableName = indexTable.getTableName().getString();
+                    
+                    StringBuilder updateStmtStr = new StringBuilder();
+                    updateStmtStr.append("UPSERT /*+ NO_INDEX */ INTO ").append(schemaName.length() == 0 ? "" : '"' + schemaName + "\".").append('"').append(tableName).append("\"(")
+                        .append(indexColumns).append(") SELECT ").append(dataColumns).append(" FROM ")
+                        .append(schemaName.length() == 0 ? "" : '"' + schemaName + "\".").append('"').append(dataTableRef.getTable().getTableName().getString()).append('"');
+                    PreparedStatement updateStmt = connection.prepareStatement(updateStmtStr.toString());
+                    int rowsUpdated = 0;
+                    updateStmt.execute();
+                    rowsUpdated = updateStmt.getUpdateCount();
+                    // Return number of rows built for index
+                    return new MutationState(rowsUpdated, connection);
+                } finally {
+                    if (!wasAutoCommit) connection.setAutoCommit(false);
+                }
+            }
+        };
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..13a70a9
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/ProjectionCompiler.java
@@ -0,0 +1,377 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.compile;
+
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Collections;
+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.util.Bytes;
+
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.apache.phoenix.compile.GroupByCompiler.GroupBy;
+import org.apache.phoenix.coprocessor.GroupedAggregateRegionObserver;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+import org.apache.phoenix.expression.CoerceExpression;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.aggregator.ClientAggregators;
+import org.apache.phoenix.expression.aggregator.ServerAggregators;
+import org.apache.phoenix.expression.function.SingleAggregateFunction;
+import org.apache.phoenix.expression.visitor.SingleAggregateFunctionVisitor;
+import org.apache.phoenix.parse.AliasedNode;
+import org.apache.phoenix.parse.BindParseNode;
+import org.apache.phoenix.parse.ColumnParseNode;
+import org.apache.phoenix.parse.FamilyWildcardParseNode;
+import org.apache.phoenix.parse.ParseNode;
+import org.apache.phoenix.parse.SelectStatement;
+import org.apache.phoenix.parse.SequenceValueParseNode;
+import org.apache.phoenix.parse.WildcardParseNode;
+import org.apache.phoenix.schema.ArgumentTypeMismatchException;
+import org.apache.phoenix.schema.ColumnNotFoundException;
+import org.apache.phoenix.schema.ColumnRef;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PColumnFamily;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.PDatum;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTable.ViewType;
+import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.schema.RowKeySchema;
+import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.util.IndexUtil;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.SizedUtil;
+
+
+/**
+ * 
+ * Class that iterates through expressions in SELECT clause and adds projected
+ * columns to scan.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class ProjectionCompiler {
+    
+    private ProjectionCompiler() {
+    }
+    
+    private static void projectAllColumnFamilies(PTable table, Scan scan) {
+        // Will project all known/declared column families
+        scan.getFamilyMap().clear();
+        for (PColumnFamily family : table.getColumnFamilies()) {
+            scan.addFamily(family.getName().getBytes());
+        }
+    }
+
+    private static void projectColumnFamily(PTable table, Scan scan, byte[] family) {
+        // Will project all colmuns for given CF
+        scan.addFamily(family);
+    }
+    
+    public static RowProjector compile(StatementContext context, SelectStatement statement, GroupBy groupBy) throws SQLException  {
+        return compile(context, statement, groupBy, Collections.<PColumn>emptyList());
+    }
+    
+    private static void projectAllTableColumns(StatementContext context, TableRef tableRef, List<Expression> projectedExpressions, List<ExpressionProjector> projectedColumns) throws SQLException {
+        PTable table = tableRef.getTable();
+        int posOffset = table.getBucketNum() == null ? 0 : 1;
+        // In SELECT *, don't include tenant column for tenant connection
+        if (tableRef.getTable().isMultiTenant() && context.getConnection().getTenantId() != null) {
+            posOffset++;
+        }
+        for (int i = posOffset; i < table.getColumns().size(); i++) {
+            ColumnRef ref = new ColumnRef(tableRef,i);
+            Expression expression = ref.newColumnExpression();
+            projectedExpressions.add(expression);
+            projectedColumns.add(new ExpressionProjector(ref.getColumn().getName().getString(), table.getName().getString(), expression, false));
+        }
+    }
+    
+    private static void projectAllIndexColumns(StatementContext context, TableRef tableRef, List<Expression> projectedExpressions, List<ExpressionProjector> projectedColumns) throws SQLException {
+        PTable index = tableRef.getTable();
+        PTable table = context.getConnection().getPMetaData().getTable(index.getParentName().getString());
+        int tableOffset = table.getBucketNum() == null ? 0 : 1;
+        int indexOffset = index.getBucketNum() == null ? 0 : 1;
+        if (index.getColumns().size()-indexOffset != table.getColumns().size()-tableOffset) {
+            // We'll end up not using this by the optimizer, so just throw
+            throw new ColumnNotFoundException(WildcardParseNode.INSTANCE.toString());
+        }
+        for (int i = tableOffset; i < table.getColumns().size(); i++) {
+            PColumn tableColumn = table.getColumns().get(i);
+            PColumn indexColumn = index.getColumn(IndexUtil.getIndexColumnName(tableColumn));
+            ColumnRef ref = new ColumnRef(tableRef,indexColumn.getPosition());
+            Expression expression = ref.newColumnExpression();
+            projectedExpressions.add(expression);
+            ExpressionProjector projector = new ExpressionProjector(tableColumn.getName().getString(), table.getName().getString(), expression, false);
+            projectedColumns.add(projector);
+        }
+    }
+    
+    private static void projectTableColumnFamily(StatementContext context, String cfName, TableRef tableRef, List<Expression> projectedExpressions, List<ExpressionProjector> projectedColumns) throws SQLException {
+        PTable table = tableRef.getTable();
+        PColumnFamily pfamily = table.getColumnFamily(cfName);
+        for (PColumn column : pfamily.getColumns()) {
+            ColumnRef ref = new ColumnRef(tableRef, column.getPosition());
+            Expression expression = ref.newColumnExpression();
+            projectedExpressions.add(expression);
+            projectedColumns.add(new ExpressionProjector(column.getName().toString(), table.getName()
+                    .getString(), expression, false));
+        }
+    }
+
+    private static void projectIndexColumnFamily(StatementContext context, String cfName, TableRef tableRef, List<Expression> projectedExpressions, List<ExpressionProjector> projectedColumns) throws SQLException {
+        PTable index = tableRef.getTable();
+        PTable table = context.getConnection().getPMetaData().getTable(index.getParentName().getString());
+        PColumnFamily pfamily = table.getColumnFamily(cfName);
+        for (PColumn column : pfamily.getColumns()) {
+            PColumn indexColumn = index.getColumn(IndexUtil.getIndexColumnName(column));
+            ColumnRef ref = new ColumnRef(tableRef, indexColumn.getPosition());
+            Expression expression = ref.newColumnExpression();
+            projectedExpressions.add(expression);
+            projectedColumns.add(new ExpressionProjector(column.getName().toString(), 
+                    table.getName().getString(), expression, false));
+        }
+    }
+    
+    /**
+     * Builds the projection for the scan
+     * @param context query context kept between compilation of different query clauses
+     * @param statement TODO
+     * @param groupBy compiled GROUP BY clause
+     * @param targetColumns list of columns, parallel to aliasedNodes, that are being set for an
+     * UPSERT SELECT statement. Used to coerce expression types to the expected target type.
+     * @return projector used to access row values during scan
+     * @throws SQLException 
+     */
+    public static RowProjector compile(StatementContext context, SelectStatement statement, GroupBy groupBy, List<? extends PDatum> targetColumns) throws SQLException {
+        List<AliasedNode> aliasedNodes = statement.getSelect();
+        // Setup projected columns in Scan
+        SelectClauseVisitor selectVisitor = new SelectClauseVisitor(context, groupBy);
+        List<ExpressionProjector> projectedColumns = new ArrayList<ExpressionProjector>();
+        TableRef tableRef = context.getResolver().getTables().get(0);
+        PTable table = tableRef.getTable();
+        boolean isWildcard = false;
+        Scan scan = context.getScan();
+        int index = 0;
+        List<Expression> projectedExpressions = Lists.newArrayListWithExpectedSize(aliasedNodes.size());
+        List<byte[]> projectedFamilies = Lists.newArrayListWithExpectedSize(aliasedNodes.size());
+        for (AliasedNode aliasedNode : aliasedNodes) {
+            ParseNode node = aliasedNode.getNode();
+            // TODO: visitor?
+            if (node instanceof WildcardParseNode) {
+                if (statement.isAggregate()) {
+                    ExpressionCompiler.throwNonAggExpressionInAggException(node.toString());
+                }
+                isWildcard = true;
+                if (tableRef.getTable().getType() == PTableType.INDEX && ((WildcardParseNode)node).isRewrite()) {
+                	projectAllIndexColumns(context, tableRef, projectedExpressions, projectedColumns);
+                } else {
+                    projectAllTableColumns(context, tableRef, projectedExpressions, projectedColumns);
+                }
+            } else if (node instanceof  FamilyWildcardParseNode){
+                // Project everything for SELECT cf.*
+                // TODO: support cf.* expressions for multiple tables the same way with *.
+                String cfName = ((FamilyWildcardParseNode) node).getName();
+                // Delay projecting to scan, as when any other column in the column family gets
+                // added to the scan, it overwrites that we want to project the entire column
+                // family. Instead, we do the projection at the end.
+                // TODO: consider having a ScanUtil.addColumn and ScanUtil.addFamily to work
+                // around this, as this code depends on this function being the last place where
+                // columns are projected (which is currently true, but could change).
+                projectedFamilies.add(Bytes.toBytes(cfName));
+                if (tableRef.getTable().getType() == PTableType.INDEX && ((FamilyWildcardParseNode)node).isRewrite()) {
+                    projectIndexColumnFamily(context, cfName, tableRef, projectedExpressions, projectedColumns);
+                } else {
+                    projectTableColumnFamily(context, cfName, tableRef, projectedExpressions, projectedColumns);
+                }
+            } else {
+                Expression expression = node.accept(selectVisitor);
+                projectedExpressions.add(expression);
+                if (index < targetColumns.size()) {
+                    PDatum targetColumn = targetColumns.get(index);
+                    if (targetColumn.getDataType() != expression.getDataType()) {
+                        PDataType targetType = targetColumn.getDataType();
+                        // Check if coerce allowed using more relaxed isCastableTo check, since we promote INTEGER to LONG 
+                        // during expression evaluation and then convert back to INTEGER on UPSERT SELECT (and we don't have
+                        // (an actual value we can specifically check against).
+                        if (expression.getDataType() != null && !expression.getDataType().isCastableTo(targetType)) {
+                            throw new ArgumentTypeMismatchException(targetType, expression.getDataType(), "column: " + targetColumn);
+                        }
+                        expression = CoerceExpression.create(expression, targetType);
+                    }
+                }
+                if (node instanceof BindParseNode) {
+                    context.getBindManager().addParamMetaData((BindParseNode)node, expression);
+                }
+                if (!node.isStateless()) {
+                    if (!selectVisitor.isAggregate() && statement.isAggregate()) {
+                        ExpressionCompiler.throwNonAggExpressionInAggException(expression.toString());
+                    }
+                }
+                String columnAlias = aliasedNode.getAlias() != null ? aliasedNode.getAlias() : SchemaUtil.normalizeIdentifier(aliasedNode.getNode().getAlias());
+                boolean isCaseSensitive = (columnAlias != null && (aliasedNode.isCaseSensitve() || SchemaUtil.isCaseSensitive(columnAlias)))  || selectVisitor.isCaseSensitive;
+                String name = columnAlias == null ? expression.toString() : columnAlias;
+                projectedColumns.add(new ExpressionProjector(name, table.getName().getString(), expression, isCaseSensitive));
+            }
+            selectVisitor.reset();
+            index++;
+        }
+
+        table = context.getCurrentTable().getTable(); // switch to current table for scan projection
+        // TODO make estimatedByteSize more accurate by counting the joined columns.
+        int estimatedKeySize = table.getRowKeySchema().getEstimatedValueLength();
+        int estimatedByteSize = 0;
+        for (Map.Entry<byte[],NavigableSet<byte[]>> entry : scan.getFamilyMap().entrySet()) {
+            PColumnFamily family = table.getColumnFamily(entry.getKey());
+            if (entry.getValue() == null) {
+                for (PColumn column : family.getColumns()) {
+                    Integer byteSize = column.getByteSize();
+                    estimatedByteSize += SizedUtil.KEY_VALUE_SIZE + estimatedKeySize + (byteSize == null ? RowKeySchema.ESTIMATED_VARIABLE_LENGTH_SIZE : byteSize);
+                }
+            } else {
+                for (byte[] cq : entry.getValue()) {
+                    PColumn column = family.getColumn(cq);
+                    Integer byteSize = column.getByteSize();
+                    estimatedByteSize += SizedUtil.KEY_VALUE_SIZE + estimatedKeySize + (byteSize == null ? RowKeySchema.ESTIMATED_VARIABLE_LENGTH_SIZE : byteSize);
+                }
+            }
+        }
+        
+        selectVisitor.compile();
+        // Since we don't have the empty key value in read-only tables,
+        // we must project everything.
+        boolean isProjectEmptyKeyValue = table.getType() != PTableType.VIEW && table.getViewType() != ViewType.MAPPED && !isWildcard;
+        if (isProjectEmptyKeyValue) {
+            for (byte[] family : projectedFamilies) {
+                projectColumnFamily(table, scan, family);       
+            }
+        } else {
+            /* 
+             * TODO: this could be optimized by detecting:
+             * - if a column is projected that's not in the where clause
+             * - if a column is grouped by that's not in the where clause
+             * - if we're not using IS NULL or CASE WHEN expressions
+             */
+             projectAllColumnFamilies(table,scan);
+        }
+        return new RowProjector(projectedColumns, estimatedByteSize, isProjectEmptyKeyValue);
+    }
+        
+    private static class SelectClauseVisitor extends ExpressionCompiler {
+        private static int getMinNullableIndex(List<SingleAggregateFunction> aggFuncs, boolean isUngroupedAggregation) {
+            int minNullableIndex = aggFuncs.size();
+            for (int i = 0; i < aggFuncs.size(); i++) {
+                SingleAggregateFunction aggFunc = aggFuncs.get(i);
+                if (isUngroupedAggregation ? aggFunc.getAggregator().isNullable() : aggFunc.getAggregatorExpression().isNullable()) {
+                    minNullableIndex = i;
+                    break;
+                }
+            }
+            return minNullableIndex;
+        }
+        
+        /**
+         * Track whether or not the projection expression is case sensitive. We use this
+         * information to determine whether or not we normalize the column name passed
+         */
+        private boolean isCaseSensitive;
+        private int elementCount;
+        
+        private SelectClauseVisitor(StatementContext context, GroupBy groupBy) {
+            super(context, groupBy);
+            reset();
+        }
+
+
+        /**
+         * Compiles projection by:
+         * 1) Adding RowCount aggregate function if not present when limiting rows. We need this
+         *    to track how many rows have been scanned.
+         * 2) Reordering aggregation functions (by putting fixed length aggregates first) to
+         *    optimize the positional access of the aggregated value.
+         */
+        private void compile() throws SQLException {
+            final Set<SingleAggregateFunction> aggFuncSet = Sets.newHashSetWithExpectedSize(context.getExpressionManager().getExpressionCount());
+    
+            Iterator<Expression> expressions = context.getExpressionManager().getExpressions();
+            while (expressions.hasNext()) {
+                Expression expression = expressions.next();
+                expression.accept(new SingleAggregateFunctionVisitor() {
+                    @Override
+                    public Iterator<Expression> visitEnter(SingleAggregateFunction function) {
+                        aggFuncSet.add(function);
+                        return Iterators.emptyIterator();
+                    }
+                });
+            }
+            if (aggFuncSet.isEmpty() && groupBy.isEmpty()) {
+                return;
+            }
+            List<SingleAggregateFunction> aggFuncs = new ArrayList<SingleAggregateFunction>(aggFuncSet);
+            Collections.sort(aggFuncs, SingleAggregateFunction.SCHEMA_COMPARATOR);
+    
+            int minNullableIndex = getMinNullableIndex(aggFuncs,groupBy.isEmpty());
+            context.getScan().setAttribute(GroupedAggregateRegionObserver.AGGREGATORS, ServerAggregators.serialize(aggFuncs, minNullableIndex));
+            ClientAggregators clientAggregators = new ClientAggregators(aggFuncs, minNullableIndex);
+            context.getAggregationManager().setAggregators(clientAggregators);
+        }
+        
+        @Override
+        public void reset() {
+            super.reset();
+            elementCount = 0;
+            isCaseSensitive = true;
+        }
+        
+        @Override
+        protected ColumnRef resolveColumn(ColumnParseNode node) throws SQLException {
+            ColumnRef ref = super.resolveColumn(node);
+            isCaseSensitive = isCaseSensitive && node.isCaseSensitive();
+            return ref;
+        }
+        
+        @Override
+        public void addElement(List<Expression> l, Expression element) {
+            elementCount++;
+            isCaseSensitive &= elementCount == 1;
+            super.addElement(l, element);
+        }
+        
+        @Override
+        public Expression visit(SequenceValueParseNode node) throws SQLException {
+            if (aggregateFunction != null) {
+                throw new SQLExceptionInfo.Builder(SQLExceptionCode.INVALID_USE_OF_NEXT_VALUE_FOR)
+                .setSchemaName(node.getTableName().getSchemaName())
+                .setTableName(node.getTableName().getTableName()).build().buildException();
+            }
+            return context.getSequenceManager().newSequenceReference(node);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryCompiler.java
new file mode 100644
index 0000000..bafacf9
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryCompiler.java
@@ -0,0 +1,289 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.compile;
+
+import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.phoenix.compile.GroupByCompiler.GroupBy;
+import org.apache.phoenix.compile.JoinCompiler.JoinSpec;
+import org.apache.phoenix.compile.JoinCompiler.JoinTable;
+import org.apache.phoenix.compile.JoinCompiler.JoinedTableColumnResolver;
+import org.apache.phoenix.compile.JoinCompiler.PTableWrapper;
+import org.apache.phoenix.compile.JoinCompiler.ProjectedPTableWrapper;
+import org.apache.phoenix.compile.OrderByCompiler.OrderBy;
+import org.apache.phoenix.execute.AggregatePlan;
+import org.apache.phoenix.execute.BasicQueryPlan;
+import org.apache.phoenix.execute.DegenerateQueryPlan;
+import org.apache.phoenix.execute.HashJoinPlan;
+import org.apache.phoenix.execute.ScanPlan;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.iterate.ParallelIterators.ParallelIteratorFactory;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.jdbc.PhoenixStatement;
+import org.apache.phoenix.join.HashJoinInfo;
+import org.apache.phoenix.join.ScanProjector;
+import org.apache.phoenix.parse.JoinTableNode.JoinType;
+import org.apache.phoenix.parse.ParseNode;
+import org.apache.phoenix.parse.SQLParser;
+import org.apache.phoenix.parse.SelectStatement;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.AmbiguousColumnException;
+import org.apache.phoenix.schema.ColumnNotFoundException;
+import org.apache.phoenix.schema.PDatum;
+import org.apache.phoenix.schema.PIndexState;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.util.ScanUtil;
+
+
+
+/**
+ * 
+ * Class used to build an executable query plan
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class QueryCompiler {
+    /* 
+     * Not using Scan.setLoadColumnFamiliesOnDemand(true) because we don't 
+     * want to introduce a dependency on 0.94.5 (where this feature was
+     * introduced). This will do the same thing. Once we do have a 
+     * dependency on 0.94.5 or above, switch this around.
+     */
+    private static final String LOAD_COLUMN_FAMILIES_ON_DEMAND_ATTR = "_ondemand_";
+    private final PhoenixStatement statement;
+    private final Scan scan;
+    private final Scan scanCopy;
+    private final List<? extends PDatum> targetColumns;
+    private final ParallelIteratorFactory parallelIteratorFactory;
+    
+    public QueryCompiler(PhoenixStatement statement) throws SQLException {
+        this(statement, Collections.<PDatum>emptyList(), null);
+    }
+    
+    public QueryCompiler(PhoenixStatement statement, List<? extends PDatum> targetColumns, ParallelIteratorFactory parallelIteratorFactory) throws SQLException {
+        this.statement = statement;
+        this.scan = new Scan();
+        this.targetColumns = targetColumns;
+        this.parallelIteratorFactory = parallelIteratorFactory;
+        if (statement.getConnection().getQueryServices().getLowestClusterHBaseVersion() >= PhoenixDatabaseMetaData.ESSENTIAL_FAMILY_VERSION_THRESHOLD) {
+            this.scan.setAttribute(LOAD_COLUMN_FAMILIES_ON_DEMAND_ATTR, QueryConstants.TRUE);
+        }
+        this.scanCopy = ScanUtil.newScan(scan);
+    }
+
+    /**
+     * Builds an executable query plan from a parsed SQL statement
+     * @param select parsed SQL statement
+     * @return executable query plan
+     * @throws SQLException if mismatched types are found, bind value do not match binds,
+     * or invalid function arguments are encountered.
+     * @throws SQLFeatureNotSupportedException if an unsupported construct is encountered
+     * @throws TableNotFoundException if table name not found in schema
+     * @throws ColumnNotFoundException if column name could not be resolved
+     * @throws AmbiguousColumnException if an unaliased column name is ambiguous across multiple tables
+     */
+    public QueryPlan compile(SelectStatement select) throws SQLException{
+        return compile(select, scan, false);
+    }
+    
+    protected QueryPlan compile(SelectStatement select, Scan scan, boolean asSubquery) throws SQLException{        
+        PhoenixConnection connection = statement.getConnection();
+        List<Object> binds = statement.getParameters();
+        ColumnResolver resolver = FromCompiler.getMultiTableResolver(select, connection);
+        // TODO: do this normalization outside of this so as it's not repeated by the optimizer
+        select = StatementNormalizer.normalize(select, resolver);
+        StatementContext context = new StatementContext(statement, resolver, binds, scan);
+        
+        if (select.getFrom().size() == 1)
+            return compileSingleQuery(context, select, binds);
+        
+        if (!asSubquery) {
+            SelectStatement optimized = JoinCompiler.optimize(context, select, statement);
+            if (optimized != select) {
+                select = optimized;
+                // TODO: this is a relatively expensive operation that shouldn't be
+                // done multiple times
+                resolver = FromCompiler.getMultiTableResolver(select, connection);
+                context.setResolver(resolver);
+            }
+        }
+        JoinSpec join = JoinCompiler.getJoinSpec(context, select);
+        return compileJoinQuery(context, select, binds, join, asSubquery);
+    }
+    
+    @SuppressWarnings("unchecked")
+    protected QueryPlan compileJoinQuery(StatementContext context, SelectStatement select, List<Object> binds, JoinSpec join, boolean asSubquery) throws SQLException {
+        byte[] emptyByteArray = new byte[0];
+        List<JoinTable> joinTables = join.getJoinTables();
+        if (joinTables.isEmpty()) {
+            ProjectedPTableWrapper projectedTable = join.createProjectedTable(join.getMainTable(), !asSubquery);
+            ScanProjector.serializeProjectorIntoScan(context.getScan(), JoinCompiler.getScanProjector(projectedTable));
+            context.setCurrentTable(join.getMainTable());
+            context.setResolver(JoinCompiler.getColumnResolver(projectedTable));
+            join.projectColumns(context.getScan(), join.getMainTable());
+            return compileSingleQuery(context, select, binds);
+        }
+        
+        boolean[] starJoinVector = JoinCompiler.getStarJoinVector(join);
+        if (starJoinVector != null) {
+            ProjectedPTableWrapper initialProjectedTable = join.createProjectedTable(join.getMainTable(), !asSubquery);
+            PTableWrapper projectedTable = initialProjectedTable;
+            int count = joinTables.size();
+            ImmutableBytesPtr[] joinIds = new ImmutableBytesPtr[count];
+            List<Expression>[] joinExpressions = new List[count];
+            List<Expression>[] hashExpressions = new List[count];
+            JoinType[] joinTypes = new JoinType[count];
+            PTable[] tables = new PTable[count];
+            int[] fieldPositions = new int[count];
+            QueryPlan[] joinPlans = new QueryPlan[count];
+            fieldPositions[0] = projectedTable.getTable().getColumns().size() - projectedTable.getTable().getPKColumns().size();
+            for (int i = 0; i < count; i++) {
+                JoinTable joinTable = joinTables.get(i);
+                SelectStatement subStatement = joinTable.getAsSubquery();
+                if (subStatement.getFrom().size() > 1)
+                    throw new SQLFeatureNotSupportedException("Sub queries not supported.");
+                ProjectedPTableWrapper subProjTable = join.createProjectedTable(joinTable.getTable(), false);
+                ColumnResolver resolver = JoinCompiler.getColumnResolver(subProjTable);
+                Scan subScan = ScanUtil.newScan(scanCopy);
+                ScanProjector.serializeProjectorIntoScan(subScan, JoinCompiler.getScanProjector(subProjTable));
+                StatementContext subContext = new StatementContext(statement, resolver, binds, subScan);
+                subContext.setCurrentTable(joinTable.getTable());
+                join.projectColumns(subScan, joinTable.getTable());
+                joinPlans[i] = compileSingleQuery(subContext, subStatement, binds);
+                boolean hasPostReference = join.hasPostReference(joinTable.getTable());
+                if (hasPostReference) {
+                    tables[i] = subProjTable.getTable();
+                    projectedTable = JoinCompiler.mergeProjectedTables(projectedTable, subProjTable, joinTable.getType() == JoinType.Inner);
+                } else {
+                    tables[i] = null;
+                }
+                ColumnResolver leftResolver = JoinCompiler.getColumnResolver(starJoinVector[i] ? initialProjectedTable : projectedTable);
+                joinIds[i] = new ImmutableBytesPtr(emptyByteArray); // place-holder
+                Pair<List<Expression>, List<Expression>> joinConditions = joinTable.compileJoinConditions(context, leftResolver, resolver);
+                joinExpressions[i] = joinConditions.getFirst();
+                hashExpressions[i] = joinConditions.getSecond();
+                joinTypes[i] = joinTable.getType();
+                if (i < count - 1) {
+                    fieldPositions[i + 1] = fieldPositions[i] + (tables[i] == null ? 0 : (tables[i].getColumns().size() - tables[i].getPKColumns().size()));
+                }
+            }
+            ScanProjector.serializeProjectorIntoScan(context.getScan(), JoinCompiler.getScanProjector(initialProjectedTable));
+            context.setCurrentTable(join.getMainTable());
+            context.setResolver(JoinCompiler.getColumnResolver(projectedTable));
+            join.projectColumns(context.getScan(), join.getMainTable());
+            BasicQueryPlan plan = compileSingleQuery(context, JoinCompiler.getSubqueryWithoutJoin(select, join), binds);
+            Expression postJoinFilterExpression = join.compilePostFilterExpression(context);
+            HashJoinInfo joinInfo = new HashJoinInfo(projectedTable.getTable(), joinIds, joinExpressions, joinTypes, starJoinVector, tables, fieldPositions, postJoinFilterExpression);
+            return new HashJoinPlan(plan, joinInfo, hashExpressions, joinPlans);
+        }
+        
+        JoinTable lastJoinTable = joinTables.get(joinTables.size() - 1);
+        JoinType type = lastJoinTable.getType();
+        if (type == JoinType.Full)
+            throw new SQLFeatureNotSupportedException("Full joins not supported.");
+        
+        if (type == JoinType.Right || type == JoinType.Inner) {
+            SelectStatement lhs = JoinCompiler.getSubQueryWithoutLastJoin(select, join);
+            SelectStatement rhs = JoinCompiler.getSubqueryForLastJoinTable(select, join);
+            JoinSpec lhsJoin = JoinCompiler.getSubJoinSpecWithoutPostFilters(join);
+            Scan subScan = ScanUtil.newScan(scanCopy);
+            StatementContext lhsCtx = new StatementContext(statement, context.getResolver(), binds, subScan);
+            QueryPlan lhsPlan = compileJoinQuery(lhsCtx, lhs, binds, lhsJoin, true);
+            ColumnResolver lhsResolver = lhsCtx.getResolver();
+            PTableWrapper lhsProjTable = ((JoinedTableColumnResolver) (lhsResolver)).getPTableWrapper();
+            ProjectedPTableWrapper rhsProjTable = join.createProjectedTable(lastJoinTable.getTable(), !asSubquery);
+            ColumnResolver rhsResolver = JoinCompiler.getColumnResolver(rhsProjTable);
+            ImmutableBytesPtr[] joinIds = new ImmutableBytesPtr[] {new ImmutableBytesPtr(emptyByteArray)};
+            Pair<List<Expression>, List<Expression>> joinConditions = lastJoinTable.compileJoinConditions(context, lhsResolver, rhsResolver);
+            List<Expression> joinExpressions = joinConditions.getSecond();
+            List<Expression> hashExpressions = joinConditions.getFirst();
+            int fieldPosition = rhsProjTable.getTable().getColumns().size() - rhsProjTable.getTable().getPKColumns().size();
+            PTableWrapper projectedTable = JoinCompiler.mergeProjectedTables(rhsProjTable, lhsProjTable, type == JoinType.Inner);
+            ScanProjector.serializeProjectorIntoScan(context.getScan(), JoinCompiler.getScanProjector(rhsProjTable));
+            context.setCurrentTable(lastJoinTable.getTable());
+            context.setResolver(JoinCompiler.getColumnResolver(projectedTable));
+            join.projectColumns(context.getScan(), lastJoinTable.getTable());
+            BasicQueryPlan rhsPlan = compileSingleQuery(context, rhs, binds);
+            Expression postJoinFilterExpression = join.compilePostFilterExpression(context);
+            HashJoinInfo joinInfo = new HashJoinInfo(projectedTable.getTable(), joinIds, new List[] {joinExpressions}, new JoinType[] {type == JoinType.Inner ? type : JoinType.Left}, new boolean[] {true}, new PTable[] {lhsProjTable.getTable()}, new int[] {fieldPosition}, postJoinFilterExpression);
+            return new HashJoinPlan(rhsPlan, joinInfo, new List[] {hashExpressions}, new QueryPlan[] {lhsPlan});
+        }
+        
+        // Do not support queries like "A right join B left join C" with hash-joins.
+        throw new SQLFeatureNotSupportedException("Joins with pattern 'A right join B left join C' not supported.");
+    }
+    
+    protected BasicQueryPlan compileSingleQuery(StatementContext context, SelectStatement select, List<Object> binds) throws SQLException{
+        PhoenixConnection connection = statement.getConnection();
+        ColumnResolver resolver = context.getResolver();
+        TableRef tableRef = context.getCurrentTable();
+        // Short circuit out if we're compiling an index query and the index isn't active.
+        // We must do this after the ColumnResolver resolves the table, as we may be updating the local
+        // cache of the index table and it may now be inactive.
+        if (tableRef.getTable().getType() == PTableType.INDEX && tableRef.getTable().getIndexState() != PIndexState.ACTIVE) {
+            return new DegenerateQueryPlan(context, select, tableRef);
+        }
+        PTable table = tableRef.getTable();
+        if (table.getViewStatement() != null) {
+            ParseNode viewNode = new SQLParser(table.getViewStatement()).parseQuery().getWhere();
+            // Push VIEW expression into select
+            select = select.combine(viewNode);
+        }
+        Integer limit = LimitCompiler.compile(context, select);
+
+        GroupBy groupBy = GroupByCompiler.compile(context, select);
+        // Optimize the HAVING clause by finding any group by expressions that can be moved
+        // to the WHERE clause
+        select = HavingCompiler.rewrite(context, select, groupBy);
+        Expression having = HavingCompiler.compile(context, select, groupBy);
+        // Don't pass groupBy when building where clause expression, because we do not want to wrap these
+        // expressions as group by key expressions since they're pre, not post filtered.
+        context.setResolver(FromCompiler.getResolver(select, connection));
+        WhereCompiler.compile(context, select);
+        context.setResolver(resolver); // recover resolver
+        OrderBy orderBy = OrderByCompiler.compile(context, select, groupBy, limit); 
+        RowProjector projector = ProjectionCompiler.compile(context, select, groupBy, targetColumns);
+        
+        // Final step is to build the query plan
+        int maxRows = statement.getMaxRows();
+        if (maxRows > 0) {
+            if (limit != null) {
+                limit = Math.min(limit, maxRows);
+            } else {
+                limit = maxRows;
+            }
+        }
+        if (select.isAggregate() || select.isDistinct()) {
+            return new AggregatePlan(context, select, tableRef, projector, limit, orderBy, parallelIteratorFactory, groupBy, having);
+        } else {
+            return new ScanPlan(context, select, tableRef, projector, limit, orderBy, parallelIteratorFactory);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryPlan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryPlan.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryPlan.java
new file mode 100644
index 0000000..d41fb7f
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryPlan.java
@@ -0,0 +1,69 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.compile;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.phoenix.compile.GroupByCompiler.GroupBy;
+import org.apache.phoenix.compile.OrderByCompiler.OrderBy;
+import org.apache.phoenix.iterate.ResultIterator;
+import org.apache.phoenix.parse.FilterableStatement;
+import org.apache.phoenix.query.KeyRange;
+import org.apache.phoenix.schema.TableRef;
+
+
+
+/**
+ * 
+ * Interface for an executable query plan
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public interface QueryPlan extends StatementPlan {
+    /**
+     * Get a result iterator to iterate over the results
+     * @return result iterator for iterating over the results
+     * @throws SQLException
+     */
+    public ResultIterator iterator() throws SQLException;
+    
+    public long getEstimatedSize();
+    
+    // TODO: change once joins are supported
+    TableRef getTableRef();
+    /**
+     * Returns projector used to formulate resultSet row
+     */
+    RowProjector getProjector();
+    
+    Integer getLimit();
+
+    OrderBy getOrderBy();
+
+    GroupBy getGroupBy();
+
+    List<KeyRange> getSplits();
+
+    StatementContext getContext();
+    
+    FilterableStatement getStatement();
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/compile/RowProjector.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/RowProjector.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/RowProjector.java
new file mode 100644
index 0000000..6b6344c
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/RowProjector.java
@@ -0,0 +1,124 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.compile;
+
+import java.sql.SQLException;
+import java.util.*;
+
+import com.google.common.collect.Maps;
+import org.apache.phoenix.schema.ColumnNotFoundException;
+import org.apache.phoenix.util.SchemaUtil;
+
+
+/**
+ * 
+ * Class that manages a set of projected columns accessed through the zero-based
+ * column index for a SELECT clause projection. The column index may be looked up
+ * via the name using {@link #getColumnIndex(String)}.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class RowProjector {
+    public static final RowProjector EMPTY_PROJECTOR = new RowProjector(Collections.<ColumnProjector>emptyList(),0, true);
+
+    private final List<? extends ColumnProjector> columnProjectors;
+    private final Map<String,Integer> reverseIndex;
+    private final boolean allCaseSensitive;
+    private final boolean someCaseSensitive;
+    private final int estimatedSize;
+    private final boolean isProjectEmptyKeyValue;
+    
+    public RowProjector(RowProjector projector, boolean isProjectEmptyKeyValue) {
+        this(projector.getColumnProjectors(), projector.getEstimatedRowByteSize(), isProjectEmptyKeyValue);
+    }
+    /**
+     * Construct RowProjector based on a list of ColumnProjectors.
+     * @param columnProjectors ordered list of ColumnProjectors corresponding to projected columns in SELECT clause
+     * aggregating coprocessor. Only required in the case of an aggregate query with a limit clause and otherwise may
+     * be null.
+     * @param estimatedRowSize 
+     */
+    public RowProjector(List<? extends ColumnProjector> columnProjectors, int estimatedRowSize, boolean isProjectEmptyKeyValue) {
+        this.columnProjectors = Collections.unmodifiableList(columnProjectors);
+        int position = columnProjectors.size();
+        reverseIndex = Maps.newHashMapWithExpectedSize(position);
+        boolean allCaseSensitive = true;
+        boolean someCaseSensitive = false;
+        for (--position; position >= 0; position--) {
+            ColumnProjector colProjector = columnProjectors.get(position);
+            allCaseSensitive &= colProjector.isCaseSensitive();
+            someCaseSensitive |= colProjector.isCaseSensitive();
+            reverseIndex.put(colProjector.getName(), position);
+        }
+        this.allCaseSensitive = allCaseSensitive;
+        this.someCaseSensitive = someCaseSensitive;
+        this.estimatedSize = estimatedRowSize;
+        this.isProjectEmptyKeyValue = isProjectEmptyKeyValue;
+    }
+    
+    public boolean isProjectEmptyKeyValue() {
+        return isProjectEmptyKeyValue;
+    }
+    
+    public List<? extends ColumnProjector> getColumnProjectors() {
+        return columnProjectors;
+    }
+    
+    public int getColumnIndex(String name) throws SQLException {
+        if (!someCaseSensitive) {
+            name = SchemaUtil.normalizeIdentifier(name);
+        }
+        Integer index = reverseIndex.get(name);
+        if (index == null) {
+            if (!allCaseSensitive && someCaseSensitive) {
+                name = SchemaUtil.normalizeIdentifier(name);
+                index = reverseIndex.get(name);
+            }
+            if (index == null) {
+                throw new ColumnNotFoundException(name);
+            }
+        }
+        return index;
+    }
+    
+    public ColumnProjector getColumnProjector(int index) {
+        return columnProjectors.get(index);
+    }
+ 
+    public int getColumnCount() {
+        return columnProjectors.size();
+    }
+    
+    @Override
+    public String toString() {
+        StringBuilder buf = new StringBuilder("[");
+        for (ColumnProjector projector : columnProjectors) {
+            buf.append(projector.getExpression());
+            buf.append(',');
+        }
+        buf.setCharAt(buf.length()-1, ']');
+        return buf.toString();
+    }
+
+    public int getEstimatedRowByteSize() {
+        return estimatedSize;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/compile/ScanRanges.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/ScanRanges.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/ScanRanges.java
new file mode 100644
index 0000000..44f4473
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/ScanRanges.java
@@ -0,0 +1,185 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.compile;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import org.apache.phoenix.filter.SkipScanFilter;
+import org.apache.phoenix.query.KeyRange;
+import org.apache.phoenix.schema.RowKeySchema;
+import org.apache.phoenix.util.ScanUtil;
+
+
+public class ScanRanges {
+    private static final List<List<KeyRange>> EVERYTHING_RANGES = Collections.<List<KeyRange>>emptyList();
+    private static final List<List<KeyRange>> NOTHING_RANGES = Collections.<List<KeyRange>>singletonList(Collections.<KeyRange>singletonList(KeyRange.EMPTY_RANGE));
+    public static final ScanRanges EVERYTHING = new ScanRanges(EVERYTHING_RANGES,null,false);
+    public static final ScanRanges NOTHING = new ScanRanges(NOTHING_RANGES,null,false);
+
+    public static ScanRanges create(List<List<KeyRange>> ranges, RowKeySchema schema) {
+        return create(ranges, schema, false);
+    }
+    
+    public static ScanRanges create(List<List<KeyRange>> ranges, RowKeySchema schema, boolean forceRangeScan) {
+        if (ranges.isEmpty()) {
+            return EVERYTHING;
+        } else if (ranges.size() == 1 && ranges.get(0).size() == 1 && ranges.get(0).get(0) == KeyRange.EMPTY_RANGE) {
+            return NOTHING;
+        }
+        return new ScanRanges(ranges, schema, forceRangeScan);
+    }
+
+    private SkipScanFilter filter;
+    private final List<List<KeyRange>> ranges;
+    private final RowKeySchema schema;
+    private final boolean forceRangeScan;
+
+    private ScanRanges (List<List<KeyRange>> ranges, RowKeySchema schema, boolean forceRangeScan) {
+        List<List<KeyRange>> sortedRanges = Lists.newArrayListWithExpectedSize(ranges.size());
+        for (int i = 0; i < ranges.size(); i++) {
+            List<KeyRange> sorted = Lists.newArrayList(ranges.get(i));
+            Collections.sort(sorted, KeyRange.COMPARATOR);
+            sortedRanges.add(ImmutableList.copyOf(sorted));
+        }
+        this.ranges = ImmutableList.copyOf(sortedRanges);
+        this.schema = schema;
+        if (schema != null && !ranges.isEmpty()) {
+            this.filter = new SkipScanFilter(this.ranges, schema);
+        }
+        this.forceRangeScan = forceRangeScan;
+    }
+
+    public SkipScanFilter getSkipScanFilter() {
+        return filter;
+    }
+    
+    public List<List<KeyRange>> getRanges() {
+        return ranges;
+    }
+
+    public RowKeySchema getSchema() {
+        return schema;
+    }
+
+    public boolean isEverything() {
+        return this == EVERYTHING;
+    }
+
+    public boolean isDegenerate() {
+        return this == NOTHING;
+    }
+    
+    /**
+     * Use SkipScanFilter under two circumstances:
+     * 1) If we have multiple ranges for a given key slot (use of IN)
+     * 2) If we have a range (i.e. not a single/point key) that is
+     *    not the last key slot
+     */
+    public boolean useSkipScanFilter() {
+        if (forceRangeScan) {
+            return false;
+        }
+        boolean hasRangeKey = false, useSkipScan = false;
+        for (List<KeyRange> orRanges : ranges) {
+            useSkipScan |= orRanges.size() > 1 | hasRangeKey;
+            if (useSkipScan) {
+                return true;
+            }
+            for (KeyRange range : orRanges) {
+                hasRangeKey |= !range.isSingleKey();
+            }
+        }
+        return false;
+    }
+
+    /**
+     * @return true if this represents the full key to a single row
+     */
+    public boolean isSingleRowScan() {
+        if (schema == null || ranges.size() < schema.getMaxFields()) {
+            return false;
+        }
+        boolean isSingleKey = true;
+        for (List<KeyRange> orRanges : ranges) {
+            if (orRanges.size() > 1) {
+                return false;
+            }
+            isSingleKey &= orRanges.get(0).isSingleKey();
+        }
+        return isSingleKey;
+    }
+
+    public void setScanStartStopRow(Scan scan) {
+        if (isEverything()) {
+            return;
+        }
+        if (isDegenerate()) {
+            scan.setStartRow(KeyRange.EMPTY_RANGE.getLowerRange());
+            scan.setStopRow(KeyRange.EMPTY_RANGE.getUpperRange());
+            return;
+        }
+        
+        byte[] expectedKey;
+        expectedKey = ScanUtil.getMinKey(schema, ranges);
+        if (expectedKey != null) {
+            scan.setStartRow(expectedKey);
+        }
+        expectedKey = ScanUtil.getMaxKey(schema, ranges);
+        if (expectedKey != null) {
+            scan.setStopRow(expectedKey);
+        }
+    }
+
+    public static final ImmutableBytesWritable UNBOUND = new ImmutableBytesWritable(KeyRange.UNBOUND);
+
+    /**
+     * Return true if the range formed by the lowerInclusiveKey and upperExclusiveKey
+     * intersects with any of the scan ranges and false otherwise. We cannot pass in
+     * a KeyRange here, because the underlying compare functions expect lower inclusive
+     * and upper exclusive keys. We cannot get their next key because the key must
+     * conform to the row key schema and if a null byte is added to a lower inclusive
+     * key, it's no longer a valid, real key.
+     * @param lowerInclusiveKey lower inclusive key
+     * @param upperExclusiveKey upper exclusive key
+     * @return true if the scan range intersects with the specified lower/upper key
+     * range
+     */
+    public boolean intersect(byte[] lowerInclusiveKey, byte[] upperExclusiveKey) {
+        if (isEverything()) {
+            return true;
+        }
+        if (isDegenerate()) {
+            return false;
+        }
+        return filter.hasIntersect(lowerInclusiveKey, upperExclusiveKey);
+   }
+
+    @Override
+    public String toString() {
+        return "ScanRanges[" + ranges.toString() + "]";
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/compile/SequenceManager.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/SequenceManager.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/SequenceManager.java
new file mode 100644
index 0000000..665c33b
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/SequenceManager.java
@@ -0,0 +1,194 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.compile;
+
+import java.sql.SQLException;
+import java.util.BitSet;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import org.apache.phoenix.expression.BaseTerminalExpression;
+import org.apache.phoenix.jdbc.PhoenixStatement;
+import org.apache.phoenix.parse.SequenceValueParseNode;
+import org.apache.phoenix.parse.TableName;
+import org.apache.phoenix.query.ConnectionQueryServices;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.PName;
+import org.apache.phoenix.schema.SequenceKey;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+public class SequenceManager {
+    private final PhoenixStatement statement;
+    private int[] sequencePosition;
+    private long[] srcSequenceValues;
+    private long[] dstSequenceValues;
+    private SQLException[] sqlExceptions;
+    private List<SequenceKey> nextSequences;
+    private List<SequenceKey> currentSequences;
+    private Map<SequenceKey,SequenceValueExpression> sequenceMap;
+    private BitSet isNextSequence;
+    
+    public SequenceManager(PhoenixStatement statement) {
+        this.statement = statement;
+    }
+    
+    public int getSequenceCount() {
+        return sequenceMap == null ? 0 : sequenceMap.size();
+    }
+    
+    private void setSequenceValues() throws SQLException {
+        SQLException eTop = null;
+        for (int i = 0; i < sqlExceptions.length; i++) {
+            SQLException e = sqlExceptions[i];
+            if (e != null) {
+                if (eTop == null) {
+                    eTop = e;
+                } else {
+                    e.setNextException(eTop.getNextException());
+                    eTop.setNextException(e);
+                }
+            } else {
+                dstSequenceValues[sequencePosition[i]] = srcSequenceValues[i];
+            }
+        }
+        if (eTop != null) {
+            throw eTop;
+        }
+    }
+    
+    public void incrementSequenceValues() throws SQLException {
+        if (sequenceMap == null) {
+            return;
+        }
+        Long scn = statement.getConnection().getSCN();
+        long timestamp = scn == null ? HConstants.LATEST_TIMESTAMP : scn;
+        ConnectionQueryServices services = this.statement.getConnection().getQueryServices();
+        services.incrementSequenceValues(nextSequences, timestamp, srcSequenceValues, sqlExceptions);
+        setSequenceValues();
+        int offset = nextSequences.size();
+        for (int i = 0; i < currentSequences.size(); i++) {
+            dstSequenceValues[sequencePosition[offset+i]] = services.getSequenceValue(currentSequences.get(i), timestamp);
+        }
+    }
+
+    public SequenceValueExpression newSequenceReference(SequenceValueParseNode node) {
+        if (sequenceMap == null) {
+            sequenceMap = Maps.newHashMap();
+            isNextSequence = new BitSet();
+        }
+        PName tenantName = statement.getConnection().getTenantId();
+        String tenantId = tenantName == null ? null : tenantName.getString();
+        TableName tableName = node.getTableName();
+        SequenceKey key = new SequenceKey(tenantId, tableName.getSchemaName(), tableName.getTableName());
+        SequenceValueExpression expression = sequenceMap.get(key);
+        if (expression == null) {
+            int index = sequenceMap.size();
+            expression = new SequenceValueExpression(index);
+            sequenceMap.put(key, expression);
+        }
+        // If we see a NEXT and a CURRENT, treat the CURRENT just like a NEXT
+        if (node.getOp() == SequenceValueParseNode.Op.NEXT_VALUE) {
+            isNextSequence.set(expression.getIndex());
+        }
+           
+        return expression;
+    }
+    
+    public void initSequences() throws SQLException {
+        if (sequenceMap == null) {
+            return;
+        }
+        int maxSize = sequenceMap.size();
+        dstSequenceValues = new long[maxSize];
+        sequencePosition = new int[maxSize];
+        nextSequences = Lists.newArrayListWithExpectedSize(maxSize);
+        currentSequences = Lists.newArrayListWithExpectedSize(maxSize);
+        for (Map.Entry<SequenceKey, SequenceValueExpression> entry : sequenceMap.entrySet()) {
+            if (isNextSequence.get(entry.getValue().getIndex())) {
+                nextSequences.add(entry.getKey());
+            } else {
+                currentSequences.add(entry.getKey());
+            }
+        }
+        srcSequenceValues = new long[nextSequences.size()];
+        sqlExceptions = new SQLException[nextSequences.size()];
+        Collections.sort(nextSequences);
+        // Create reverse indexes
+        for (int i = 0; i < nextSequences.size(); i++) {
+            sequencePosition[i] = sequenceMap.get(nextSequences.get(i)).getIndex();
+        }
+        int offset = nextSequences.size();
+        for (int i = 0; i < currentSequences.size(); i++) {
+            sequencePosition[i+offset] = sequenceMap.get(currentSequences.get(i)).getIndex();
+        }
+        ConnectionQueryServices services = this.statement.getConnection().getQueryServices();
+        Long scn = statement.getConnection().getSCN();
+        long timestamp = scn == null ? HConstants.LATEST_TIMESTAMP : scn;
+        services.reserveSequenceValues(nextSequences, timestamp, srcSequenceValues, sqlExceptions);
+        setSequenceValues();
+    }
+    
+    private class SequenceValueExpression extends BaseTerminalExpression {
+        private final int index;
+        private final byte[] valueBuffer = new byte[PDataType.LONG.getByteSize()];
+
+        private SequenceValueExpression(int index) {
+            this.index = index;
+        }
+
+        public int getIndex() {
+            return index;
+        }
+        
+        @Override
+        public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+            PDataType.LONG.getCodec().encodeLong(dstSequenceValues[index], valueBuffer, 0);
+            ptr.set(valueBuffer);
+            return true;
+        }
+
+        @Override
+        public PDataType getDataType() {
+            return PDataType.LONG;
+        }
+        
+        @Override
+        public boolean isNullable() {
+            return false;
+        }
+        
+        @Override
+        public boolean isDeterministic() {
+            return false;
+        }
+        
+        @Override
+        public boolean isStateless() {
+            return true;
+        }
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/compile/StatementContext.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/StatementContext.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/StatementContext.java
new file mode 100644
index 0000000..5736536
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/StatementContext.java
@@ -0,0 +1,227 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.compile;
+
+import java.sql.SQLException;
+import java.text.Format;
+import java.util.List;
+
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixStatement;
+import org.apache.phoenix.query.KeyRange;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.schema.MetaDataClient;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.util.DateUtil;
+import org.apache.phoenix.util.NumberUtil;
+import org.apache.phoenix.util.ScanUtil;
+
+
+/**
+ *
+ * Class that keeps common state used across processing the various clauses in a
+ * top level JDBC statement such as SELECT, UPSERT, DELETE, etc.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class StatementContext {
+    private ColumnResolver resolver;
+    private final BindManager binds;
+    private final Scan scan;
+    private final ExpressionManager expressions;
+    private final AggregationManager aggregates;
+    private final String dateFormat;
+    private final Format dateFormatter;
+    private final Format dateParser;
+    private final String numberFormat;
+    private final ImmutableBytesWritable tempPtr;
+    private final PhoenixStatement statement;
+    
+    private long currentTime = QueryConstants.UNSET_TIMESTAMP;
+    private ScanRanges scanRanges = ScanRanges.EVERYTHING;
+    private KeyRange minMaxRange = null;
+    private final SequenceManager sequences; 
+
+    private TableRef currentTable;
+    
+    public StatementContext(PhoenixStatement statement, ColumnResolver resolver, List<Object> binds, Scan scan) {
+        this.statement = statement;
+        this.resolver = resolver;
+        this.scan = scan;
+        this.binds = new BindManager(binds);
+        this.aggregates = new AggregationManager();
+        this.expressions = new ExpressionManager();
+        PhoenixConnection connection = statement.getConnection();
+        this.dateFormat = connection.getQueryServices().getProps().get(QueryServices.DATE_FORMAT_ATTRIB, DateUtil.DEFAULT_DATE_FORMAT);
+        this.dateFormatter = DateUtil.getDateFormatter(dateFormat);
+        this.dateParser = DateUtil.getDateParser(dateFormat);
+        this.numberFormat = connection.getQueryServices().getProps().get(QueryServices.NUMBER_FORMAT_ATTRIB, NumberUtil.DEFAULT_NUMBER_FORMAT);
+        this.tempPtr = new ImmutableBytesWritable();
+        this.currentTable = resolver != null && !resolver.getTables().isEmpty() ? resolver.getTables().get(0) : null;
+        this.sequences = new SequenceManager(statement);
+    }
+
+    public String getDateFormat() {
+        return dateFormat;
+    }
+
+    public Format getDateFormatter() {
+        return dateFormatter;
+    }
+
+    public Format getDateParser() {
+        return dateParser;
+    }
+    
+    public String getNumberFormat() {
+        return numberFormat;
+    }
+    
+    public Scan getScan() {
+        return scan;
+    }
+
+    public BindManager getBindManager() {
+        return binds;
+    }
+    
+    public TableRef getCurrentTable() {
+        return currentTable;
+    }
+    
+    public void setCurrentTable(TableRef table) {
+        this.currentTable = table;
+    }
+
+    public AggregationManager getAggregationManager() {
+        return aggregates;
+    }
+
+    public ColumnResolver getResolver() {
+        return resolver;
+    }
+
+    public void setResolver(ColumnResolver resolver) {
+        this.resolver = resolver;
+    }
+
+    public ExpressionManager getExpressionManager() {
+        return expressions;
+    }
+
+
+    public ImmutableBytesWritable getTempPtr() {
+        return tempPtr;
+    }
+
+    public ScanRanges getScanRanges() {
+        return this.scanRanges;
+    }
+    
+    public void setScanRanges(ScanRanges scanRanges) {
+        setScanRanges(scanRanges, null);
+    }
+
+    public void setScanRanges(ScanRanges scanRanges, KeyRange minMaxRange) {
+        this.scanRanges = scanRanges;
+        this.scanRanges.setScanStartStopRow(scan);
+        PTable table = this.getCurrentTable().getTable();
+        if (minMaxRange != null) {
+            // Ensure minMaxRange is lower inclusive and upper exclusive, as that's
+            // what we need to intersect against for the HBase scan.
+            byte[] lowerRange = minMaxRange.getLowerRange();
+            if (!minMaxRange.lowerUnbound()) {
+                if (!minMaxRange.isLowerInclusive()) {
+                    lowerRange = ScanUtil.nextKey(lowerRange, table, tempPtr);
+                }
+            }
+            
+            byte[] upperRange = minMaxRange.getUpperRange();
+            if (!minMaxRange.upperUnbound()) {
+                if (minMaxRange.isUpperInclusive()) {
+                    upperRange = ScanUtil.nextKey(upperRange, table, tempPtr);
+                }
+            }
+            if (minMaxRange.getLowerRange() != lowerRange || minMaxRange.getUpperRange() != upperRange) {
+                minMaxRange = KeyRange.getKeyRange(lowerRange, true, upperRange, false);
+            }
+            // If we're not salting, we can intersect this now with the scan range.
+            // Otherwise, we have to wait to do this when we chunk up the scan.
+            if (table.getBucketNum() == null) {
+                minMaxRange = minMaxRange.intersect(KeyRange.getKeyRange(scan.getStartRow(), scan.getStopRow()));
+                scan.setStartRow(minMaxRange.getLowerRange());
+                scan.setStopRow(minMaxRange.getUpperRange());
+            }
+            this.minMaxRange = minMaxRange;
+        }
+    }
+    
+    public PhoenixConnection getConnection() {
+        return statement.getConnection();
+    }
+
+    public PhoenixStatement getStatement() {
+        return statement;
+    }
+
+    public long getCurrentTime() throws SQLException {
+        long ts = this.getCurrentTable().getTimeStamp();
+        if (ts != QueryConstants.UNSET_TIMESTAMP) {
+            return ts;
+        }
+        if (currentTime != QueryConstants.UNSET_TIMESTAMP) {
+            return currentTime;
+        }
+        /*
+         * For an UPSERT VALUES where autocommit off, we won't hit the server until the commit.
+         * However, if the statement has a CURRENT_DATE() call as a value, we need to know the
+         * current time at execution time. In that case, we'll call MetaDataClient.updateCache
+         * purely to bind the current time based on the server time.
+         */
+        PTable table = this.getCurrentTable().getTable();
+        PhoenixConnection connection = getConnection();
+        MetaDataClient client = new MetaDataClient(connection);
+        currentTime = client.getCurrentTime(table.getSchemaName().getString(), table.getTableName().getString());
+        return currentTime;
+    }
+
+    /**
+     * Get the key range derived from row value constructor usage in where clause. These are orthogonal to the ScanRanges
+     * and form a range for which each scan is intersected against.
+     */
+    public KeyRange getMinMaxRange () {
+        return minMaxRange;
+    }
+    
+    public boolean isSingleRowScan() {
+        return this.getScanRanges().isSingleRowScan() && ! (this.getScan().getFilter() instanceof FilterList);
+    }
+    
+    public SequenceManager getSequenceManager(){
+        return sequences;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/compile/StatementNormalizer.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/StatementNormalizer.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/StatementNormalizer.java
new file mode 100644
index 0000000..8eb769e
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/StatementNormalizer.java
@@ -0,0 +1,102 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.compile;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import com.google.common.collect.Lists;
+import org.apache.phoenix.parse.BetweenParseNode;
+import org.apache.phoenix.parse.ColumnParseNode;
+import org.apache.phoenix.parse.ComparisonParseNode;
+import org.apache.phoenix.parse.LessThanOrEqualParseNode;
+import org.apache.phoenix.parse.ParseNode;
+import org.apache.phoenix.parse.ParseNodeRewriter;
+import org.apache.phoenix.parse.SelectStatement;
+import org.apache.phoenix.util.SchemaUtil;
+
+
+/**
+ * 
+ * Class that creates a new select statement ensuring that a literal always occurs
+ * on the RHS (i.e. if literal found on the LHS, then the operator is reversed and
+ * the literal is put on the RHS)
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class StatementNormalizer extends ParseNodeRewriter {
+    private boolean useFullNameForAlias;
+    
+    public StatementNormalizer(ColumnResolver resolver, int expectedAliasCount, boolean useFullNameForAlias) {
+        super(resolver, expectedAliasCount);
+        this.useFullNameForAlias = useFullNameForAlias;
+    }
+
+    public static ParseNode normalize(ParseNode where, ColumnResolver resolver) throws SQLException {
+        return rewrite(where, new StatementNormalizer(resolver, 0, false));
+    }
+    
+    /**
+     * Rewrite the select statement by switching any constants to the right hand side
+     * of the expression.
+     * @param statement the select statement
+     * @param resolver 
+     * @return new select statement
+     * @throws SQLException 
+     */
+    public static SelectStatement normalize(SelectStatement statement, ColumnResolver resolver) throws SQLException {
+        return rewrite(statement, new StatementNormalizer(resolver, statement.getSelect().size(), statement.getFrom().size() > 1));
+    }
+    
+    @Override
+    public ParseNode visitLeave(ComparisonParseNode node, List<ParseNode> nodes) throws SQLException {
+         if (nodes.get(0).isStateless() && !nodes.get(1).isStateless()) {
+             List<ParseNode> normNodes = Lists.newArrayListWithExpectedSize(2);
+             normNodes.add(nodes.get(1));
+             normNodes.add(nodes.get(0));
+             nodes = normNodes;
+             node = NODE_FACTORY.comparison(node.getInvertFilterOp(), nodes.get(0), nodes.get(1));
+         }
+         return super.visitLeave(node, nodes);
+    }
+    
+    @Override
+    public ParseNode visitLeave(final BetweenParseNode node, List<ParseNode> nodes) throws SQLException {
+       
+        LessThanOrEqualParseNode lhsNode =  NODE_FACTORY.lte(node.getChildren().get(1), node.getChildren().get(0));
+        LessThanOrEqualParseNode rhsNode =  NODE_FACTORY.lte(node.getChildren().get(0), node.getChildren().get(2));
+        List<ParseNode> parseNodes = Lists.newArrayListWithExpectedSize(2);
+        parseNodes.add(this.visitLeave(lhsNode, lhsNode.getChildren()));
+        parseNodes.add(this.visitLeave(rhsNode, rhsNode.getChildren()));
+        return super.visitLeave(node, parseNodes);
+    }
+
+    @Override
+    public ParseNode visit(ColumnParseNode node) throws SQLException {
+        if (useFullNameForAlias 
+                && node.getAlias() != null 
+                && node.getTableName() != null
+                && SchemaUtil.normalizeIdentifier(node.getAlias()).equals(node.getName())) {
+            node = NODE_FACTORY.column(NODE_FACTORY.table(node.getSchemaName(), node.getTableName()), node.isCaseSensitive() ? '"' + node.getName() + '"' : node.getName(), node.getFullName());
+        }
+        return super.visit(node);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/compile/StatementPlan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/StatementPlan.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/StatementPlan.java
new file mode 100644
index 0000000..e41e86a
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/StatementPlan.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.compile;
+
+import java.sql.ParameterMetaData;
+import java.sql.SQLException;
+
+import org.apache.phoenix.jdbc.PhoenixParameterMetaData;
+
+
+public interface StatementPlan {
+    public final StatementPlan EMPTY_PLAN = new StatementPlan() {
+        @Override
+        public ParameterMetaData getParameterMetaData() {
+            return new PhoenixParameterMetaData(0);
+        }
+
+        @Override
+        public ExplainPlan getExplainPlan() throws SQLException {
+            return ExplainPlan.EMPTY_PLAN;
+        }
+    };
+    
+    /**
+     * Returns the ParameterMetaData for the statement
+     */
+    ParameterMetaData getParameterMetaData();
+    
+    ExplainPlan getExplainPlan() throws SQLException;
+}


[16/51] [partial] Initial commit of master branch from github

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/SelectStatementRewriter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/SelectStatementRewriter.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/SelectStatementRewriter.java
new file mode 100644
index 0000000..1d18154
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/SelectStatementRewriter.java
@@ -0,0 +1,188 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Set;
+
+
+
+/**
+ * 
+ * Class that creates a new select statement by filtering out nodes.
+ * Currently only supports filtering out boolean nodes (i.e. nodes
+ * that may be ANDed and ORed together.
+ *
+ * TODO: generize this
+ * @author jtaylor
+ * @since 0.1
+ */
+public class SelectStatementRewriter extends ParseNodeRewriter {
+    
+    /**
+     * Rewrite the select statement by filtering out expression nodes from the WHERE clause
+     * @param statement the select statement from which to filter.
+     * @param removeNodes expression nodes to filter out of WHERE clause.
+     * @return new select statement
+     * @throws SQLException 
+     */
+    public static SelectStatement removeFromWhereClause(SelectStatement statement, Set<ParseNode> removeNodes) throws SQLException {
+        if (removeNodes.isEmpty()) {
+            return statement;
+        }
+        ParseNode where = statement.getWhere();
+        SelectStatementRewriter rewriter = new SelectStatementRewriter(removeNodes);
+        where = where.accept(rewriter);
+        // Return new SELECT statement with updated WHERE clause
+        return NODE_FACTORY.select(statement, where, statement.getHaving());
+    }
+    
+    /**
+     * Rewrite the select statement by filtering out expression nodes from the HAVING clause
+     * and anding them with the WHERE clause.
+     * @param statement the select statement from which to move the nodes.
+     * @param moveNodes expression nodes to filter out of HAVING clause and add to WHERE clause.
+     * @return new select statement
+     * @throws SQLException 
+     */
+    public static SelectStatement moveFromHavingToWhereClause(SelectStatement statement, Set<ParseNode> moveNodes) throws SQLException {
+        if (moveNodes.isEmpty()) {
+            return statement;
+        }
+        ParseNode andNode = NODE_FACTORY.and(new ArrayList<ParseNode>(moveNodes));
+        ParseNode having = statement.getHaving();
+        SelectStatementRewriter rewriter = new SelectStatementRewriter(moveNodes);
+        having = having.accept(rewriter);
+        ParseNode where = statement.getWhere();
+        if (where == null) {
+            where = andNode;
+        } else {
+            where = NODE_FACTORY.and(Arrays.asList(where,andNode));
+        }
+        // Return new SELECT statement with updated WHERE and HAVING clauses
+        return NODE_FACTORY.select(statement, where, having);
+    }
+    
+    private static final ParseNodeFactory NODE_FACTORY = new ParseNodeFactory();
+
+    private final Set<ParseNode> removeNodes;
+    
+    private SelectStatementRewriter(Set<ParseNode> removeNodes) {
+        this.removeNodes = removeNodes;
+    }
+    
+    private static interface CompoundNodeFactory {
+        ParseNode createNode(List<ParseNode> children);
+    }
+    
+    private boolean enterCompoundNode(ParseNode node) {
+        if (removeNodes.contains(node)) {
+            return false;
+        }
+        return true;
+    }
+    
+    private ParseNode leaveCompoundNode(CompoundParseNode node, List<ParseNode> children, CompoundNodeFactory factory) {
+        int newSize = children.size();
+        int oldSize = node.getChildren().size();
+        if (newSize == oldSize) {
+            return node;
+        } else if (newSize > 1) {
+            return factory.createNode(children);
+        } else if (newSize == 1) {
+            // TODO: keep or collapse? Maybe be helpful as context of where a problem occurs if a node could not be consumed
+            return(children.get(0));
+        } else {
+            return null;
+        }
+    }
+    
+    @Override
+    public boolean visitEnter(AndParseNode node) throws SQLException {
+        return enterCompoundNode(node);
+    }
+
+    @Override
+    public ParseNode visitLeave(AndParseNode node, List<ParseNode> nodes) throws SQLException {
+        return leaveCompoundNode(node, nodes, new CompoundNodeFactory() {
+            @Override
+            public ParseNode createNode(List<ParseNode> children) {
+                return NODE_FACTORY.and(children);
+            }
+        });
+    }
+
+    @Override
+    public boolean visitEnter(OrParseNode node) throws SQLException {
+        return enterCompoundNode(node);
+    }
+
+    @Override
+    public ParseNode visitLeave(OrParseNode node, List<ParseNode> nodes) throws SQLException {
+        return leaveCompoundNode(node, nodes, new CompoundNodeFactory() {
+            @Override
+            public ParseNode createNode(List<ParseNode> children) {
+                return NODE_FACTORY.or(children);
+            }
+        });
+    }
+    
+    @Override
+    public boolean visitEnter(ComparisonParseNode node) throws SQLException {
+        if (removeNodes.contains(node)) {
+            return false;
+        }
+        return true;
+    }
+
+    @Override
+    public ParseNode visitLeave(ComparisonParseNode node, List<ParseNode> c) throws SQLException {
+        return c.isEmpty() ? null : node;
+    }
+    
+    @Override
+    public boolean visitEnter(LikeParseNode node) throws SQLException {
+        if (removeNodes.contains(node)) {
+            return false;
+        }
+        return true;
+    }
+    
+    @Override
+    public ParseNode visitLeave(LikeParseNode node, List<ParseNode> c) throws SQLException {
+        return c.isEmpty() ? null : node;
+    }
+    
+    @Override
+    public boolean visitEnter(InListParseNode node) throws SQLException {
+        if (removeNodes.contains(node)) {
+            return false;
+        }
+        return true;
+    }
+    
+    @Override
+    public ParseNode visitLeave(InListParseNode node, List<ParseNode> c) throws SQLException {
+        return c.isEmpty() ? null : node;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/SequenceValueParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/SequenceValueParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/SequenceValueParseNode.java
new file mode 100644
index 0000000..f72d60d
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/SequenceValueParseNode.java
@@ -0,0 +1,33 @@
+package org.apache.phoenix.parse;
+
+import java.sql.SQLException;
+
+
+public class SequenceValueParseNode extends TerminalParseNode {
+    public enum Op {NEXT_VALUE, CURRENT_VALUE};
+	private final TableName tableName;
+	private final Op op;
+
+	public SequenceValueParseNode(TableName tableName, Op op) {
+		this.tableName = tableName;
+		this.op = op;
+	}
+
+	@Override
+	public <T> T accept(ParseNodeVisitor<T> visitor) throws SQLException {
+		return visitor.visit(this);
+	}
+
+	public TableName getTableName() {
+		return tableName;
+	}
+
+    @Override
+    public boolean isStateless() {
+        return true;
+    }
+
+    public Op getOp() {
+        return op;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/SingleTableSQLStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/SingleTableSQLStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/SingleTableSQLStatement.java
new file mode 100644
index 0000000..ec5976e
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/SingleTableSQLStatement.java
@@ -0,0 +1,39 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+public abstract class SingleTableSQLStatement implements BindableStatement {
+    private final NamedTableNode table;
+    private final int bindCount;
+
+    public SingleTableSQLStatement(NamedTableNode table, int bindCount) {
+        this.table = table;
+        this.bindCount = bindCount;
+    }
+    
+    public NamedTableNode getTable() {
+        return table;
+    }
+
+    @Override
+    public int getBindCount() {
+        return bindCount;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/StatelessTraverseAllParseNodeVisitor.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/StatelessTraverseAllParseNodeVisitor.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/StatelessTraverseAllParseNodeVisitor.java
new file mode 100644
index 0000000..beaa2df
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/StatelessTraverseAllParseNodeVisitor.java
@@ -0,0 +1,91 @@
+package org.apache.phoenix.parse;
+
+import java.sql.SQLException;
+import java.util.List;
+
+public class StatelessTraverseAllParseNodeVisitor extends TraverseAllParseNodeVisitor<Void> {
+    @Override
+    public Void visitLeave(LikeParseNode node, List<Void> l) throws SQLException {
+        return null;
+    }
+
+    @Override
+    public Void visitLeave(AndParseNode node, List<Void> l) throws SQLException {
+        return null;
+    }
+
+    @Override
+    public Void visitLeave(OrParseNode node, List<Void> l) throws SQLException {
+        return null;
+    }
+
+    @Override
+    public Void visitLeave(FunctionParseNode node, List<Void> l) throws SQLException {
+        return null;
+    }
+
+    @Override
+    public Void visitLeave(ComparisonParseNode node, List<Void> l) throws SQLException {
+        return null;
+    }
+
+    @Override
+    public Void visitLeave(CaseParseNode node, List<Void> l) throws SQLException {
+        return null;
+    }
+
+    @Override
+    public Void visitLeave(AddParseNode node, List<Void> l) throws SQLException {
+        return null;
+    }
+
+    @Override
+    public Void visitLeave(MultiplyParseNode node, List<Void> l) throws SQLException {
+        return null;
+    }
+
+    @Override
+    public Void visitLeave(DivideParseNode node, List<Void> l) throws SQLException {
+        return null;
+    }
+
+    @Override
+    public Void visitLeave(SubtractParseNode node, List<Void> l) throws SQLException {
+        return null;
+    }
+
+    @Override
+    public Void visitLeave(NotParseNode node, List<Void> l) throws SQLException {
+        return null;
+    }
+    
+    @Override
+    public Void visitLeave(CastParseNode node, List<Void> l) throws SQLException {
+        return null;
+    }
+    
+    @Override
+    public Void visitLeave(InListParseNode node, List<Void> l) throws SQLException {
+        return null;
+    }
+
+    @Override
+    public Void visitLeave(StringConcatParseNode node, List<Void> l) throws SQLException {
+        return null;
+    }
+
+    @Override
+    public Void visitLeave(BetweenParseNode node, List<Void> l) throws SQLException {
+        return null;
+    }
+
+    @Override
+    public Void visitLeave(RowValueConstructorParseNode node, List<Void> l) throws SQLException {
+        return null;
+    }
+
+    @Override
+    public Void visitLeave(ArrayConstructorNode node, List<Void> l) throws SQLException {
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/StringConcatParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/StringConcatParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/StringConcatParseNode.java
new file mode 100644
index 0000000..b5134d2
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/StringConcatParseNode.java
@@ -0,0 +1,51 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.List;
+
+
+
+
+/**
+ * 
+ * Node representing || String concatenation in a SQL expression
+ *
+ * @author kmahadik
+ * @since 0.1
+ */
+public class StringConcatParseNode extends CompoundParseNode {
+
+    StringConcatParseNode(List<ParseNode> children) {
+        super(children);
+    }
+
+    @Override
+    public <T> T accept(ParseNodeVisitor<T> visitor) throws SQLException {
+        List<T> l = Collections.emptyList();
+        if (visitor.visitEnter(this)) {
+            l = acceptChildren(visitor);
+        }
+        return visitor.visitLeave(this, l);
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/SubqueryParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/SubqueryParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/SubqueryParseNode.java
new file mode 100644
index 0000000..b2073df
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/SubqueryParseNode.java
@@ -0,0 +1,49 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+
+
+
+/**
+ * 
+ * Node representing a subquery in SQL
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class SubqueryParseNode extends TerminalParseNode {
+    private final SelectStatement select;
+
+    SubqueryParseNode(SelectStatement select) {
+        this.select = select;
+    }
+    
+    public SelectStatement getSelectNode() {
+        return select;
+    }
+
+    @Override
+    public <T> T accept(ParseNodeVisitor<T> visitor) throws SQLException {
+        return visitor.visit(this);
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/SubtractParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/SubtractParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/SubtractParseNode.java
new file mode 100644
index 0000000..c12efef
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/SubtractParseNode.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.List;
+
+
+
+/**
+ * 
+ * Node representing subtraction in a SQL expression
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class SubtractParseNode extends ArithmeticParseNode {
+    SubtractParseNode(List<ParseNode> children) {
+        super(children);
+    }
+
+    @Override
+    public <T> T accept(ParseNodeVisitor<T> visitor) throws SQLException {
+        List<T> l = Collections.emptyList();
+        if (visitor.visitEnter(this)) {
+            l = acceptChildren(visitor);
+        }
+        return visitor.visitLeave(this, l);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/SumAggregateParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/SumAggregateParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/SumAggregateParseNode.java
new file mode 100644
index 0000000..8669b8d
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/SumAggregateParseNode.java
@@ -0,0 +1,41 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.function.FunctionExpression;
+import org.apache.phoenix.expression.function.SumAggregateFunction;
+
+
+public class SumAggregateParseNode extends DelegateConstantToCountParseNode {
+
+    public SumAggregateParseNode(String name, List<ParseNode> children, BuiltInFunctionInfo info) {
+        super(name, children, info);
+    }
+    
+    @Override
+    public FunctionExpression create(List<Expression> children, StatementContext context) throws SQLException {
+        return new SumAggregateFunction(children, getDelegateFunction(children,context));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/TableName.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/TableName.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/TableName.java
new file mode 100644
index 0000000..e552592
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/TableName.java
@@ -0,0 +1,89 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.util.SchemaUtil;
+
+public class TableName {
+    private final String tableName;
+    private final String schemaName;
+    
+    public static TableName createNormalized(String schemaName, String tableName) {
+        schemaName = schemaName == null ? null : SchemaUtil.normalizeIdentifier(schemaName);
+        tableName = SchemaUtil.normalizeIdentifier(tableName);
+        return new TableName(schemaName, tableName);
+    }
+    
+    public static TableName create(String schemaName, String tableName) {
+        return new TableName(schemaName,tableName);
+    }
+    
+    private TableName(String schemaName, String tableName) {
+        this.schemaName = schemaName;
+        this.tableName = tableName;
+    }
+
+    public String getTableName() {
+        return tableName;
+    }
+
+    public String getSchemaName() {
+        return schemaName;
+    }
+    
+    @Override
+    public String toString() {
+        return (schemaName == null ? "" : schemaName + QueryConstants.NAME_SEPARATOR)  + tableName;
+    }
+
+	@Override
+	public int hashCode() {
+		final int prime = 31;
+		int result = 1;
+		result = prime * result
+				+ ((schemaName == null) ? 0 : schemaName.hashCode());
+		result = prime * result
+				+ ((tableName == null) ? 0 : tableName.hashCode());
+		return result;
+	}
+
+	@Override
+	public boolean equals(Object obj) {
+		if (this == obj)
+			return true;
+		if (obj == null)
+			return false;
+		if (getClass() != obj.getClass())
+			return false;
+		TableName other = (TableName) obj;
+		if (schemaName == null) {
+			if (other.schemaName != null)
+				return false;
+		} else if (!schemaName.equals(other.schemaName))
+			return false;
+		if (tableName == null) {
+			if (other.tableName != null)
+				return false;
+		} else if (!tableName.equals(other.tableName))
+			return false;
+		return true;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/TableNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/TableNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/TableNode.java
new file mode 100644
index 0000000..0eaf140
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/TableNode.java
@@ -0,0 +1,45 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+
+
+
+/**
+ * 
+ * Abstract base class for FROM clause data sources
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public abstract class TableNode {
+    private final String alias;
+
+    TableNode(String alias) {
+        this.alias = alias;
+    }
+
+    public String getAlias() {
+        return alias;
+    }
+
+    public abstract void accept(TableNodeVisitor visitor) throws SQLException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/TableNodeVisitor.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/TableNodeVisitor.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/TableNodeVisitor.java
new file mode 100644
index 0000000..6a14da8
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/TableNodeVisitor.java
@@ -0,0 +1,37 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+
+
+/**
+ * 
+ * Visitor for nodes in the FROM clause
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public interface TableNodeVisitor {
+    void visit(BindTableNode boundTableNode) throws SQLException;
+    void visit(JoinTableNode joinNode) throws SQLException;
+    void visit(NamedTableNode namedTableNode) throws SQLException;
+    void visit(DerivedTableNode subselectNode) throws SQLException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/TerminalParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/TerminalParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/TerminalParseNode.java
new file mode 100644
index 0000000..2c31341
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/TerminalParseNode.java
@@ -0,0 +1,37 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * 
+ * Abstract node for expressions that have no children
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public abstract class TerminalParseNode extends ParseNode {
+    @Override
+    public final List<ParseNode> getChildren() {
+        return Collections.emptyList();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/ToCharParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ToCharParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ToCharParseNode.java
new file mode 100644
index 0000000..ef4d390
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ToCharParseNode.java
@@ -0,0 +1,67 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+import java.text.Format;
+import java.util.List;
+
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.LiteralExpression;
+import org.apache.phoenix.expression.function.FunctionArgumentType;
+import org.apache.phoenix.expression.function.FunctionExpression;
+import org.apache.phoenix.expression.function.ToCharFunction;
+import org.apache.phoenix.schema.PDataType;
+
+
+public class ToCharParseNode extends FunctionParseNode {
+
+    public ToCharParseNode(String name, List<ParseNode> children, BuiltInFunctionInfo info) {
+        super(name, children, info);
+    }
+
+    @Override
+    public FunctionExpression create(List<Expression> children, StatementContext context) throws SQLException {
+        PDataType dataType = children.get(0).getDataType();
+        String formatString = (String)((LiteralExpression)children.get(1)).getValue(); // either date or number format string
+        Format formatter;
+        FunctionArgumentType type;
+        if (dataType.isCoercibleTo(PDataType.TIMESTAMP)) {
+            if (formatString == null) {
+                formatString = context.getDateFormat();
+                formatter = context.getDateFormatter();
+            } else {
+                formatter = FunctionArgumentType.TEMPORAL.getFormatter(formatString);
+            }
+            type = FunctionArgumentType.TEMPORAL;
+        }
+        else if (dataType.isCoercibleTo(PDataType.DECIMAL)) {
+            if (formatString == null)
+                formatString = context.getNumberFormat();
+            formatter = FunctionArgumentType.NUMERIC.getFormatter(formatString);
+            type = FunctionArgumentType.NUMERIC;
+        }
+        else {
+            throw new SQLException(dataType + " type is unsupported for TO_CHAR().  Numeric and temporal types are supported.");
+        }
+        return new ToCharFunction(children, type, formatString, formatter);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/ToDateParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ToDateParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ToDateParseNode.java
new file mode 100644
index 0000000..9863902
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ToDateParseNode.java
@@ -0,0 +1,51 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+import java.text.Format;
+import java.util.List;
+
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.LiteralExpression;
+import org.apache.phoenix.expression.function.FunctionExpression;
+import org.apache.phoenix.expression.function.ToDateFunction;
+import org.apache.phoenix.util.DateUtil;
+
+
+public class ToDateParseNode extends FunctionParseNode {
+    public ToDateParseNode(String name, List<ParseNode> children, BuiltInFunctionInfo info) {
+        super(name, children, info);
+    }
+
+    @Override
+    public FunctionExpression create(List<Expression> children, StatementContext context) throws SQLException {
+        Format dateParser;
+        String dateFormat = (String)((LiteralExpression)children.get(1)).getValue();
+        if (dateFormat == null) {
+            dateFormat = context.getDateFormat();
+            dateParser = context.getDateParser();
+        } else {
+            dateParser = DateUtil.getDateParser(dateFormat);
+        }
+        return new ToDateFunction(children, dateFormat, dateParser);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/ToNumberParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ToNumberParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ToNumberParseNode.java
new file mode 100644
index 0000000..cb2d149
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ToNumberParseNode.java
@@ -0,0 +1,49 @@
+package org.apache.phoenix.parse;
+
+import java.sql.SQLException;
+import java.text.Format;
+import java.util.List;
+
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.LiteralExpression;
+import org.apache.phoenix.expression.function.FunctionArgumentType;
+import org.apache.phoenix.expression.function.FunctionExpression;
+import org.apache.phoenix.expression.function.ToNumberFunction;
+import org.apache.phoenix.schema.PDataType;
+
+public class ToNumberParseNode extends FunctionParseNode {
+
+    ToNumberParseNode(String name, List<ParseNode> children,
+            BuiltInFunctionInfo info) {
+        super(name, children, info);
+    }
+
+    @Override
+    public FunctionExpression create(List<Expression> children, StatementContext context) throws SQLException {
+        PDataType dataType = children.get(0).getDataType();
+        String formatString = (String)((LiteralExpression)children.get(1)).getValue(); // either date or number format string
+        Format formatter =  null;
+        FunctionArgumentType type;
+        
+        if (dataType.isCoercibleTo(PDataType.TIMESTAMP)) {
+            if (formatString == null) {
+                formatString = context.getDateFormat();
+                formatter = context.getDateFormatter();
+            } else {
+                formatter = FunctionArgumentType.TEMPORAL.getFormatter(formatString);
+            }
+            type = FunctionArgumentType.TEMPORAL;
+        }
+        else if (dataType.isCoercibleTo(PDataType.CHAR)) {
+            if (formatString != null) {
+                formatter = FunctionArgumentType.CHAR.getFormatter(formatString);
+            }
+            type = FunctionArgumentType.CHAR;
+        }
+        else {
+            throw new SQLException(dataType + " type is unsupported for TO_NUMBER().  Numeric and temporal types are supported.");
+        }
+        return new ToNumberFunction(children, type, formatString, formatter);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/TraverseAllParseNodeVisitor.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/TraverseAllParseNodeVisitor.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/TraverseAllParseNodeVisitor.java
new file mode 100644
index 0000000..1a4a4e6
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/TraverseAllParseNodeVisitor.java
@@ -0,0 +1,158 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+import java.util.List;
+
+
+/**
+ * 
+ * Visitor that traverses into all parse nodes
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public abstract class TraverseAllParseNodeVisitor<T> extends BaseParseNodeVisitor<T> {
+    @Override
+    public boolean visitEnter(AndParseNode node) throws SQLException {
+        return true;
+    }
+
+    @Override
+    public boolean visitEnter(OrParseNode node) throws SQLException {
+        return true;
+    }
+
+    @Override
+    public boolean visitEnter(FunctionParseNode node) throws SQLException {
+        return true;
+    }
+
+    @Override
+    public boolean visitEnter(CaseParseNode node) throws SQLException {
+        return true;
+    }
+
+    @Override
+    public boolean visitEnter(ComparisonParseNode node) throws SQLException {
+        return true;
+    }
+    
+    @Override
+    public boolean visitEnter(LikeParseNode node) throws SQLException {
+        return true;
+    }
+    
+    @Override
+    public boolean visitEnter(NotParseNode node) throws SQLException {
+        return true;
+    }
+    
+    @Override
+    public boolean visitEnter(CastParseNode node) throws SQLException {
+        return true;
+    }
+    
+    @Override
+    public boolean visitEnter(InListParseNode node) throws SQLException {
+        return true;
+    }
+    
+    @Override
+    public boolean visitEnter(IsNullParseNode node) throws SQLException {
+        return true;
+    }
+    
+    @Override
+    public boolean visitEnter(MultiplyParseNode node) throws SQLException {
+        return true;
+    }
+    
+    @Override
+    public boolean visitEnter(SubtractParseNode node) throws SQLException {
+        return true;
+    }
+    
+    @Override
+    public boolean visitEnter(AddParseNode node) throws SQLException {
+        return true;
+    }
+    
+    @Override
+    public boolean visitEnter(DivideParseNode node) throws SQLException {
+        return true;
+    }
+
+    @Override
+    public boolean visitEnter(BetweenParseNode node) throws SQLException {
+        return true;
+    }
+    
+    @Override
+    public T visit(ColumnParseNode node) throws SQLException {
+        return null;
+    }
+
+    @Override
+    public T visitLeave(IsNullParseNode node, List<T> l) throws SQLException {
+        return null;
+    }
+    
+    @Override
+    public T visit(LiteralParseNode node) throws SQLException {
+        return null;
+    }
+    
+    @Override
+    public T visit(BindParseNode node) throws SQLException {
+        return null;
+    }
+
+    @Override
+    public T visit(WildcardParseNode node) throws SQLException {
+        return null;
+    }
+
+    @Override
+    public T visit(FamilyWildcardParseNode node) throws SQLException {
+        return null;
+    }
+    
+    @Override
+    public boolean visitEnter(StringConcatParseNode node) throws SQLException {
+        return true;
+    }
+    
+    @Override
+    public boolean visitEnter(RowValueConstructorParseNode node) throws SQLException {
+        return true;
+    }
+    
+    @Override
+    public T visit(SequenceValueParseNode node) throws SQLException {			
+		return null;
+	}
+    
+    @Override
+    public boolean visitEnter(ArrayConstructorNode node) throws SQLException {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/TraverseNoParseNodeVisitor.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/TraverseNoParseNodeVisitor.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/TraverseNoParseNodeVisitor.java
new file mode 100644
index 0000000..c3d62fd
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/TraverseNoParseNodeVisitor.java
@@ -0,0 +1,242 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+import java.util.List;
+
+
+/**
+ * 
+ * Visitor that traverses into no parse nodes
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public abstract class TraverseNoParseNodeVisitor<T> extends BaseParseNodeVisitor<T> {
+    @Override
+    public boolean visitEnter(AndParseNode node) throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean visitEnter(OrParseNode node) throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean visitEnter(FunctionParseNode node) throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean visitEnter(ComparisonParseNode node) throws SQLException {
+        return false;
+    }
+    
+    @Override
+    public boolean visitEnter(CaseParseNode node) throws SQLException {
+        return false;
+    }
+
+    @Override
+    public boolean visitEnter(LikeParseNode node) throws SQLException {
+        return false;
+    }
+    
+    @Override
+    public boolean visitEnter(BetweenParseNode node) throws SQLException {
+        return false;
+    }
+    
+    @Override
+    public T visitLeave(LikeParseNode node, List<T> l) throws SQLException {
+        return null;
+    }
+    
+    @Override
+    public boolean visitEnter(NotParseNode node) throws SQLException {
+        return false;
+    }
+    
+    @Override
+    public boolean visitEnter(CastParseNode node) throws SQLException {
+        return false;
+    }
+    
+    @Override
+    public T visitLeave(NotParseNode node, List<T> l) throws SQLException {
+        return null;
+    }
+    
+    @Override
+    public T visitLeave(CastParseNode node, List<T> l) throws SQLException {
+        return null;
+    }
+    
+    @Override
+    public boolean visitEnter(InListParseNode node) throws SQLException {
+        return false;
+    }
+    
+    @Override
+    public T visitLeave(InListParseNode node, List<T> l) throws SQLException {
+        return null;
+    }
+    
+    @Override
+    public boolean visitEnter(IsNullParseNode node) throws SQLException {
+        return false;
+    }
+    
+    @Override
+    public T visitLeave(IsNullParseNode node, List<T> l) throws SQLException {
+        return null;
+    }
+    
+    @Override
+    public T visit(ColumnParseNode node) throws SQLException {
+        return null;
+    }
+
+    @Override
+    public T visit(LiteralParseNode node) throws SQLException {
+        return null;
+    }
+
+    @Override
+    public T visit(BindParseNode node) throws SQLException {
+        return null;
+    }
+
+    @Override
+    public T visit(WildcardParseNode node) throws SQLException {
+        return null;
+    }
+    
+    @Override
+    public T visit(FamilyWildcardParseNode node) throws SQLException {
+        return null;
+    }
+
+    @Override
+    public T visitLeave(AndParseNode node, List<T> l) throws SQLException {
+        return null;
+    }
+
+    @Override
+    public T visitLeave(OrParseNode node, List<T> l) throws SQLException {
+        return null;
+    }
+
+    @Override
+    public T visitLeave(FunctionParseNode node, List<T> l) throws SQLException {
+        return null;
+    }
+
+    @Override
+    public T visitLeave(ComparisonParseNode node, List<T> l) throws SQLException {
+        return null;
+    }
+    
+    @Override
+    public T visitLeave(CaseParseNode node, List<T> l) throws SQLException {
+        return null;
+    }
+
+    @Override
+    public boolean visitEnter(MultiplyParseNode node) throws SQLException {
+        return false;
+    }
+
+    @Override
+    public T visitLeave(MultiplyParseNode node, List<T> l) throws SQLException {
+        return null;
+    }
+    
+    @Override
+    public boolean visitEnter(SubtractParseNode node) throws SQLException {
+        return false;
+    }
+
+    @Override
+    public T visitLeave(SubtractParseNode node, List<T> l) throws SQLException {
+        return null;
+    }
+    
+    @Override
+    public boolean visitEnter(AddParseNode node) throws SQLException {
+        return false;
+    }
+
+    @Override
+    public T visitLeave(AddParseNode node, List<T> l) throws SQLException {
+        return null;
+    }
+    
+    @Override
+    public boolean visitEnter(DivideParseNode node) throws SQLException {
+        return false;
+    }
+
+    @Override
+    public T visitLeave(DivideParseNode node, List<T> l) throws SQLException {
+        return null;
+    }
+    @Override
+    public boolean visitEnter(StringConcatParseNode node) throws SQLException {
+        return false;
+    }
+
+    @Override
+    public T visitLeave(StringConcatParseNode node, List<T> l) throws SQLException {
+        return null;
+    }
+    
+    @Override
+    public T visitLeave(BetweenParseNode node, List<T> l) throws SQLException {
+        return null;
+    }
+    
+    @Override
+    public boolean visitEnter(RowValueConstructorParseNode node) throws SQLException {
+        return false;
+    }
+
+    @Override
+    public T visitLeave(RowValueConstructorParseNode node, List<T> l) throws SQLException {
+        return null;
+    }
+    
+    @Override
+    public T visit(SequenceValueParseNode node) throws SQLException {			
+		return null;
+	}
+
+    @Override
+    public boolean visitEnter(ArrayConstructorNode node) throws SQLException {
+        return false;
+    }
+
+    @Override
+    public T visitLeave(ArrayConstructorNode node, List<T> l) throws SQLException {
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/UnaryParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/UnaryParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/UnaryParseNode.java
new file mode 100644
index 0000000..356d047
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/UnaryParseNode.java
@@ -0,0 +1,35 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.util.Collections;
+
+/**
+ * 
+ * Abstract node representing an expression that has a single child in SQL
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public abstract class UnaryParseNode extends CompoundParseNode {
+    UnaryParseNode(ParseNode expr) {
+        super(Collections.singletonList(expr));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/UnsupportedAllParseNodeVisitor.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/UnsupportedAllParseNodeVisitor.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/UnsupportedAllParseNodeVisitor.java
new file mode 100644
index 0000000..24bfcf5
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/UnsupportedAllParseNodeVisitor.java
@@ -0,0 +1,228 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
+import java.util.List;
+
+
+/**
+ * 
+ * Visitor that throws UnsupportedOperationException for every
+ * node.  Meant to be sub-classed for the case of a small subset
+ * of nodes being supported, in which case only those applicable
+ * methods would be overridden.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+abstract public class UnsupportedAllParseNodeVisitor<E> extends BaseParseNodeVisitor<E> {
+
+    @Override
+    public E visit(ColumnParseNode node) throws SQLException {
+        throw new SQLFeatureNotSupportedException(node.toString());
+    }
+
+    @Override
+    public E visit(LiteralParseNode node) throws SQLException {
+        throw new SQLFeatureNotSupportedException(node.toString());
+    }
+
+	@Override
+	public boolean visitEnter(ArrayConstructorNode node) throws SQLException {
+		throw new SQLFeatureNotSupportedException(node.toString());
+	}
+
+	@Override
+	public E visitLeave(ArrayConstructorNode node, List<E> l)
+			throws SQLException {
+		throw new SQLFeatureNotSupportedException(node.toString());
+	}
+
+    @Override
+    public E visit(BindParseNode node) throws SQLException {
+        throw new SQLFeatureNotSupportedException(node.toString());
+    }
+
+    @Override
+    public E visit(WildcardParseNode node) throws SQLException {
+        throw new SQLFeatureNotSupportedException(node.toString());
+    }
+
+    @Override
+    public E visit(FamilyWildcardParseNode node) throws SQLException {
+        throw new SQLFeatureNotSupportedException(node.toString());
+    }
+
+    @Override
+    public boolean visitEnter(AndParseNode node) throws SQLException {
+        throw new SQLFeatureNotSupportedException(node.toString());
+    }
+
+    @Override
+    public boolean visitEnter(OrParseNode node) throws SQLException {
+        throw new SQLFeatureNotSupportedException(node.toString());
+    }
+
+    @Override
+    public boolean visitEnter(FunctionParseNode node) throws SQLException {
+        throw new SQLFeatureNotSupportedException(node.toString());
+    }
+
+    @Override
+    public boolean visitEnter(ComparisonParseNode node) throws SQLException {
+        throw new SQLFeatureNotSupportedException(node.toString());
+    }
+    
+    @Override
+    public boolean visitEnter(BetweenParseNode node) throws SQLException{
+        throw new SQLFeatureNotSupportedException(node.toString());
+    }
+    
+    @Override
+    public E visitLeave(AndParseNode node, List<E> l) throws SQLException {
+        throw new SQLFeatureNotSupportedException(node.toString());
+    }
+    
+    @Override
+    public E visitLeave(OrParseNode node, List<E> l) throws SQLException {
+        throw new SQLFeatureNotSupportedException(node.toString());
+    }
+
+    @Override
+    public E visitLeave(FunctionParseNode node, List<E> l) throws SQLException {
+        throw new SQLFeatureNotSupportedException(node.toString());
+    }
+
+    @Override
+    public E visitLeave(ComparisonParseNode node, List<E> l) throws SQLException {
+        throw new SQLFeatureNotSupportedException(node.toString());
+    }
+
+    @Override
+    public boolean visitEnter(LikeParseNode node) throws SQLException {
+        throw new SQLFeatureNotSupportedException(node.toString());
+    }
+
+    @Override
+    public E visitLeave(LikeParseNode node, List<E> l) throws SQLException {
+        throw new SQLFeatureNotSupportedException(node.toString());
+    }
+
+    @Override
+    public E visitLeave(NotParseNode node, List<E> l) throws SQLException {
+        throw new SQLFeatureNotSupportedException(node.toString());
+    }
+    
+    @Override
+    public boolean visitEnter(NotParseNode node) throws SQLException {
+        throw new SQLFeatureNotSupportedException(node.toString());
+    }
+    
+    @Override
+    public E visitLeave(CastParseNode node, List<E> l) throws SQLException {
+        throw new SQLFeatureNotSupportedException(node.toString());
+    }
+    
+    @Override
+    public boolean visitEnter(CastParseNode node) throws SQLException {
+        throw new SQLFeatureNotSupportedException(node.toString());
+    }
+    
+    @Override
+    public E visitLeave(InListParseNode node, List<E> l) throws SQLException {
+        throw new SQLFeatureNotSupportedException(node.toString());
+    }
+
+    @Override
+    public E visitLeave(BetweenParseNode node, List<E> l) throws SQLException {
+        throw new SQLFeatureNotSupportedException(node.toString());
+    }
+    
+    @Override
+    public boolean visitEnter(InListParseNode node) throws SQLException {
+        throw new SQLFeatureNotSupportedException(node.toString());
+    }
+
+    @Override
+    public boolean visitEnter(IsNullParseNode node) throws SQLException {
+        throw new SQLFeatureNotSupportedException(node.toString());
+    }
+
+    @Override
+    public E visitLeave(IsNullParseNode node, List<E> l) throws SQLException {
+        throw new SQLFeatureNotSupportedException(node.toString());
+    }
+
+    @Override
+    public boolean visitEnter(AddParseNode node) throws SQLException {
+        throw new SQLFeatureNotSupportedException(node.toString());
+    }
+
+    @Override
+    public E visitLeave(AddParseNode node, List<E> l) throws SQLException {
+        throw new SQLFeatureNotSupportedException(node.toString());
+    }
+
+    @Override
+    public boolean visitEnter(SubtractParseNode node) throws SQLException {
+        throw new SQLFeatureNotSupportedException(node.toString());
+    }
+
+    @Override
+    public E visitLeave(SubtractParseNode node, List<E> l) throws SQLException {
+        throw new SQLFeatureNotSupportedException(node.toString());
+    }
+
+    @Override
+    public boolean visitEnter(MultiplyParseNode node) throws SQLException {
+        throw new SQLFeatureNotSupportedException(node.toString());
+    }
+
+    @Override
+    public E visitLeave(MultiplyParseNode node, List<E> l) throws SQLException {
+        throw new SQLFeatureNotSupportedException(node.toString());
+    }
+
+    @Override
+    public boolean visitEnter(DivideParseNode node) throws SQLException {
+        throw new SQLFeatureNotSupportedException(node.toString());
+    }
+
+    @Override
+    public E visitLeave(DivideParseNode node, List<E> l) throws SQLException {
+        throw new SQLFeatureNotSupportedException(node.toString());
+    }
+
+    @Override
+    public List<E> newElementList(int size) {
+        return null;
+    }
+
+    @Override
+    public void addElement(List<E> a, E element) {
+    }
+    
+    @Override
+    public E visit(SequenceValueParseNode node) throws SQLException {			
+		return null;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/UpsertStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/UpsertStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/UpsertStatement.java
new file mode 100644
index 0000000..777a64c
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/UpsertStatement.java
@@ -0,0 +1,54 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.util.Collections;
+import java.util.List;
+
+public class UpsertStatement extends SingleTableSQLStatement { 
+    private final List<ColumnName> columns;
+    private final List<ParseNode> values;
+    private final SelectStatement select;
+    private final HintNode hint;
+
+    public UpsertStatement(NamedTableNode table, HintNode hint, List<ColumnName> columns, List<ParseNode> values, SelectStatement select, int bindCount) {
+        super(table, bindCount);
+        this.columns = columns == null ? Collections.<ColumnName>emptyList() : columns;
+        this.values = values;
+        this.select = select;
+        this.hint = hint == null ? HintNode.EMPTY_HINT_NODE : hint;
+    }
+
+    public List<ColumnName> getColumns() {
+        return columns;
+    }
+
+    public List<ParseNode> getValues() {
+        return values;
+    }
+
+    public SelectStatement getSelect() {
+        return select;
+    }
+
+    public HintNode getHint() {
+        return hint;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/parse/WildcardParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/WildcardParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/WildcardParseNode.java
new file mode 100644
index 0000000..1ae27fc
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/WildcardParseNode.java
@@ -0,0 +1,58 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.parse;
+
+import java.sql.SQLException;
+
+
+
+/**
+ * 
+ * Node representing the selection of all columns (*) in the SELECT clause of SQL
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class WildcardParseNode extends TerminalParseNode {
+    public static final String NAME = "*";
+    public static final WildcardParseNode INSTANCE = new WildcardParseNode(false);
+    public static final WildcardParseNode REWRITE_INSTANCE = new WildcardParseNode(true);
+
+    private final boolean isRewrite;
+
+    private WildcardParseNode(boolean isRewrite) {
+        this.isRewrite = isRewrite;
+    }
+
+    @Override
+    public <T> T accept(ParseNodeVisitor<T> visitor) throws SQLException {
+        return visitor.visit(this);
+    }
+
+    @Override
+    public String toString() {
+        return NAME;
+    }
+
+    public boolean isRewrite() {
+        return isRewrite;
+    }    
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/query/BaseQueryServicesImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/BaseQueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/BaseQueryServicesImpl.java
new file mode 100644
index 0000000..27fa10e
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/BaseQueryServicesImpl.java
@@ -0,0 +1,80 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.util.concurrent.ExecutorService;
+
+import org.apache.phoenix.job.JobManager;
+import org.apache.phoenix.memory.GlobalMemoryManager;
+import org.apache.phoenix.memory.MemoryManager;
+import org.apache.phoenix.optimize.QueryOptimizer;
+import org.apache.phoenix.util.ReadOnlyProps;
+
+
+
+/**
+ * 
+ * Base class for QueryService implementors.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public abstract class BaseQueryServicesImpl implements QueryServices {
+    private final ExecutorService executor;
+    private final MemoryManager memoryManager;
+    private final ReadOnlyProps props;
+    private final QueryOptimizer queryOptimizer;
+    
+    public BaseQueryServicesImpl(QueryServicesOptions options) {
+        this.executor =  JobManager.createThreadPoolExec(
+                options.getKeepAliveMs(), 
+                options.getThreadPoolSize(), 
+                options.getQueueSize());
+        this.memoryManager = new GlobalMemoryManager(
+                Runtime.getRuntime().totalMemory() * options.getMaxMemoryPerc() / 100,
+                options.getMaxMemoryWaitMs());
+        this.props = options.getProps();
+        this.queryOptimizer = new QueryOptimizer(this);
+    }
+    
+    @Override
+    public ExecutorService getExecutor() {
+        return executor;
+    }
+
+    @Override
+    public MemoryManager getMemoryManager() {
+        return memoryManager;
+    }
+
+    @Override
+    public final ReadOnlyProps getProps() {
+        return props;
+    }
+
+    @Override
+    public void close() {
+    }
+
+    @Override
+    public QueryOptimizer getOptimizer() {
+        return queryOptimizer;
+    }   
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/query/ChildQueryServices.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ChildQueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ChildQueryServices.java
new file mode 100644
index 0000000..93613d6
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ChildQueryServices.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 org.apache.phoenix.memory.ChildMemoryManager;
+import org.apache.phoenix.memory.MemoryManager;
+
+/**
+ * 
+ * Child QueryServices that delegates through to global QueryService.
+ * Used to track memory used by each org to allow a max percentage threshold.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class ChildQueryServices extends DelegateConnectionQueryServices {
+    private final MemoryManager memoryManager;
+    private static final int DEFAULT_MAX_ORG_MEMORY_PERC = 30;
+    
+    public ChildQueryServices(ConnectionQueryServices services) {
+        super(services);
+        int maxOrgMemPerc = getProps().getInt(MAX_TENANT_MEMORY_PERC_ATTRIB, DEFAULT_MAX_ORG_MEMORY_PERC);
+        this.memoryManager = new ChildMemoryManager(services.getMemoryManager(), maxOrgMemPerc);
+    }
+
+    @Override
+    public MemoryManager getMemoryManager() {
+        return memoryManager;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/query/ConfigurationFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConfigurationFactory.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConfigurationFactory.java
new file mode 100644
index 0000000..77d737e
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConfigurationFactory.java
@@ -0,0 +1,46 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+
+/**
+ * Creates {@link Configuration} instances that contain HBase/Hadoop settings.
+ *
+ * @author aaraujo
+ * @since 2.0
+ */
+public interface ConfigurationFactory {
+    /**
+     * @return Configuration containing HBase/Hadoop settings
+     */
+    Configuration getConfiguration();
+
+    /**
+     * Default implementation uses {@link org.apache.hadoop.hbase.HBaseConfiguration#create()}.
+     */
+    static class ConfigurationFactoryImpl implements ConfigurationFactory {
+        @Override
+        public Configuration getConfiguration() {
+            return HBaseConfiguration.create();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java
new file mode 100644
index 0000000..47eae18
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java
@@ -0,0 +1,102 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.phoenix.client.KeyValueBuilder;
+import org.apache.phoenix.compile.MutationPlan;
+import org.apache.phoenix.coprocessor.MetaDataProtocol.MetaDataMutationResult;
+import org.apache.phoenix.execute.MutationState;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.schema.SequenceKey;
+
+
+public interface ConnectionQueryServices extends QueryServices, MetaDataMutated {
+    /**
+     * Get (and create if necessary) a child QueryService for a given tenantId.
+     * The QueryService will be cached for the lifetime of the parent QueryService
+     * @param tenantId the organization ID
+     * @return the child QueryService
+     */
+    public ConnectionQueryServices getChildQueryServices(ImmutableBytesWritable tenantId);
+
+    /**
+     * Get an HTableInterface by the given name. It is the callers
+     * responsibility to close the returned HTableInterface.
+     * @param tableName the name of the HTable
+     * @return the HTableInterface
+     * @throws SQLException 
+     */
+    public HTableInterface getTable(byte[] tableName) throws SQLException;
+
+    public HTableDescriptor getTableDescriptor(byte[] tableName) throws SQLException;
+
+    public StatsManager getStatsManager();
+
+    public List<HRegionLocation> getAllTableRegions(byte[] tableName) throws SQLException;
+
+    public PhoenixConnection connect(String url, Properties info) throws SQLException;
+
+    public MetaDataMutationResult getTable(byte[] tenantId, byte[] schemaName, byte[] tableName, long tableTimestamp, long clientTimetamp) throws SQLException;
+    public MetaDataMutationResult createTable(List<Mutation> tableMetaData, byte[] tableName, PTableType tableType, Map<String,Object> tableProps, List<Pair<byte[],Map<String,Object>>> families, byte[][] splits) throws SQLException;
+    public MetaDataMutationResult dropTable(List<Mutation> tableMetadata, PTableType tableType) throws SQLException;
+    public MetaDataMutationResult addColumn(List<Mutation> tableMetaData, PTableType tableType, List<Pair<byte[],Map<String,Object>>> families) throws SQLException;
+    public MetaDataMutationResult dropColumn(List<Mutation> tableMetadata, PTableType tableType) throws SQLException;
+    public MetaDataMutationResult updateIndexState(List<Mutation> tableMetadata, String parentTableName) throws SQLException;
+    public MutationState updateData(MutationPlan plan) throws SQLException;
+
+    public void init(String url, Properties props) throws SQLException;
+
+    public int getLowestClusterHBaseVersion();
+    public HBaseAdmin getAdmin() throws SQLException;
+
+    void clearTableRegionCache(byte[] tableName) throws SQLException;
+
+    boolean hasInvalidIndexConfiguration();
+    
+    long createSequence(String tenantId, String schemaName, String sequenceName, long startWith, long incrementBy, int batchSize, long timestamp) throws SQLException;
+    long dropSequence(String tenantId, String schemaName, String sequenceName, long timestamp) throws SQLException;
+    void reserveSequenceValues(List<SequenceKey> sequenceKeys, long timestamp, long[] values, SQLException[] exceptions) throws SQLException;
+    void incrementSequenceValues(List<SequenceKey> sequenceKeys, long timestamp, long[] values, SQLException[] exceptions) throws SQLException;
+    long getSequenceValue(SequenceKey sequenceKey, long timestamp) throws SQLException;
+    void returnSequenceValues(List<SequenceKey> sequenceKeys, long timestamp, SQLException[] exceptions) throws SQLException;
+    void addConnection(PhoenixConnection connection) throws SQLException;
+    void removeConnection(PhoenixConnection connection) throws SQLException;
+
+    /**
+     * @return the {@link KeyValueBuilder} that is valid for the locally installed version of HBase.
+     */
+    public KeyValueBuilder getKeyValueBuilder();
+    
+    public enum Feature {REVERSE_SCAN};
+    public boolean supportsFeature(Feature feature);
+}
\ No newline at end of file


[09/51] [partial] Initial commit of master branch from github

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/PDateColumn.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PDateColumn.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PDateColumn.java
new file mode 100644
index 0000000..8ed6c4a
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PDateColumn.java
@@ -0,0 +1,27 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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;
+
+abstract public class PDateColumn extends PBaseColumn {
+    @Override
+    public PDataType getDataType() {
+        return PDataType.DATE;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/PDatum.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PDatum.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PDatum.java
new file mode 100644
index 0000000..e1946f5
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PDatum.java
@@ -0,0 +1,54 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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;
+
+
+public interface PDatum {
+    /**
+     * @return is this column nullable?
+     */
+    boolean isNullable();
+
+    /**
+     * @return data type of the column
+     */
+    PDataType getDataType();
+
+    /**
+     * @return maximum byte length of the column
+     */
+    Integer getByteSize();
+
+    /**
+     * @return the actual length of the column. For decimal, it would be its precision. For char or
+     * varchar, it would be the maximum length as specified during schema definition.
+     */
+    Integer getMaxLength();
+
+    /**
+     * @return scale of a decimal number.
+     */
+    Integer getScale();
+    
+    /**
+     * @return The modifier for this column or null if it doesn't have a modifier
+     */
+    ColumnModifier getColumnModifier();
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/PIndexState.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PIndexState.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PIndexState.java
new file mode 100644
index 0000000..330c5c9
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PIndexState.java
@@ -0,0 +1,92 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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;
+
+
+public enum PIndexState {
+    BUILDING("b"),
+    USABLE("e"),
+    UNUSABLE("d"),
+    ACTIVE("a"),
+    INACTIVE("i"),
+    DISABLE("x"),
+    REBUILD("r");
+
+    private final String serializedValue;
+    private final byte[] serializedBytes;
+    private final byte[] nameBytesValue;
+
+    private PIndexState(String value) {
+        this.serializedValue = value;
+        this.serializedBytes = PDataType.VARCHAR.toBytes(value);
+        this.nameBytesValue = PDataType.VARCHAR.toBytes(this.toString());
+    }
+
+    public String getSerializedValue() {
+        return serializedValue;
+    }
+
+    public byte[] getSerializedBytes() {
+        return serializedBytes;
+    }
+
+    public byte[] toBytes() {
+        return nameBytesValue;
+    }
+
+    private static final PIndexState[] FROM_VALUE;
+    private static final int FROM_VALUE_OFFSET;
+    static {
+        int minChar = Integer.MAX_VALUE;
+        int maxChar = Integer.MIN_VALUE;
+        for (PIndexState state: PIndexState.values()) {
+            char c = state.getSerializedValue().charAt(0);
+            if (c < minChar) {
+                minChar = c;
+            }
+            if (c > maxChar) {
+                maxChar = c;
+            }
+        }
+        FROM_VALUE_OFFSET = minChar;
+        FROM_VALUE = new PIndexState[maxChar - minChar + 1];
+        for (PIndexState state: PIndexState.values()) {
+            FROM_VALUE[state.getSerializedValue().charAt(0) - minChar] = state;
+        }
+    }
+
+    public static PIndexState fromSerializedValue(String serializedValue) {
+        if (serializedValue.length() == 1) {
+            int i = serializedValue.charAt(0) - FROM_VALUE_OFFSET;
+            if (i >= 0 && i < FROM_VALUE.length && FROM_VALUE[i] != null) {
+                return FROM_VALUE[i];
+            }
+        }
+        throw new IllegalArgumentException("Unable to PIndexState enum for serialized value of '" + serializedValue + "'");
+    }
+
+    public static PIndexState fromSerializedValue(byte serializedByte) {
+        int i = serializedByte - FROM_VALUE_OFFSET;
+        if (i >= 0 && i < FROM_VALUE.length && FROM_VALUE[i] != null) {
+            return FROM_VALUE[i];
+        }
+        throw new IllegalArgumentException("Unable to PIndexState enum for serialized value of '" + (char)serializedByte + "'");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/PIntegerColumn.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PIntegerColumn.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PIntegerColumn.java
new file mode 100644
index 0000000..5c00117
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PIntegerColumn.java
@@ -0,0 +1,35 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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;
+
+
+/**
+ * 
+ * Base class for PColumn implementors of type Integer.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public abstract class PIntegerColumn extends PBaseColumn {
+    @Override
+    public final PDataType getDataType() {
+        return PDataType.INTEGER;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/PLongColumn.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PLongColumn.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PLongColumn.java
new file mode 100644
index 0000000..69fd7e9
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PLongColumn.java
@@ -0,0 +1,35 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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;
+
+
+/**
+ * 
+ * Base class for PColumn implementors of type Long.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public abstract class PLongColumn extends PBaseColumn {
+    @Override
+    public final PDataType getDataType() {
+        return PDataType.LONG;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaData.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaData.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaData.java
new file mode 100644
index 0000000..1af5c4a
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaData.java
@@ -0,0 +1,30 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.util.Map;
+
+import org.apache.phoenix.query.MetaDataMutated;
+
+
+public interface PMetaData extends MetaDataMutated {
+    public PTable getTable(String name) throws TableNotFoundException;
+    public Map<String, PTable> getTables();
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..cbe5d50
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java
@@ -0,0 +1,200 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.sql.SQLException;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+public class PMetaDataImpl implements PMetaData {
+    public static final PMetaData EMPTY_META_DATA = new PMetaDataImpl(Collections.<String,PTable>emptyMap());
+    private final Map<String,PTable> metaData;
+    
+    public PMetaDataImpl(Map<String,PTable> tables) {
+        this.metaData = ImmutableMap.copyOf(tables);
+    }
+    
+    @Override
+    public PTable getTable(String name) throws TableNotFoundException {
+        PTable table = metaData.get(name);
+        if (table == null) {
+            throw new TableNotFoundException(name);
+        }
+        return table;
+    }
+
+    @Override
+    public Map<String,PTable> getTables() {
+        return metaData;
+    }
+
+
+    @Override
+    public PMetaData addTable(PTable table) throws SQLException {
+        Map<String,PTable> tables = Maps.newHashMap(metaData);
+        PTable oldTable = tables.put(table.getName().getString(), table);
+        if (table.getParentName() != null) { // Upsert new index table into parent data table list
+            String parentName = table.getParentName().getString();
+            PTable parentTable = tables.get(parentName);
+            // If parentTable isn't cached, that's ok we can skip this
+            if (parentTable != null) {
+                List<PTable> oldIndexes = parentTable.getIndexes();
+                List<PTable> newIndexes = Lists.newArrayListWithExpectedSize(oldIndexes.size() + 1);
+                newIndexes.addAll(oldIndexes);
+                if (oldTable != null) {
+                    newIndexes.remove(oldTable);
+                }
+                newIndexes.add(table);
+                tables.put(parentName, PTableImpl.makePTable(parentTable, table.getTimeStamp(), newIndexes));
+            }
+        }
+        for (PTable index : table.getIndexes()) {
+            tables.put(index.getName().getString(), index);
+        }
+        return new PMetaDataImpl(tables);
+    }
+
+    @Override
+    public PMetaData addColumn(String tableName, List<PColumn> columnsToAdd, long tableTimeStamp, long tableSeqNum, boolean isImmutableRows) throws SQLException {
+        PTable table = getTable(tableName);
+        Map<String,PTable> tables = Maps.newHashMap(metaData);
+        List<PColumn> oldColumns = PTableImpl.getColumnsToClone(table);
+        List<PColumn> newColumns;
+        if (columnsToAdd.isEmpty()) {
+            newColumns = oldColumns;
+        } else {
+            newColumns = Lists.newArrayListWithExpectedSize(oldColumns.size() + columnsToAdd.size());
+            newColumns.addAll(oldColumns);
+            newColumns.addAll(columnsToAdd);
+        }
+        PTable newTable = PTableImpl.makePTable(table, tableTimeStamp, tableSeqNum, newColumns, isImmutableRows);
+        tables.put(tableName, newTable);
+        return new PMetaDataImpl(tables);
+    }
+
+    @Override
+    public PMetaData removeTable(String tableName) throws SQLException {
+        PTable table;
+        Map<String,PTable> tables = Maps.newHashMap(metaData);
+        if ((table=tables.remove(tableName)) == null) {
+            throw new TableNotFoundException(tableName);
+        } else {
+            for (PTable index : table.getIndexes()) {
+                if (tables.remove(index.getName().getString()) == null) {
+                    throw new TableNotFoundException(index.getName().getString());
+                }
+            }
+        }
+        return new PMetaDataImpl(tables);
+    }
+    
+    @Override
+    public PMetaData removeColumn(String tableName, String familyName, String columnName, long tableTimeStamp, long tableSeqNum) throws SQLException {
+        PTable table = getTable(tableName);
+        Map<String,PTable> tables = Maps.newHashMap(metaData);
+        PColumn column;
+        if (familyName == null) {
+            column = table.getPKColumn(columnName);
+        } else {
+            column = table.getColumnFamily(familyName).getColumn(columnName);
+        }
+        int positionOffset = 0;
+        int position = column.getPosition();
+        List<PColumn> oldColumns = table.getColumns();
+        if (table.getBucketNum() != null) {
+            position--;
+            positionOffset = 1;
+            oldColumns = oldColumns.subList(positionOffset, oldColumns.size());
+        }
+        List<PColumn> columns = Lists.newArrayListWithExpectedSize(oldColumns.size() - 1);
+        columns.addAll(oldColumns.subList(0, position));
+        // 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.getColumnModifier(), oldColumn.getArraySize());
+            columns.add(newColumn);
+        }
+        
+        PTable newTable = PTableImpl.makePTable(table, tableTimeStamp, tableSeqNum, columns);
+        tables.put(tableName, newTable);
+        return new PMetaDataImpl(tables);
+    }
+
+    public static PMetaData pruneNewerTables(long scn, PMetaData metaData) {
+        if (!hasNewerMetaData(scn, metaData)) {
+            return metaData;
+        }
+        Map<String,PTable> newTables = Maps.newHashMap(metaData.getTables());
+        Iterator<Map.Entry<String, PTable>> tableIterator = newTables.entrySet().iterator();
+        boolean wasModified = false;
+        while (tableIterator.hasNext()) {
+            PTable table = tableIterator.next().getValue();
+            if (table.getTimeStamp() >= scn && table.getType() != PTableType.SYSTEM) {
+                tableIterator.remove();
+                wasModified = true;
+            }
+        }
+    
+        if (wasModified) {
+            return new PMetaDataImpl(newTables);
+        }
+        return metaData;
+    }
+
+    private static boolean hasNewerMetaData(long scn, PMetaData metaData) {
+        for (PTable table : metaData.getTables().values()) {
+            if (table.getTimeStamp() >= scn) {
+                return true;
+            }
+        }
+        return false;
+    }
+    
+    private static boolean hasMultiTenantMetaData(PMetaData metaData) {
+        for (PTable table : metaData.getTables().values()) {
+            if (table.isMultiTenant()) {
+                return true;
+            }
+        }
+        return false;
+    }
+    
+    public static PMetaData pruneMultiTenant(PMetaData metaData) {
+        if (!hasMultiTenantMetaData(metaData)) {
+            return metaData;
+        }
+        Map<String,PTable> newTables = Maps.newHashMap(metaData.getTables());
+        Iterator<Map.Entry<String, PTable>> tableIterator = newTables.entrySet().iterator();
+        while (tableIterator.hasNext()) {
+            PTable table = tableIterator.next().getValue();
+            if (table.isMultiTenant()) {
+                tableIterator.remove();
+            }
+        }
+    
+        return new PMetaDataImpl(newTables);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/PName.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PName.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PName.java
new file mode 100644
index 0000000..97d8989
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PName.java
@@ -0,0 +1,96 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.util.ByteUtil;
+
+
+/**
+ * 
+ * Interface to encapsulate both the client-side name
+ * together with the server-side name for a named object
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public interface PName {
+    public static PName EMPTY_NAME = new PName() {
+        @Override
+        public String getString() {
+            return "";
+        }
+
+        @Override
+        public byte[] getBytes() {
+            return ByteUtil.EMPTY_BYTE_ARRAY;
+        }
+        
+        @Override
+        public String toString() {
+            return getString();
+        }
+
+        @Override
+        public ImmutableBytesPtr getBytesPtr() {
+            return ByteUtil.EMPTY_BYTE_ARRAY_PTR;
+        }
+    };
+    public static PName EMPTY_COLUMN_NAME = new PName() {
+        @Override
+        public String getString() {
+            return QueryConstants.EMPTY_COLUMN_NAME;
+        }
+
+        @Override
+        public byte[] getBytes() {
+            return QueryConstants.EMPTY_COLUMN_BYTES;
+        }
+        
+        @Override
+        public String toString() {
+            return getString();
+        }
+
+        @Override
+        public ImmutableBytesPtr getBytesPtr() {
+            return QueryConstants.EMPTY_COLUMN_BYTES_PTR;
+        }
+    };
+    /**
+     * Get the client-side, normalized name as referenced
+     * in a SQL statement.
+     * @return the normalized string name
+     */
+    String getString();
+    
+    /**
+     * Get the server-side name as referenced in HBase-related
+     * APIs such as Scan, Filter, etc.
+     * @return the name as a byte array
+     */
+    byte[] getBytes();
+
+    /**
+     * @return a pointer to the underlying bytes
+     */
+    ImmutableBytesPtr getBytesPtr();
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/PNameFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PNameFactory.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PNameFactory.java
new file mode 100644
index 0000000..cbea54c
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PNameFactory.java
@@ -0,0 +1,23 @@
+package org.apache.phoenix.schema;
+
+import org.apache.hadoop.hbase.util.Bytes;
+
+import org.apache.phoenix.query.QueryConstants;
+
+public class PNameFactory {
+
+    private PNameFactory() {
+    }
+
+    public static PName newName(String name) {
+        return name == null || name.isEmpty() ? PName.EMPTY_NAME : 
+            name.equals(QueryConstants.EMPTY_COLUMN_NAME ) ?  PName.EMPTY_COLUMN_NAME : 
+                new PNameImpl(name);
+    }
+    
+    public static PName newName(byte[] bytes) {
+        return bytes == null || bytes.length == 0 ? PName.EMPTY_NAME : 
+            Bytes.compareTo(bytes, QueryConstants.EMPTY_COLUMN_BYTES) == 0 ? PName.EMPTY_COLUMN_NAME :
+                new PNameImpl(bytes);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/PNameImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PNameImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PNameImpl.java
new file mode 100644
index 0000000..8feca89
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PNameImpl.java
@@ -0,0 +1,101 @@
+/**
+ * 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.util.Bytes;
+import org.apache.http.annotation.Immutable;
+
+import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+
+@Immutable
+public class PNameImpl implements PName {
+    /**
+     */
+    private static class PNameImplData {
+        /**  */
+        public String stringName;
+        /**  */
+        public byte[] bytesName;
+        /**  */
+        public ImmutableBytesPtr ptr;
+
+        /**
+         *
+         */
+        public PNameImplData() {
+        }
+    }
+    private PNameImplData data = new PNameImplData();
+
+    PNameImpl(String name) {
+        this.data.stringName = name;
+        this.data.bytesName = Bytes.toBytes(name);
+    }
+
+    PNameImpl(byte[] name) {
+        this.data.stringName = Bytes.toString(name);
+        this.data.bytesName = name;
+    }
+
+    @Override
+    public String getString() {
+        return data.stringName;
+    }
+
+    @Override
+    public byte[] getBytes() {
+        return data.bytesName;
+    }
+
+    @Override
+    public ImmutableBytesPtr getBytesPtr() {
+        if (data.ptr == null) {
+            synchronized (data.bytesName) {
+                if (data.ptr == null) {
+                    this.data.ptr = new ImmutableBytesPtr(data.bytesName);
+                }
+            }
+        }
+        return this.data.ptr;
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + data.stringName.hashCode();
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) return true;
+        if (obj == null) return false;
+        if (getClass() != obj.getClass()) return false;
+        PNameImpl other = (PNameImpl) obj;
+        // Compare normalized stringName for equality, since bytesName
+        // may differ since it remains case sensitive.
+        if (!data.stringName.equals(other.data.stringName)) return false;
+        return true;
+    }
+
+    @Override
+    public String toString() {
+        return data.stringName;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/PNormalizedName.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PNormalizedName.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PNormalizedName.java
new file mode 100644
index 0000000..4876de9
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PNormalizedName.java
@@ -0,0 +1,53 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.http.annotation.Immutable;
+
+import org.apache.phoenix.util.SchemaUtil;
+
+
+@Immutable
+public class PNormalizedName extends PNameImpl {
+    
+    public PNormalizedName(String nonNormalizedName) {
+        super(SchemaUtil.normalizeIdentifier(nonNormalizedName));
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + getString().hashCode();
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) return true;
+        if (obj == null) return false;
+        if (getClass() != obj.getClass()) return false;
+        PNormalizedName other = (PNormalizedName)obj;
+        // Compare normalized stringName for equality, since bytesName
+        // may differ since it remains case sensitive.
+        if (!getString().equals(other.getString())) return false;
+        return true;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/PRow.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PRow.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PRow.java
new file mode 100644
index 0000000..22decfc
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PRow.java
@@ -0,0 +1,75 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.client.Mutation;
+
+import com.google.common.collect.ImmutableMap;
+
+/**
+ * 
+ * Provide a client API for updating rows. The updates are processed in
+ * the calling order. Calling setValue after calling delete will cause the
+ * delete to be canceled.  Conversely, calling delete after calling
+ * setValue will cause all prior setValue calls to be canceled.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public interface PRow {
+    Map<PColumn, byte[]> DELETE_MARKER = ImmutableMap.of();
+
+    /**
+     * Get the list of {@link org.apache.hadoop.hbase.client.Mutation} used to
+     * update an HTable after all mutations through calls to
+     * {@link #setValue(PColumn, Object)} or {@link #delete()}.
+     * @return the list of mutations representing all changes made to a row
+     * @throws ConstraintViolationException if row data violates schema
+     * constraint
+     */
+    public List<Mutation> toRowMutations();
+    
+    /**
+     * Set a column value in the row
+     * @param col the column for which the value is being set
+     * @param value the value
+     * @throws ConstraintViolationException if row data violates schema
+     * constraint
+     */
+    public void setValue(PColumn col, Object value);
+    
+    /**
+     * Set a column value in the row
+     * @param col the column for which the value is being set
+     * @param value the value
+     * @throws ConstraintViolationException if row data violates schema
+     * constraint
+     */
+    public void setValue(PColumn col, byte[] value);
+    
+    /**
+     * Delete the row. Note that a delete take precedence over any
+     * values that may have been set before or after the delete call.
+     */
+    public void delete();
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/PStringColumn.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PStringColumn.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PStringColumn.java
new file mode 100644
index 0000000..606b541
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PStringColumn.java
@@ -0,0 +1,36 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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;
+
+/**
+ * 
+ * Abstract class for columns of type {@link org.apache.phoenix.schema.PDataType#VARCHAR}
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public abstract class PStringColumn extends PBaseColumn {
+
+    @Override
+    public PDataType getDataType() {
+        return PDataType.VARCHAR;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..10a568e
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java
@@ -0,0 +1,280 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.Writable;
+import org.apache.phoenix.client.KeyValueBuilder;
+import org.apache.phoenix.index.IndexMaintainer;
+import org.apache.phoenix.schema.stat.PTableStats;
+
+
+/**
+ * Definition of a Phoenix table
+ *
+ * @author wmacklem,jtaylor
+ * @since 0.1
+ */
+public interface PTable extends Writable {
+    public static final long INITIAL_SEQ_NUM = 0;
+    public static final String IS_IMMUTABLE_ROWS_PROP_NAME = "IMMUTABLE_ROWS";
+    public static final boolean DEFAULT_DISABLE_WAL = false;
+    
+    public enum ViewType { 
+        MAPPED((byte)1),
+        READ_ONLY((byte)2),
+        UPDATABLE((byte)3);
+
+        private final byte[] byteValue;
+        private final byte serializedValue;
+        
+        ViewType(byte serializedValue) {
+            this.serializedValue = serializedValue;
+            this.byteValue = Bytes.toBytes(this.name());
+        }
+        
+        public byte[] getBytes() {
+            return byteValue;
+        }
+        
+        public boolean isReadOnly() {
+            return this != UPDATABLE;
+        }
+        
+        public byte getSerializedValue() {
+            return this.serializedValue;
+        }
+        
+        public static ViewType fromSerializedValue(byte serializedValue) {
+            if (serializedValue < 1 || serializedValue > ViewType.values().length) {
+                throw new IllegalArgumentException("Invalid ViewType " + serializedValue);
+            }
+            return ViewType.values()[serializedValue-1];
+        }
+        
+        public ViewType combine(ViewType otherType) {
+            if (otherType == null) {
+                return this;
+            }
+            if (this == UPDATABLE && otherType == UPDATABLE) {
+                return UPDATABLE;
+            }
+            return READ_ONLY;
+        }
+    }
+
+    public enum LinkType {
+        /**
+         * Link from a table to its index table
+         */
+        INDEX_TABLE((byte)1),
+        /**
+         * Link from a view to its physical table
+         */
+        PHYSICAL_TABLE((byte)2);
+
+        private final byte[] byteValue;
+        private final byte serializedValue;
+        
+        LinkType(byte serializedValue) {
+            this.serializedValue = serializedValue;
+            this.byteValue = Bytes.toBytes(this.name());
+        }
+        
+        public byte[] getBytes() {
+            return byteValue;
+        }
+        
+        public byte getSerializedValue() {
+            return this.serializedValue;
+        }
+        
+        public static LinkType fromSerializedValue(byte serializedValue) {
+            if (serializedValue < 1 || serializedValue > LinkType.values().length) {
+                return null;
+            }
+            return LinkType.values()[serializedValue-1];
+        }
+    }
+
+    long getTimeStamp();
+    long getSequenceNumber();
+    /**
+     * @return table name
+     */
+    PName getName();
+    PName getSchemaName(); 
+    PName getTableName(); 
+
+    /**
+     * @return the table type
+     */
+    PTableType getType();
+
+    PName getPKName();
+
+    /**
+     * Get the PK columns ordered by position.
+     * @return a list of the PK columns
+     */
+    List<PColumn> getPKColumns();
+
+    /**
+     * Get all columns ordered by position.
+     * @return a list of all columns
+     */
+    List<PColumn> getColumns();
+
+    /**
+     * @return A list of the column families of this table
+     *  ordered by position.
+     */
+    List<PColumnFamily> getColumnFamilies();
+
+    /**
+     * Get the column family with the given name
+     * @param family the column family name
+     * @return the PColumnFamily with the given name
+     * @throws ColumnFamilyNotFoundException if the column family cannot be found
+     */
+    PColumnFamily getColumnFamily(byte[] family) throws ColumnFamilyNotFoundException;
+
+    PColumnFamily getColumnFamily(String family) throws ColumnFamilyNotFoundException;
+
+    /**
+     * Get the column with the given string name.
+     * @param name the column name
+     * @return the PColumn with the given name
+     * @throws ColumnNotFoundException if no column with the given name
+     * can be found
+     * @throws AmbiguousColumnException if multiple columns are found with the given name
+     */
+    PColumn getColumn(String name) throws ColumnNotFoundException, AmbiguousColumnException;
+
+    /**
+     * Get the PK column with the given name.
+     * @param name the column name
+     * @return the PColumn with the given name
+     * @throws ColumnNotFoundException if no PK column with the given name
+     * can be found
+     * @throws ColumnNotFoundException 
+     */
+    PColumn getPKColumn(String name) throws ColumnNotFoundException;
+
+    /**
+     * Creates a new row at the specified timestamp using the key
+     * for the PK values (from {@link #newKey(ImmutableBytesWritable, byte[][])}
+     * and the optional key values specified using values.
+     * @param ts the timestamp that the key value will have when committed
+     * @param key the row key of the key value
+     * @param values the optional key values
+     * @return the new row. Use {@link org.apache.phoenix.schema.PRow#toRowMutations()} to
+     * generate the Row to send to the HBase server.
+     * @throws ConstraintViolationException if row data violates schema
+     * constraint
+     */
+    PRow newRow(KeyValueBuilder builder, long ts, ImmutableBytesWritable key, byte[]... values);
+
+    /**
+     * Creates a new row for the PK values (from {@link #newKey(ImmutableBytesWritable, byte[][])}
+     * and the optional key values specified using values. The timestamp of the key value
+     * will be set by the HBase server.
+     * @param key the row key of the key value
+     * @param values the optional key values
+     * @return the new row. Use {@link org.apache.phoenix.schema.PRow#toRowMutations()} to
+     * generate the row to send to the HBase server.
+     * @throws ConstraintViolationException if row data violates schema
+     * constraint
+     */
+    PRow newRow(KeyValueBuilder builder, ImmutableBytesWritable key, byte[]... values);
+
+    /**
+     * Formulates a row key using the values provided. The values must be in
+     * the same order as {@link #getPKColumns()}.
+     * @param key bytes pointer that will be filled in with the row key
+     * @param values the PK column values
+     * @return the number of values that were used from values to set
+     * the row key
+     */
+    int newKey(ImmutableBytesWritable key, byte[][] values);
+
+    /**
+     * Return the statistics table associated with this PTable.
+     * @return the statistics table.
+     */
+    PTableStats getTableStats();
+
+    RowKeySchema getRowKeySchema();
+
+    /**
+     * Return the number of buckets used by this table for salting. If the table does
+     * not use salting, returns null.
+     * @return number of buckets used by this table for salting, or null if salting is not used.
+     */
+    Integer getBucketNum();
+
+    /**
+     * Return the list of indexes defined on this table.
+     * @return the list of indexes.
+     */
+    List<PTable> getIndexes();
+
+    /**
+     * For a table of index type, return the state of the table.
+     * @return the state of the index.
+     */
+    PIndexState getIndexState();
+
+    /**
+     * Gets the full name of the data table for an index table.
+     * @return the name of the data table that this index is on
+     *  or null if not an index.
+     */
+    PName getParentName();
+    /**
+     * Gets the table name of the data table for an index table.
+     * @return the table name of the data table that this index is
+     * on or null if not an index.
+     */
+    PName getParentTableName();
+    
+    /**
+     * For a view, return the name of table in Phoenix that physically stores data.
+     * Currently a single name, but when views are allowed over multiple tables, will become multi-valued.
+     * @return the name of the physical table storing the data.
+     */
+    public List<PName> getPhysicalNames();
+
+    PName getPhysicalName();
+    boolean isImmutableRows();
+
+    void getIndexMaintainers(ImmutableBytesWritable ptr);
+    IndexMaintainer getIndexMaintainer(PTable dataTable);
+    PName getDefaultFamilyName();
+    
+    boolean isWALDisabled();
+    boolean isMultiTenant();
+
+    ViewType getViewType();
+    String getViewStatement();
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..11d6c94
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
@@ -0,0 +1,863 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.apache.phoenix.client.KeyValueBuilder.addQuietly;
+import static org.apache.phoenix.client.KeyValueBuilder.deleteQuietly;
+import static org.apache.phoenix.query.QueryConstants.SEPARATOR_BYTE;
+import static org.apache.phoenix.schema.SaltingUtil.SALTING_COLUMN;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+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 org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.phoenix.client.KeyValueBuilder;
+import org.apache.phoenix.index.IndexMaintainer;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.RowKeySchema.RowKeySchemaBuilder;
+import org.apache.phoenix.schema.stat.PTableStats;
+import org.apache.phoenix.schema.stat.PTableStatsImpl;
+import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.StringUtil;
+import org.apache.phoenix.util.TrustedByteArrayOutputStream;
+
+import com.google.common.base.Objects;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSortedMap;
+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
+ * storing data in a single column (ColumnLayout.SINGLE) or in
+ * multiple columns (ColumnLayout.MULTI).
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class PTableImpl implements PTable {
+    private static final Integer NO_SALTING = -1;
+    
+    private PName name;
+    private PName schemaName;
+    private PName tableName;
+    private PTableType type;
+    private PIndexState state;
+    private long sequenceNumber;
+    private long timeStamp;
+    // Have MultiMap for String->PColumn (may need family qualifier)
+    private List<PColumn> pkColumns;
+    private List<PColumn> allColumns;
+    private List<PColumnFamily> families;
+    private Map<byte[], PColumnFamily> familyByBytes;
+    private Map<String, PColumnFamily> familyByString;
+    private ListMultimap<String,PColumn> columnsByName;
+    private PName pkName;
+    private Integer bucketNum;
+    // Statistics associated with this table.
+    private PTableStats stats;
+    private RowKeySchema rowKeySchema;
+    // Indexes associated with this table.
+    private List<PTable> indexes;
+    // Data table name that the index is created on.
+    private PName parentName;
+    private PName parentTableName;
+    private List<PName> physicalNames;
+    private boolean isImmutableRows;
+    private IndexMaintainer indexMaintainer;
+    private ImmutableBytesWritable indexMaintainersPtr;
+    private PName defaultFamilyName;
+    private String viewStatement;
+    private boolean disableWAL;
+    private boolean multiTenant;
+    private ViewType viewType;
+    
+    public PTableImpl() {
+    }
+
+    public PTableImpl(PName name) { // For finding table ref
+        this.name = name;
+    }
+
+    public PTableImpl(String schemaName, String tableName, long timestamp, List<PColumnFamily> families) { // For base table of mapped VIEW
+        this.name = PNameFactory.newName(SchemaUtil.getTableName(schemaName, tableName));
+        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;
+    }
+
+    public PTableImpl(long timeStamp) { // For delete marker
+        this(timeStamp, false);
+    }
+
+    public PTableImpl(long timeStamp, boolean isIndex) { // For index delete marker
+        if (isIndex) {
+            this.type = PTableType.INDEX;
+            this.state = PIndexState.INACTIVE;
+        } else {
+            this.type = PTableType.TABLE;
+        }
+        this.timeStamp = timeStamp;
+        this.pkColumns = this.allColumns = Collections.emptyList();
+        this.families = Collections.emptyList();
+        this.familyByBytes = Collections.emptyMap();
+        this.familyByString = Collections.emptyMap();
+        this.rowKeySchema = RowKeySchema.EMPTY_SCHEMA;
+        this.indexes = Collections.emptyList();
+    }
+
+    // When cloning table, ignore the salt column as it will be added back in the constructor
+    public static List<PColumn> getColumnsToClone(PTable table) {
+        return table.getBucketNum() == null ? table.getColumns() : table.getColumns().subList(1, table.getColumns().size());
+    }
+    
+    public static PTableImpl makePTable(PTable table, long timeStamp, List<PTable> indexes) throws SQLException {
+        return new PTableImpl(
+                table.getSchemaName(), table.getTableName(), table.getType(), table.getIndexState(), timeStamp, table.getSequenceNumber() + 1, 
+                table.getPKName(), table.getBucketNum(), getColumnsToClone(table), table.getParentTableName(), indexes, table.isImmutableRows(),
+                table.getPhysicalNames(), table.getDefaultFamilyName(), table.getViewStatement(), table.isWALDisabled(), table.isMultiTenant(), table.getViewType());
+    }
+
+    public static PTableImpl makePTable(PTable table, List<PColumn> columns) throws SQLException {
+        return new PTableImpl(
+                table.getSchemaName(), table.getTableName(), table.getType(), table.getIndexState(), table.getTimeStamp(), table.getSequenceNumber(), 
+                table.getPKName(), table.getBucketNum(), columns, table.getParentTableName(), table.getIndexes(), table.isImmutableRows(), table.getPhysicalNames(),
+                table.getDefaultFamilyName(), table.getViewStatement(), table.isWALDisabled(), table.isMultiTenant(), table.getViewType());
+    }
+
+    public static PTableImpl makePTable(PTable table, long timeStamp, long sequenceNumber, List<PColumn> columns) throws SQLException {
+        return new PTableImpl(
+                table.getSchemaName(), table.getTableName(), table.getType(), table.getIndexState(), timeStamp, sequenceNumber, 
+                table.getPKName(), table.getBucketNum(), columns, table.getParentTableName(), table.getIndexes(), table.isImmutableRows(), table.getPhysicalNames(),
+                table.getDefaultFamilyName(), table.getViewStatement(), table.isWALDisabled(), table.isMultiTenant(), table.getViewType());
+    }
+
+    public static PTableImpl makePTable(PTable table, long timeStamp, long sequenceNumber, List<PColumn> columns, boolean isImmutableRows) throws SQLException {
+        return new PTableImpl(
+                table.getSchemaName(), table.getTableName(), table.getType(), table.getIndexState(), timeStamp, sequenceNumber, 
+                table.getPKName(), table.getBucketNum(), columns, table.getParentTableName(), table.getIndexes(), isImmutableRows,
+                table.getPhysicalNames(), table.getDefaultFamilyName(), table.getViewStatement(), table.isWALDisabled(), table.isMultiTenant(), table.getViewType());
+    }
+
+    public static PTableImpl makePTable(PTable table, PIndexState state) throws SQLException {
+        return new PTableImpl(
+                table.getSchemaName(), table.getTableName(), table.getType(), state, table.getTimeStamp(), table.getSequenceNumber(), 
+                table.getPKName(), table.getBucketNum(), getColumnsToClone(table), table.getParentTableName(), 
+                table.getIndexes(), table.isImmutableRows(), table.getPhysicalNames(),
+                table.getDefaultFamilyName(), table.getViewStatement(), table.isWALDisabled(), table.isMultiTenant(), table.getViewType());
+    }
+
+    public static PTableImpl makePTable(PName schemaName, PName tableName, PTableType type, PIndexState state, long timeStamp, long sequenceNumber, PName pkName,
+            Integer bucketNum, List<PColumn> columns, PName dataTableName, List<PTable> indexes, boolean isImmutableRows, List<PName> physicalNames,
+            PName defaultFamilyName, String viewExpression, boolean disableWAL, boolean multiTenant, ViewType viewType) throws SQLException {
+        return new PTableImpl(schemaName, tableName, type, state, timeStamp, sequenceNumber, pkName, bucketNum, columns, dataTableName, indexes,
+                isImmutableRows, physicalNames, defaultFamilyName, viewExpression, disableWAL, multiTenant, viewType);
+    }
+
+    private PTableImpl(PName schemaName, PName tableName, PTableType type, PIndexState state, long timeStamp, long sequenceNumber, PName pkName,
+            Integer bucketNum, List<PColumn> columns, PName dataTableName, List<PTable> indexes, boolean isImmutableRows, List<PName> physicalNames,
+            PName defaultFamilyName, String viewExpression, boolean disableWAL, boolean multiTenant, ViewType viewType) throws SQLException {
+        init(schemaName, tableName, type, state, timeStamp, sequenceNumber, pkName, bucketNum, columns, new PTableStatsImpl(),
+                dataTableName, indexes, isImmutableRows, physicalNames, defaultFamilyName, viewExpression, disableWAL, multiTenant, viewType);
+    }
+
+    @Override
+    public boolean isMultiTenant() {
+        return multiTenant;
+    }
+    
+    @Override
+    public ViewType getViewType() {
+        return viewType;
+    }
+    
+    private void init(PName schemaName, PName tableName, PTableType type, PIndexState state, long timeStamp, long sequenceNumber, PName pkName,
+            Integer bucketNum, List<PColumn> columns, PTableStats stats, PName parentTableName, List<PTable> indexes, boolean isImmutableRows,
+            List<PName> physicalNames, PName defaultFamilyName, String viewExpression, boolean disableWAL, boolean multiTenant, ViewType viewType) throws SQLException {
+        if (schemaName == null) {
+            throw new NullPointerException();
+        }
+        this.schemaName = schemaName;
+        this.tableName = tableName;
+        this.name = PNameFactory.newName(SchemaUtil.getTableName(schemaName.getString(), tableName.getString()));
+        this.type = type;
+        this.state = state;
+        this.timeStamp = timeStamp;
+        this.sequenceNumber = sequenceNumber;
+        this.pkName = pkName;
+        this.isImmutableRows = isImmutableRows;
+        this.defaultFamilyName = defaultFamilyName;
+        this.viewStatement = viewExpression;
+        this.disableWAL = disableWAL;
+        this.multiTenant = multiTenant;
+        this.viewType = viewType;
+        List<PColumn> pkColumns;
+        PColumn[] allColumns;
+
+        this.columnsByName = ArrayListMultimap.create(columns.size(), 1);
+        if (bucketNum != null) {
+            // Add salt column to allColumns and pkColumns, but don't add to
+            // columnsByName, since it should not be addressable via name.
+            allColumns = new PColumn[columns.size()+1];
+            allColumns[SALTING_COLUMN.getPosition()] = SALTING_COLUMN;
+            pkColumns = Lists.newArrayListWithExpectedSize(columns.size()+1);
+            pkColumns.add(SALTING_COLUMN);
+        } else {
+            allColumns = new PColumn[columns.size()];
+            pkColumns = Lists.newArrayListWithExpectedSize(columns.size());
+        }
+        for (int i = 0; i < columns.size(); i++) {
+            PColumn column = columns.get(i);
+            allColumns[column.getPosition()] = column;
+            PName familyName = column.getFamilyName();
+            if (familyName == null) {
+                pkColumns.add(column);
+            }
+            String columnName = column.getName().getString();
+            if (columnsByName.put(columnName, column)) {
+                int count = 0;
+                for (PColumn dupColumn : columnsByName.get(columnName)) {
+                    if (Objects.equal(familyName, dupColumn.getFamilyName())) {
+                        count++;
+                        if (count > 1) {
+                            throw new ColumnAlreadyExistsException(null, name.getString(), columnName);
+                        }
+                    }
+                }
+            }
+        }
+        this.bucketNum = bucketNum;
+        this.pkColumns = ImmutableList.copyOf(pkColumns);
+        this.allColumns = ImmutableList.copyOf(allColumns);
+        
+        RowKeySchemaBuilder builder = new RowKeySchemaBuilder(pkColumns.size());
+        // Two pass so that column order in column families matches overall column order
+        // and to ensure that column family order is constant
+        int maxExpectedSize = allColumns.length - pkColumns.size();
+        // Maintain iteration order so that column families are ordered as they are listed
+        Map<PName, List<PColumn>> familyMap = Maps.newLinkedHashMap();
+        for (PColumn column : allColumns) {
+            PName familyName = column.getFamilyName();
+            if (familyName == null) {
+                builder.addField(column, column.isNullable(), column.getColumnModifier());
+            } else {
+                List<PColumn> columnsInFamily = familyMap.get(familyName);
+                if (columnsInFamily == null) {
+                    columnsInFamily = Lists.newArrayListWithExpectedSize(maxExpectedSize);
+                    familyMap.put(familyName, columnsInFamily);
+                }
+                columnsInFamily.add(column);
+            }
+        }
+        
+        this.rowKeySchema = builder.build();
+        Iterator<Map.Entry<PName,List<PColumn>>> iterator = familyMap.entrySet().iterator();
+        PColumnFamily[] families = new PColumnFamily[familyMap.size()];
+        ImmutableMap.Builder<String, PColumnFamily> familyByString = ImmutableMap.builder();
+        ImmutableSortedMap.Builder<byte[], PColumnFamily> familyByBytes = ImmutableSortedMap.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());
+            families[i] = family;
+            familyByString.put(family.getName().getString(), family);
+            familyByBytes.put(family.getName().getBytes(), family);
+        }
+        this.families = ImmutableList.copyOf(families);
+        this.familyByBytes = familyByBytes.build();
+        this.familyByString = familyByString.build();
+        this.stats = stats;
+        this.indexes = indexes;
+        this.parentTableName = parentTableName;
+        this.parentName = parentTableName == null ? null : PNameFactory.newName(SchemaUtil.getTableName(schemaName.getString(), parentTableName.getString()));
+        this.physicalNames = physicalNames == null ? ImmutableList.<PName>of() : ImmutableList.copyOf(physicalNames);
+    }
+
+    @Override
+    public boolean isImmutableRows() {
+        return isImmutableRows;
+    }
+    
+    @Override
+    public String toString() {
+        return name.getString();
+    }
+
+    @Override
+    public List<PColumn> getPKColumns() {
+        return pkColumns;
+    }
+
+    @Override
+    public final PName getName() {
+        return name;
+    }
+
+    @Override
+    public final PName getSchemaName() {
+        return schemaName;
+    }
+
+    @Override
+    public final PName getTableName() {
+        return tableName;
+    }
+
+    @Override
+    public final PTableType getType() {
+        return type;
+    }
+
+    @Override
+    public final List<PColumnFamily> getColumnFamilies() {
+        return families;
+    }
+
+    @Override
+    public int newKey(ImmutableBytesWritable key, byte[][] values) {
+        int nValues = values.length;
+        while (nValues > 0 && (values[nValues-1] == null || values[nValues-1].length == 0)) {
+            nValues--;
+        }
+        int i = 0;
+        TrustedByteArrayOutputStream os = new TrustedByteArrayOutputStream(SchemaUtil.estimateKeyLength(this));
+        try {
+            Integer bucketNum = this.getBucketNum();
+            if (bucketNum != null) {
+                // Write place holder for salt byte
+                i++;
+                os.write(QueryConstants.SEPARATOR_BYTE_ARRAY);
+            }
+            List<PColumn> columns = getPKColumns();
+            int nColumns = columns.size();
+            PDataType type = null;
+            while (i < nValues && i < nColumns) {
+                // Separate variable length column values in key with zero byte
+                if (type != null && !type.isFixedWidth()) {
+                    os.write(SEPARATOR_BYTE);
+                }
+                PColumn column = columns.get(i);
+                type = column.getDataType();
+                // This will throw if the value is null and the type doesn't allow null
+                byte[] byteValue = values[i++];
+                if (byteValue == null) {
+                    byteValue = ByteUtil.EMPTY_BYTE_ARRAY;
+                }
+                // An empty byte array return value means null. Do this,
+                // since a type may have muliple representations of null.
+                // For example, VARCHAR treats both null and an empty string
+                // as null. This way we don't need to leak that part of the
+                // implementation outside of PDataType by checking the value
+                // here.
+                if (byteValue.length == 0 && !column.isNullable()) { 
+                    throw new ConstraintViolationException(name.getString() + "." + column.getName().getString() + " may not be null");
+                }
+                Integer	byteSize = column.getByteSize();
+                if (byteSize != null && type.isFixedWidth() && byteValue.length <= byteSize) {
+                    byteValue = StringUtil.padChar(byteValue, byteSize);
+                } else if (byteSize != null && byteValue.length > byteSize) {
+                    throw new ConstraintViolationException(name.getString() + "." + column.getName().getString() + " may not exceed " + byteSize + " bytes (" + SchemaUtil.toString(type, byteValue) + ")");
+                }
+                os.write(byteValue, 0, byteValue.length);
+            }
+            // If some non null pk values aren't set, then throw
+            if (i < nColumns) {
+                PColumn column = columns.get(i);
+                type = column.getDataType();
+                if (type.isFixedWidth() || !column.isNullable()) {
+                    throw new ConstraintViolationException(name.getString() + "." + column.getName().getString() + " may not be null");
+                }
+            }
+            byte[] buf = os.getBuffer();
+            int size = os.size();
+            if (bucketNum != null) {
+                buf[0] = SaltingUtil.getSaltingByte(buf, 1, size-1, bucketNum);
+            }
+            key.set(buf,0,size);
+            return i;
+        } finally {
+            try {
+                os.close();
+            } catch (IOException e) {
+                throw new RuntimeException(e); // Impossible
+            }
+        }
+    }
+
+    private PRow newRow(KeyValueBuilder builder, long ts, ImmutableBytesWritable key, int i, byte[]... values) {
+        PRow row = new PRowImpl(builder, key, ts, getBucketNum());
+        if (i < values.length) {
+            for (PColumnFamily family : getColumnFamilies()) {
+                for (PColumn column : family.getColumns()) {
+                    row.setValue(column, values[i++]);
+                    if (i == values.length)
+                        return row;
+                }
+            }
+        }
+        return row;
+    }
+
+    @Override
+    public PRow newRow(KeyValueBuilder builder, long ts, ImmutableBytesWritable key,
+            byte[]... values) {
+        return newRow(builder, ts, key, 0, values);
+    }
+
+    @Override
+    public PRow newRow(KeyValueBuilder builder, ImmutableBytesWritable key, byte[]... values) {
+        return newRow(builder, HConstants.LATEST_TIMESTAMP, key, values);
+    }
+
+    @Override
+    public PColumn getColumn(String name) throws ColumnNotFoundException, AmbiguousColumnException {
+        List<PColumn> columns = columnsByName.get(name);
+        int size = columns.size();
+        if (size == 0) {
+            throw new ColumnNotFoundException(name);
+        }
+        if (size > 1) {
+            for (PColumn column : columns) {
+                if (QueryConstants.DEFAULT_COLUMN_FAMILY.equals(column.getFamilyName().getString())) {
+                    // Allow ambiguity with default column, since a user would not know how to prefix it.
+                    return column;
+                }
+            }
+            throw new AmbiguousColumnException(name);
+        }
+        return columns.get(0);
+    }
+
+    /**
+     * 
+     * PRow implementation for ColumnLayout.MULTI mode which stores column
+     * values across multiple hbase columns.
+     *
+     * @author jtaylor
+     * @since 0.1
+     */
+    private class PRowImpl implements PRow {
+        private final byte[] key;
+        private final ImmutableBytesWritable keyPtr;
+        // default to the generic builder, and only override when we know on the client
+        private final KeyValueBuilder kvBuilder;
+
+        private Put setValues;
+        private Delete unsetValues;
+        private Delete deleteRow;
+        private final long ts;
+
+        public PRowImpl(KeyValueBuilder kvBuilder, ImmutableBytesWritable key, long ts, Integer bucketNum) {
+            this.kvBuilder = kvBuilder;
+            this.ts = ts;
+            if (bucketNum != null) {
+                this.key = SaltingUtil.getSaltedKey(key, bucketNum);
+                this.keyPtr = new ImmutableBytesPtr(this.key);
+            } else {
+                this.keyPtr =  new ImmutableBytesPtr(key);
+                this.key = ByteUtil.copyKeyBytesIfNecessary(key);
+            }
+
+            newMutations();
+        }
+        
+        @SuppressWarnings("deprecation")
+        private void newMutations() {
+            this.setValues = new Put(this.key);
+            this.unsetValues = new Delete(this.key);
+            this.setValues.setWriteToWAL(!isWALDisabled());
+            this.unsetValues.setWriteToWAL(!isWALDisabled());
+       }
+
+        @Override
+        public List<Mutation> toRowMutations() {
+            List<Mutation> mutations = new ArrayList<Mutation>(3);
+            if (deleteRow != null) {
+                // Include only deleteRow mutation if present because it takes precedence over all others
+                mutations.add(deleteRow);
+            } else {
+                // 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
+                // way HBase works.
+                addQuietly(setValues, kvBuilder, kvBuilder.buildPut(keyPtr,
+                    SchemaUtil.getEmptyColumnFamilyPtr(getColumnFamilies()),
+                    QueryConstants.EMPTY_COLUMN_BYTES_PTR, ts, ByteUtil.EMPTY_BYTE_ARRAY_PTR));
+                mutations.add(setValues);
+                if (!unsetValues.isEmpty()) {
+                    mutations.add(unsetValues);
+                }
+            }
+            return mutations;
+        }
+
+        private void removeIfPresent(Mutation m, byte[] family, byte[] qualifier) {
+            Map<byte[],List<KeyValue>> familyMap = m.getFamilyMap();
+            List<KeyValue> kvs = familyMap.get(family);
+            if (kvs != null) {
+                Iterator<KeyValue> iterator = kvs.iterator();
+                while (iterator.hasNext()) {
+                    KeyValue kv = iterator.next();
+                    if (Bytes.compareTo(kv.getQualifier(), qualifier) == 0) {
+                        iterator.remove();
+                    }
+                }
+            }
+        }
+
+        @Override
+        public void setValue(PColumn column, Object value) {
+            byte[] byteValue = value == null ? ByteUtil.EMPTY_BYTE_ARRAY : column.getDataType().toBytes(value);
+            setValue(column, byteValue);
+        }
+
+        @Override
+        public void setValue(PColumn column, byte[] byteValue) {
+            deleteRow = null;
+            byte[] family = column.getFamilyName().getBytes();
+            byte[] qualifier = column.getName().getBytes();
+            PDataType type = column.getDataType();
+            // Check null, since some types have no byte representation for null
+            if (byteValue == null || byteValue.length == 0) {
+                if (!column.isNullable()) { 
+                    throw new ConstraintViolationException(name.getString() + "." + column.getName().getString() + " may not be null");
+                }
+                removeIfPresent(setValues, family, qualifier);
+                deleteQuietly(unsetValues, kvBuilder, kvBuilder.buildDeleteColumns(keyPtr, column
+                        .getFamilyName().getBytesPtr(), column.getName().getBytesPtr(), ts));
+            } else {
+            	Integer	byteSize = column.getByteSize();
+				if (byteSize != null && type.isFixedWidth()
+						&& byteValue.length <= byteSize) { 
+                    byteValue = StringUtil.padChar(byteValue, byteSize);
+                } else if (byteSize != null && byteValue.length > byteSize) {
+                    throw new ConstraintViolationException(name.getString() + "." + column.getName().getString() + " may not exceed " + byteSize + " bytes (" + type.toObject(byteValue) + ")");
+                }
+                removeIfPresent(unsetValues, family, qualifier);
+                addQuietly(setValues, kvBuilder, kvBuilder.buildPut(keyPtr, column.getFamilyName()
+                        .getBytesPtr(),
+                        column.getName().getBytesPtr(), ts, new ImmutableBytesPtr(byteValue)));
+            }
+        }
+        
+        @SuppressWarnings("deprecation")
+        @Override
+        public void delete() {
+            newMutations();
+            // FIXME: the version of the Delete constructor without the lock args was introduced
+            // in 0.94.4, thus if we try to use it here we can no longer use the 0.94.2 version
+            // of the client.
+            Delete delete = new Delete(key,ts,null);
+            deleteRow = delete;
+            deleteRow.setWriteToWAL(!isWALDisabled());
+        }
+    }
+
+    @Override
+    public PColumnFamily getColumnFamily(String familyName) throws ColumnFamilyNotFoundException {
+        PColumnFamily family = familyByString.get(familyName);
+        if (family == null) {
+            throw new ColumnFamilyNotFoundException(familyName);
+        }
+        return family;
+    }
+
+    @Override
+    public PColumnFamily getColumnFamily(byte[] familyBytes) throws ColumnFamilyNotFoundException {
+        PColumnFamily family = familyByBytes.get(familyBytes);
+        if (family == null) {
+            String familyName = Bytes.toString(familyBytes);
+            throw new ColumnFamilyNotFoundException(familyName);
+        }
+        return family;
+    }
+
+    @Override
+    public List<PColumn> getColumns() {
+        return allColumns;
+    }
+
+    @Override
+    public long getSequenceNumber() {
+        return sequenceNumber;
+    }
+
+    @Override
+    public long getTimeStamp() {
+        return timeStamp;
+    }
+
+    @Override
+    public PTableStats getTableStats() {
+        return stats;
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        byte[] schemaNameBytes = Bytes.readByteArray(input);
+        byte[] tableNameBytes = Bytes.readByteArray(input);
+        PName schemaName = PNameFactory.newName(schemaNameBytes);
+        PName tableName = PNameFactory.newName(tableNameBytes);
+        PTableType tableType = PTableType.values()[WritableUtils.readVInt(input)];
+        PIndexState indexState = null;
+        if (tableType == PTableType.INDEX) {
+            int ordinal = WritableUtils.readVInt(input);
+            if (ordinal >= 0) {
+                indexState = PIndexState.values()[ordinal];
+            }
+        }
+        long sequenceNumber = WritableUtils.readVLong(input);
+        long timeStamp = input.readLong();
+        byte[] pkNameBytes = Bytes.readByteArray(input);
+        PName pkName = pkNameBytes.length == 0 ? null : PNameFactory.newName(pkNameBytes);
+        Integer bucketNum = WritableUtils.readVInt(input);
+        int nColumns = WritableUtils.readVInt(input);
+        List<PColumn> columns = Lists.newArrayListWithExpectedSize(nColumns);
+        for (int i = 0; i < nColumns; i++) {
+            PColumn column = new PColumnImpl();
+            column.readFields(input);
+            columns.add(column);
+        }
+        int nIndexes = WritableUtils.readVInt(input);
+        List<PTable> indexes = Lists.newArrayListWithExpectedSize(nIndexes);
+        for (int i = 0; i < nIndexes; i++) {
+            PTable index = new PTableImpl();
+            index.readFields(input);
+            indexes.add(index);
+        }
+        boolean isImmutableRows = input.readBoolean();
+        Map<String, byte[][]> guidePosts = new HashMap<String, byte[][]>();
+        int size = WritableUtils.readVInt(input);
+        for (int i=0; i<size; i++) {
+            String key = WritableUtils.readString(input);
+            int valueSize = WritableUtils.readVInt(input);
+            byte[][] value = new byte[valueSize][];
+            for (int j=0; j<valueSize; j++) {
+                value[j] = Bytes.readByteArray(input);
+            }
+            guidePosts.put(key, value);
+        }
+        byte[] dataTableNameBytes = Bytes.readByteArray(input);
+        PName dataTableName = dataTableNameBytes.length == 0 ? null : PNameFactory.newName(dataTableNameBytes);
+        byte[] defaultFamilyNameBytes = Bytes.readByteArray(input);
+        PName defaultFamilyName = defaultFamilyNameBytes.length == 0 ? null : PNameFactory.newName(defaultFamilyNameBytes);
+        boolean disableWAL = input.readBoolean();
+        boolean multiTenant = input.readBoolean();
+        ViewType viewType = null;
+        String viewStatement = null;
+        List<PName> physicalNames = Collections.emptyList();
+        if (tableType == PTableType.VIEW) {
+            viewType = ViewType.fromSerializedValue(input.readByte());
+            byte[] viewStatementBytes = Bytes.readByteArray(input);
+            viewStatement = viewStatementBytes.length == 0 ? null : (String)PDataType.VARCHAR.toObject(viewStatementBytes);
+            int nPhysicalNames = WritableUtils.readVInt(input);
+            physicalNames = Lists.newArrayListWithExpectedSize(nPhysicalNames);
+            for (int i = 0; i < nPhysicalNames; i++) {
+                byte[] physicalNameBytes = Bytes.readByteArray(input);
+                physicalNames.add(PNameFactory.newName(physicalNameBytes));
+            }
+        }
+        PTableStats stats = new PTableStatsImpl(guidePosts);
+        try {
+            init(schemaName, tableName, tableType, indexState, timeStamp, sequenceNumber, pkName,
+                 bucketNum.equals(NO_SALTING) ? null : bucketNum, columns, stats, dataTableName,
+                 indexes, isImmutableRows, physicalNames, defaultFamilyName,
+                 viewStatement, disableWAL, multiTenant, viewType);
+        } catch (SQLException e) {
+            throw new RuntimeException(e); // Impossible
+        }
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+        Bytes.writeByteArray(output, schemaName.getBytes());
+        Bytes.writeByteArray(output, tableName.getBytes());
+        WritableUtils.writeVInt(output, type.ordinal());
+        if (type == PTableType.INDEX) {
+            WritableUtils.writeVInt(output, state == null ? -1 : state.ordinal());
+        }
+        WritableUtils.writeVLong(output, sequenceNumber);
+        output.writeLong(timeStamp);
+        Bytes.writeByteArray(output, pkName == null ? ByteUtil.EMPTY_BYTE_ARRAY : pkName.getBytes());
+        int offset = 0, nColumns = allColumns.size();
+        if (bucketNum == null) {
+            WritableUtils.writeVInt(output, NO_SALTING);
+        } else {
+            offset = 1;
+            nColumns--;
+            WritableUtils.writeVInt(output, bucketNum);
+        }
+        WritableUtils.writeVInt(output, nColumns);
+        for (int i = offset; i < allColumns.size(); i++) {
+            PColumn column = allColumns.get(i);
+            column.write(output);
+        }
+        WritableUtils.writeVInt(output, indexes.size());
+        for (PTable index: indexes) {
+            index.write(output);
+        }
+        output.writeBoolean(isImmutableRows);
+        stats.write(output);
+        Bytes.writeByteArray(output, parentTableName == null ? ByteUtil.EMPTY_BYTE_ARRAY : parentTableName.getBytes());
+        Bytes.writeByteArray(output, defaultFamilyName == null ? ByteUtil.EMPTY_BYTE_ARRAY : defaultFamilyName.getBytes());
+        output.writeBoolean(disableWAL);
+        output.writeBoolean(multiTenant);
+        if (type == PTableType.VIEW) {
+            output.writeByte(viewType.getSerializedValue());
+            Bytes.writeByteArray(output, viewStatement == null ? ByteUtil.EMPTY_BYTE_ARRAY : PDataType.VARCHAR.toBytes(viewStatement));
+            WritableUtils.writeVInt(output, physicalNames.size());
+            for (int i = 0; i < physicalNames.size(); i++) {
+                Bytes.writeByteArray(output, physicalNames.get(i).getBytes());
+            }
+        }
+    }
+
+    @Override
+    public PColumn getPKColumn(String name) throws ColumnNotFoundException {
+        List<PColumn> columns = columnsByName.get(name);
+        int size = columns.size();
+        if (size == 0) {
+            throw new ColumnNotFoundException(name);
+        }
+        if (size > 1) {
+            do {
+                PColumn column = columns.get(--size);
+                if (column.getFamilyName() == null) {
+                    return column;
+                }
+            } while (size > 0);
+            throw new ColumnNotFoundException(name);
+        }
+        return columns.get(0);
+    }
+
+    @Override
+    public PName getPKName() {
+        return pkName;
+    }
+
+    @Override
+    public RowKeySchema getRowKeySchema() {
+        return rowKeySchema;
+    }
+
+    @Override
+    public Integer getBucketNum() {
+        return bucketNum;
+    }
+
+    @Override
+    public List<PTable> getIndexes() {
+        return indexes;
+    }
+
+    @Override
+    public PIndexState getIndexState() {
+        return state;
+    }
+
+    @Override
+    public PName getParentTableName() {
+        return parentTableName;
+    }
+
+    @Override
+    public PName getParentName() {
+        return parentName;
+    }
+
+    @Override
+    public synchronized IndexMaintainer getIndexMaintainer(PTable dataTable) {
+        if (indexMaintainer == null) {
+            indexMaintainer = IndexMaintainer.create(dataTable, this);
+        }
+        return indexMaintainer;
+    }
+
+    @Override
+    public synchronized void getIndexMaintainers(ImmutableBytesWritable ptr) {
+        if (indexMaintainersPtr == null) {
+            indexMaintainersPtr = new ImmutableBytesWritable();
+            if (indexes.isEmpty()) {
+                indexMaintainersPtr.set(ByteUtil.EMPTY_BYTE_ARRAY);
+            } else {
+                IndexMaintainer.serialize(this, indexMaintainersPtr);
+            }
+        }
+        ptr.set(indexMaintainersPtr.get(), indexMaintainersPtr.getOffset(), indexMaintainersPtr.getLength());
+    }
+    
+    @Override
+    public PName getPhysicalName() {
+        return physicalNames.isEmpty() ? getName() : physicalNames.get(0);
+    }
+    
+    @Override
+    public List<PName> getPhysicalNames() {
+        return physicalNames;
+    }
+
+    @Override
+    public PName getDefaultFamilyName() {
+        return defaultFamilyName;
+    }
+
+    @Override
+    public String getViewStatement() {
+        return viewStatement;
+    }
+
+    @Override
+    public boolean isWALDisabled() {
+        return disableWAL;
+    }
+}
\ No newline at end of file


[29/51] [partial] Initial commit of master branch from github

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CountAggregateFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CountAggregateFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CountAggregateFunction.java
new file mode 100644
index 0000000..5e7dceb
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CountAggregateFunction.java
@@ -0,0 +1,116 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.LiteralExpression;
+import org.apache.phoenix.expression.aggregator.Aggregator;
+import org.apache.phoenix.expression.aggregator.CountAggregator;
+import org.apache.phoenix.expression.aggregator.LongSumAggregator;
+import org.apache.phoenix.parse.FunctionParseNode.Argument;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.util.SchemaUtil;
+
+
+/**
+ * 
+ * Built-in function for COUNT(<expression>) aggregate function,
+ * for example COUNT(foo), COUNT(1), COUNT(*)
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+@BuiltInFunction(name=CountAggregateFunction.NAME, args= {@Argument()} )
+public class CountAggregateFunction extends SingleAggregateFunction {
+    public static final String NAME = "COUNT";
+    public static final List<Expression> STAR = Arrays.<Expression>asList(LiteralExpression.newConstant(1, true));
+    public static final String NORMALIZED_NAME = SchemaUtil.normalizeIdentifier(NAME);
+    
+    public CountAggregateFunction() {
+    }
+    
+    public CountAggregateFunction(List<Expression> childExpressions) {
+        super(childExpressions);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) return true;
+        if (obj == null) return false;
+        if (getClass() != obj.getClass()) return false;
+        CountAggregateFunction other = (CountAggregateFunction)obj;
+        return (isConstantExpression() && other.isConstantExpression()) || children.equals(other.getChildren());
+    }
+
+    @Override
+    public int hashCode() {
+        return isConstantExpression() ? 0 : super.hashCode();
+    }
+
+    /**
+     * The COUNT function never returns null
+     */
+    @Override
+    public boolean isNullable() {
+        return false;
+    }
+    
+    @Override
+    public PDataType getDataType() {
+        return PDataType.LONG;
+    }
+
+    @Override 
+    public LongSumAggregator newClientAggregator() {
+        // Since COUNT can never be null, ensure the aggregator is not nullable.
+        // This allows COUNT(*) to return 0 with the initial state of ClientAggregators
+        // when no rows are returned. 
+        return new LongSumAggregator(null) {
+            @Override
+            public boolean isNullable() {
+                return false;
+            }
+        };
+    }
+    
+    @Override 
+    public Aggregator newServerAggregator(Configuration conf) {
+        return new CountAggregator();
+    }
+    
+    @Override
+    public String getName() {
+        return NAME;
+    }
+
+    @Override
+    public Aggregator newServerAggregator(Configuration config, ImmutableBytesWritable ptr) {
+        LongSumAggregator sumAgg = newClientAggregator();
+        sumAgg.aggregate(null, ptr);
+        return new CountAggregator(sumAgg);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CurrentDateFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CurrentDateFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CurrentDateFunction.java
new file mode 100644
index 0000000..ad9e989
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CurrentDateFunction.java
@@ -0,0 +1,69 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.expression.CurrentDateTimeFunction;
+import org.apache.phoenix.parse.CurrentDateParseNode;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+
+/**
+ * 
+ * Function that returns the current date accurate to the millisecond. Note that this
+ * function is never evaluated on the server-side, instead the server side date is
+ * retrieved (piggy-backed on the call to check that the metadata is up-to-date) and
+ * passed into this function at create time.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+@BuiltInFunction(name=CurrentDateFunction.NAME, nodeClass=CurrentDateParseNode.class, args= {} )
+public class CurrentDateFunction extends CurrentDateTimeFunction {
+    public static final String NAME = "CURRENT_DATE";
+    private final ImmutableBytesWritable currentDate = new ImmutableBytesWritable(new byte[PDataType.DATE.getByteSize()]);
+    
+    public CurrentDateFunction() {
+        this(System.currentTimeMillis());
+    }
+
+    public CurrentDateFunction(long timeStamp) {
+        getDataType().getCodec().encodeLong(timeStamp, currentDate);
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        ptr.set(currentDate.get(), 0, PDataType.DATE.getByteSize());
+        return true;
+    }
+
+    @Override
+    public final PDataType getDataType() {
+        return PDataType.DATE;
+    }
+
+    @Override
+    public String getName() {
+        return NAME;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CurrentTimeFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CurrentTimeFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CurrentTimeFunction.java
new file mode 100644
index 0000000..71a1602
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CurrentTimeFunction.java
@@ -0,0 +1,69 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.expression.CurrentDateTimeFunction;
+import org.apache.phoenix.parse.CurrentTimeParseNode;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+
+/**
+ * 
+ * Function that returns the current date accurate to the millisecond. Note that this
+ * function is never evaluated on the server-side, instead the server side date is
+ * retrieved (piggy-backed on the call to check that the metadata is up-to-date) and
+ * passed into this function at create time.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+@BuiltInFunction(name=CurrentTimeFunction.NAME, nodeClass=CurrentTimeParseNode.class, args={} )
+public class CurrentTimeFunction extends CurrentDateTimeFunction {
+    public static final String NAME = "CURRENT_TIME";
+    private final ImmutableBytesWritable currentDate = new ImmutableBytesWritable(new byte[PDataType.TIME.getByteSize()]);
+    
+    public CurrentTimeFunction() {
+        this(System.currentTimeMillis());
+    }
+
+    public CurrentTimeFunction(long timeStamp) {
+        getDataType().getCodec().encodeLong(timeStamp, currentDate);
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        ptr.set(currentDate.get(), 0, PDataType.TIME.getByteSize());
+        return true;
+    }
+
+    @Override
+    public final PDataType getDataType() {
+        return PDataType.TIME;
+    }
+
+    @Override
+    public String getName() {
+        return NAME;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/DelegateConstantToCountAggregateFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/DelegateConstantToCountAggregateFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/DelegateConstantToCountAggregateFunction.java
new file mode 100644
index 0000000..744e5bb
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/DelegateConstantToCountAggregateFunction.java
@@ -0,0 +1,75 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+
+/**
+ * 
+ * Base class for non composite aggregation functions that optimize aggregation by
+ * delegating to {@link CountAggregateFunction} when the child expression is a 
+ * constant.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+abstract public class DelegateConstantToCountAggregateFunction extends SingleAggregateFunction {
+    private static final ImmutableBytesWritable ZERO = new ImmutableBytesWritable(PDataType.LONG.toBytes(0L));
+    private CountAggregateFunction delegate;
+    
+    public DelegateConstantToCountAggregateFunction() {
+    }
+    
+    public DelegateConstantToCountAggregateFunction(List<Expression> childExpressions, CountAggregateFunction delegate) {
+        super(childExpressions);
+        // Using a delegate here causes us to optimize the number of aggregators
+        // by sharing the CountAggregator across functions. On the server side,
+        // this will always be null, since if it had not been null on the client,
+        // the function would not have been transfered over (the delegate would
+        // have instead).
+        this.delegate = delegate;
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        if (delegate == null) {
+            return super.evaluate(tuple, ptr);
+        }
+        delegate.evaluate(tuple, ptr);
+        if (PDataType.LONG.compareTo(ptr,ZERO) == 0) {
+            return false;
+        }
+        return true;
+    }
+
+
+    @Override
+    protected SingleAggregateFunction getDelegate() {
+        return delegate != null ? delegate : super.getDelegate();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/DistinctCountAggregateFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/DistinctCountAggregateFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/DistinctCountAggregateFunction.java
new file mode 100644
index 0000000..b6ea58e
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/DistinctCountAggregateFunction.java
@@ -0,0 +1,124 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.aggregator.Aggregator;
+import org.apache.phoenix.expression.aggregator.DistinctCountClientAggregator;
+import org.apache.phoenix.expression.aggregator.DistinctValueWithCountServerAggregator;
+import org.apache.phoenix.parse.FunctionParseNode.Argument;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.SchemaUtil;
+
+
+/**
+ * 
+ * Built-in function for COUNT(distinct <expression>) aggregate function,
+ *
+ * @author anoopsjohn
+ * @since 1.2.1
+ */
+@BuiltInFunction(name=DistinctCountAggregateFunction.NAME, args= {@Argument()} )
+public class DistinctCountAggregateFunction extends DelegateConstantToCountAggregateFunction {
+    public static final String NAME = "DISTINCT_COUNT";
+    public static final String NORMALIZED_NAME = SchemaUtil.normalizeIdentifier(NAME);
+    public final static byte[] ZERO = PDataType.LONG.toBytes(0L);
+    public final static byte[] ONE = PDataType.LONG.toBytes(1L);
+    
+    public DistinctCountAggregateFunction() {
+    }
+
+    public DistinctCountAggregateFunction(List<Expression> childExpressions) {
+        this(childExpressions, null);
+    }
+
+    public DistinctCountAggregateFunction(List<Expression> childExpressions,
+            CountAggregateFunction delegate) {
+        super(childExpressions, delegate);
+        assert childExpressions.size() == 1;
+    }
+    
+    @Override
+    public int hashCode() {
+        return isConstantExpression() ? 0 : super.hashCode();
+    }
+
+    /**
+     * The COUNT function never returns null
+     */
+    @Override
+    public boolean isNullable() {
+        return false;
+    }
+    
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) return true;
+        if (obj == null) return false;
+        if (getClass() != obj.getClass()) return false;
+        DistinctCountAggregateFunction other = (DistinctCountAggregateFunction)obj;
+        return (isConstantExpression() && other.isConstantExpression()) || children.equals(other.getChildren());
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return PDataType.LONG;
+    }
+
+    @Override 
+    public DistinctCountClientAggregator newClientAggregator() {
+        return new DistinctCountClientAggregator(getAggregatorExpression().getColumnModifier());
+    }
+    
+    @Override 
+    public Aggregator newServerAggregator(Configuration conf) {
+        return new DistinctValueWithCountServerAggregator(conf);
+    }
+    
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        // TODO: optimize query plan of this to run scan serially for a limit of one row
+        if (!super.evaluate(tuple, ptr)) {
+            ptr.set(ZERO); // If evaluate returns false, then no rows were found, so result is 0
+        } else if (isConstantExpression()) {
+            ptr.set(ONE); // Otherwise, we found one or more rows, so a distinct on a constant is 1
+        }
+        return true; // Always evaluates to a LONG value
+    }
+    
+    @Override
+    public String getName() {
+        return NAME;
+    }
+
+    @Override
+    public Aggregator newServerAggregator(Configuration config, ImmutableBytesWritable ptr) {
+        DistinctCountClientAggregator clientAgg = newClientAggregator();
+        clientAgg.aggregate(null, ptr);
+        return new DistinctValueWithCountServerAggregator(config, clientAgg);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/DistinctValueWithCountAggregateFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/DistinctValueWithCountAggregateFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/DistinctValueWithCountAggregateFunction.java
new file mode 100644
index 0000000..b12c96a
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/DistinctValueWithCountAggregateFunction.java
@@ -0,0 +1,50 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.aggregator.Aggregator;
+import org.apache.phoenix.expression.aggregator.DistinctValueWithCountClientAggregator;
+import org.apache.phoenix.expression.aggregator.DistinctValueWithCountServerAggregator;
+
+public abstract class DistinctValueWithCountAggregateFunction extends SingleAggregateFunction {
+
+    public DistinctValueWithCountAggregateFunction() {
+    }
+
+    public DistinctValueWithCountAggregateFunction(List<Expression> children) {
+        super(children);
+    }
+
+    @Override
+    abstract public DistinctValueWithCountClientAggregator newClientAggregator();
+    
+    @Override
+    public Aggregator newServerAggregator(Configuration config, ImmutableBytesWritable ptr) {
+        DistinctValueWithCountClientAggregator clientAgg = newClientAggregator();
+        clientAgg.aggregate(null, ptr);
+        return new DistinctValueWithCountServerAggregator(config, clientAgg);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/FloorDateExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/FloorDateExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/FloorDateExpression.java
new file mode 100644
index 0000000..bfc3969
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/FloorDateExpression.java
@@ -0,0 +1,88 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import com.google.common.collect.Lists;
+import org.apache.phoenix.expression.CoerceExpression;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.schema.PDataType;
+
+/**
+ * 
+ * Class encapsulating the FLOOR operation on 
+ * a column/literal of type {@link org.apache.phoenix.schema.PDataType#DATE}.
+ *
+ * @author samarth.jain
+ * @since 3.0.0
+ */
+public class FloorDateExpression extends RoundDateExpression {
+    
+    public FloorDateExpression() {}
+    
+    private FloorDateExpression(List<Expression> children) {
+        super(children);
+    }
+    
+    public static Expression create(List<Expression> children) throws SQLException {
+        Expression firstChild = children.get(0);
+        PDataType firstChildDataType = firstChild.getDataType();
+        if (firstChildDataType == PDataType.TIMESTAMP || firstChildDataType == PDataType.UNSIGNED_TIMESTAMP){
+            // Coerce TIMESTAMP to DATE, as the nanos has no affect
+            List<Expression> newChildren = Lists.newArrayListWithExpectedSize(children.size());
+            newChildren.add(CoerceExpression.create(firstChild, firstChildDataType == PDataType.TIMESTAMP ? PDataType.DATE : PDataType.UNSIGNED_DATE));
+            newChildren.addAll(children.subList(1, children.size()));
+            children = newChildren;
+        }
+        return new FloorDateExpression(children);
+    }
+    
+    /**
+     * @param timeUnit - unit of time to round up to.
+     * Creates a {@link FloorDateExpression} with default multiplier of 1.
+     */
+    public static Expression create(Expression expr, TimeUnit timeUnit) throws SQLException {
+        return create(expr, timeUnit, 1);
+    }
+    
+    /**
+     * @param timeUnit - unit of time to round up to
+     * @param multiplier - determines the roll up window size.
+     * Create a {@link FloorDateExpression}. 
+     */
+    public static Expression create(Expression expr, TimeUnit timeUnit, int multiplier) throws SQLException {
+        Expression timeUnitExpr = getTimeUnitExpr(timeUnit);
+        Expression defaultMultiplierExpr = getMultiplierExpr(multiplier);
+        List<Expression> expressions = Lists.newArrayList(expr, timeUnitExpr, defaultMultiplierExpr);
+        return create(expressions);
+    }
+   
+    @Override
+    protected long getRoundUpAmount() {
+        return 0;
+    }
+    
+    @Override
+    public String getName() {
+        return FloorFunction.NAME;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/FloorDecimalExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/FloorDecimalExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/FloorDecimalExpression.java
new file mode 100644
index 0000000..e8353f2
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/FloorDecimalExpression.java
@@ -0,0 +1,77 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import java.math.RoundingMode;
+import java.sql.SQLException;
+import java.util.List;
+
+import com.google.common.collect.Lists;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.LiteralExpression;
+import org.apache.phoenix.schema.PDataType;
+
+/**
+ * 
+ * Class encapsulating the FLOOR operation on 
+ * a column/literal of type {@link org.apache.phoenix.schema.PDataType#DECIMAL}.
+ *
+ * @author samarth.jain
+ * @since 3.0.0
+ */
+public class FloorDecimalExpression extends RoundDecimalExpression {
+    
+    public FloorDecimalExpression() {}
+    
+    public FloorDecimalExpression(List<Expression> children) {
+        super(children);
+    }
+    
+    /**
+     * Creates a {@link FloorDecimalExpression} with rounding scale given by @param scale. 
+     *
+     */
+    public static Expression create(Expression expr, int scale) throws SQLException {
+        if (expr.getDataType().isCoercibleTo(PDataType.LONG)) {
+            return expr;
+        }
+        Expression scaleExpr = LiteralExpression.newConstant(scale, PDataType.INTEGER, true);
+        List<Expression> expressions = Lists.newArrayList(expr, scaleExpr);
+        return new FloorDecimalExpression(expressions);
+    }
+    
+    /**
+     * Creates a {@link FloorDecimalExpression} with a default scale of 0 used for rounding. 
+     *
+     */
+    public static Expression create(Expression expr) throws SQLException {
+        return create(expr, 0);
+    }
+    
+    @Override
+    protected RoundingMode getRoundingMode() {
+        return RoundingMode.FLOOR;
+    }
+    
+    @Override
+    public String getName() {
+        return FloorFunction.NAME;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/FloorFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/FloorFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/FloorFunction.java
new file mode 100644
index 0000000..cceb966
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/FloorFunction.java
@@ -0,0 +1,56 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import java.util.List;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.parse.FloorParseNode;
+import org.apache.phoenix.parse.FunctionParseNode.Argument;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.schema.PDataType;
+/**
+ * 
+ * Base class for built-in FLOOR function.
+ *
+ * @author samarth.jain
+ * @since 3.0.0
+ */
+@BuiltInFunction(name = FloorFunction.NAME,
+                 nodeClass = FloorParseNode.class,
+                 args = {
+                        @Argument(allowedTypes={PDataType.TIMESTAMP, PDataType.DECIMAL}),
+                        @Argument(allowedTypes={PDataType.VARCHAR, PDataType.INTEGER}, defaultValue = "null", isConstant=true),
+                        @Argument(allowedTypes={PDataType.INTEGER}, defaultValue="1", isConstant=true)
+                        } 
+                )
+public abstract class FloorFunction extends ScalarFunction {
+    
+    public static final String NAME = "FLOOR";
+    
+    public FloorFunction(List<Expression> children) {
+        super(children);
+    }
+    
+    @Override
+    public String getName() {
+        return NAME;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/FunctionArgumentType.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/FunctionArgumentType.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/FunctionArgumentType.java
new file mode 100644
index 0000000..0284caf
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/FunctionArgumentType.java
@@ -0,0 +1,35 @@
+package org.apache.phoenix.expression.function;
+
+import java.text.DecimalFormat;
+import java.text.Format;
+
+import org.apache.phoenix.util.DateUtil;
+
+public enum FunctionArgumentType {
+    TEMPORAL {
+        @Override
+        public Format getFormatter(String format) {
+            return DateUtil.getDateFormatter(format);
+        }
+    }, 
+    NUMERIC {
+        @Override
+        public Format getFormatter(String format) {
+            return new DecimalFormat(format);
+        }
+    },
+    CHAR {
+        @Override
+        public Format getFormatter(String format) {
+            return getDecimalFormat(format);
+        }
+    };        
+
+    public abstract Format getFormatter(String format);
+    
+    private static DecimalFormat getDecimalFormat(String formatString) {
+        DecimalFormat result = new DecimalFormat(formatString);
+        result.setParseBigDecimal(true);
+        return result;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/FunctionExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/FunctionExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/FunctionExpression.java
new file mode 100644
index 0000000..db305d3
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/FunctionExpression.java
@@ -0,0 +1,71 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import java.util.List;
+
+import org.apache.phoenix.expression.BaseCompoundExpression;
+import org.apache.phoenix.expression.Expression;
+
+/**
+ * 
+ * Compiled representation of a built-in function
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public abstract class FunctionExpression extends BaseCompoundExpression {
+    public enum OrderPreserving {NO, YES_IF_LAST, YES};
+    public FunctionExpression() {
+    }
+    
+    public FunctionExpression(List<Expression> children) {
+        super(children);
+    }
+    
+    /**
+     * Determines whether or not the result of the function invocation
+     * will be ordered in the same way as the input to the function.
+     * Returning YES enables an optimization to occur when a
+     * GROUP BY contains function invocations using the leading PK
+     * column(s).
+     * @return YES if the function invocation will always preserve order for
+     * the inputs versus the outputs and false otherwise, YES_IF_LAST if the
+     * function preserves order, but any further column reference would not
+     * continue to preserve order, and NO if the function does not preserve
+     * order.
+     */
+    public OrderPreserving preservesOrder() {
+        return OrderPreserving.NO;
+    }
+
+    abstract public String getName();
+    
+    @Override
+    public final String toString() {
+        StringBuilder buf = new StringBuilder(getName() + "(");
+        for (int i = 0; i < children.size() - 1; i++) {
+            buf.append(children.get(i) + ", ");
+        }
+        buf.append(children.get(children.size()-1) + ")");
+        return buf.toString();
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/IndexStateNameFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/IndexStateNameFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/IndexStateNameFunction.java
new file mode 100644
index 0000000..41890bc
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/IndexStateNameFunction.java
@@ -0,0 +1,81 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.parse.FunctionParseNode.Argument;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.PIndexState;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+
+/**
+ * 
+ * Function used to get the index state name from the serialized byte value
+ * Usage:
+ * IndexStateName('a')
+ * will return 'ACTIVE'
+ * 
+ * @author jtaylor
+ * @since 2.1
+ */
+@BuiltInFunction(name=IndexStateNameFunction.NAME, args= {
+    @Argument(allowedTypes=PDataType.CHAR)} )
+public class IndexStateNameFunction extends ScalarFunction {
+    public static final String NAME = "IndexStateName";
+
+    public IndexStateNameFunction() {
+    }
+    
+    public IndexStateNameFunction(List<Expression> children) throws SQLException {
+        super(children);
+    }
+    
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        Expression child = children.get(0);
+        if (!child.evaluate(tuple, ptr)) {
+            return false;
+        }
+        if (ptr.getLength() == 0) {
+            return true;
+        }
+        byte serializedByte = ptr.get()[ptr.getOffset()];
+        PIndexState indexState = PIndexState.fromSerializedValue(serializedByte);
+        ptr.set(indexState.toBytes());
+        return true;
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return PDataType.VARCHAR;
+    }
+    
+    @Override
+    public String getName() {
+        return NAME;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/InvertFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/InvertFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/InvertFunction.java
new file mode 100644
index 0000000..72ac31a
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/InvertFunction.java
@@ -0,0 +1,129 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.compile.KeyPart;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.parse.FunctionParseNode.Argument;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.query.KeyRange;
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+@BuiltInFunction(name = InvertFunction.NAME, args = { @Argument() })
+public class InvertFunction extends ScalarFunction {
+    public static final String NAME = "INVERT";
+
+    public InvertFunction() throws SQLException {}
+
+    public InvertFunction(List<Expression> children) throws SQLException {
+        super(children);
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        if (!getChildExpression().evaluate(tuple, ptr)) { return false; }
+        if (ptr.getLength() == 0) { return true; }
+        byte[] buf = new byte[ptr.getLength()];
+        ColumnModifier.SORT_DESC.apply(ptr.get(), ptr.getOffset(), buf, 0, ptr.getLength());
+        ptr.set(buf);
+        return true;
+    }
+
+    @Override
+    public ColumnModifier getColumnModifier() {
+        return getChildExpression().getColumnModifier() == null ? ColumnModifier.SORT_DESC : null;
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return getChildExpression().getDataType();
+    }
+
+    @Override
+    public Integer getMaxLength() {
+        return getChildExpression().getMaxLength();
+    }
+
+    @Override
+    public Integer getByteSize() {
+        return getChildExpression().getByteSize();
+    }
+
+    @Override
+    public boolean isNullable() {
+        return getChildExpression().isNullable();
+    }
+
+    @Override
+    public String getName() {
+        return NAME;
+    }
+
+    /**
+     * INVERT may be optimized through
+     */
+    @Override
+    public int getKeyFormationTraversalIndex() {
+        return 0;
+    }
+
+    /**
+     * Invert the childPart key range
+     */
+    @Override
+    public KeyPart newKeyPart(final KeyPart childPart) {
+        return new KeyPart() {
+
+            @Override
+            public KeyRange getKeyRange(CompareOp op, Expression rhs) {
+                KeyRange range = childPart.getKeyRange(op, rhs);
+                return range.invert();
+            }
+
+            @Override
+            public List<Expression> getExtractNodes() {
+                return childPart.getExtractNodes();
+            }
+
+            @Override
+            public PColumn getColumn() {
+                return childPart.getColumn();
+            }
+        };
+    }
+    
+    @Override
+    public OrderPreserving preservesOrder() {
+        return OrderPreserving.YES;
+    }
+
+    private Expression getChildExpression() {
+        return children.get(0);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/LTrimFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/LTrimFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/LTrimFunction.java
new file mode 100644
index 0000000..1622f55
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/LTrimFunction.java
@@ -0,0 +1,115 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.parse.FunctionParseNode.Argument;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.StringUtil;
+
+
+/**
+ * 
+ * Implementation of the LTrim(<string>) build-in function. It removes from the left end of
+ * <string> space character and other function bytes in single byte utf8 characters 
+ * set.
+ * 
+ * @author zhuang
+ * @since 0.1
+ */
+@BuiltInFunction(name=LTrimFunction.NAME, args={
+    @Argument(allowedTypes={PDataType.VARCHAR})})
+public class LTrimFunction extends ScalarFunction {
+    public static final String NAME = "LTRIM";
+
+    private Integer byteSize;
+
+    public LTrimFunction() { }
+
+    public LTrimFunction(List<Expression> children) throws SQLException {
+        super(children);
+        if (getStringExpression().getDataType().isFixedWidth()) {
+            byteSize = getStringExpression().getByteSize();
+        }
+    }
+
+    private Expression getStringExpression() {
+        return children.get(0);
+    }
+
+    @Override
+    public ColumnModifier getColumnModifier() {
+        return children.get(0).getColumnModifier();
+    }    
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        // Starting from the front of the byte, look for all single bytes at the end of the string
+        // that is below SPACE_UTF8 (space and control characters) or 0x7f (control chars).
+        if (!getStringExpression().evaluate(tuple, ptr)) {
+            return false;
+        }
+        
+        if (ptr.getLength() == 0) {
+            ptr.set(ByteUtil.EMPTY_BYTE_ARRAY);
+            return true;
+        }
+        byte[] string = ptr.get();
+        int offset = ptr.getOffset();
+        int length = ptr.getLength();
+        
+        ColumnModifier columnModifier = getStringExpression().getColumnModifier();
+        // TODO: when we have ColumnModifier.REVERSE, we'll need to trim from the end instead of
+        // the beginning (just delegate to RTrimFunction or replace from ExpressionCompiler instead?)
+        int i = StringUtil.getFirstNonBlankCharIdxFromStart(string, offset, length, columnModifier);
+        if (i == offset + length) {
+            ptr.set(ByteUtil.EMPTY_BYTE_ARRAY);
+            return true;
+        }
+        
+        ptr.set(string, i, offset + length - i);
+        return true;
+    }
+    
+    @Override
+    public Integer getByteSize() {
+        return byteSize;
+    }
+
+    @Override
+    public PDataType getDataType() {
+      return PDataType.VARCHAR;
+    }
+
+    @Override
+    public String getName() {
+        return NAME;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/LengthFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/LengthFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/LengthFunction.java
new file mode 100644
index 0000000..770f77a
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/LengthFunction.java
@@ -0,0 +1,96 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import java.io.UnsupportedEncodingException;
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.parse.FunctionParseNode.Argument;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.StringUtil;
+
+
+/**
+ * 
+ * Implementation of the LENGTH(<string>) build-in function. <string> is the string
+ * of characters we want to find the length of. If <string> is NULL or empty, null
+ * is returned.
+ * 
+ * @author zhuang
+ * @since 0.1
+ */
+@BuiltInFunction(name=LengthFunction.NAME, args={
+    @Argument(allowedTypes={PDataType.VARCHAR})} )
+public class LengthFunction extends ScalarFunction {
+    public static final String NAME = "LENGTH";
+
+    public LengthFunction() { }
+
+    public LengthFunction(List<Expression> children) throws SQLException {
+        super(children);
+    }
+
+    private Expression getStringExpression() {
+        return children.get(0);
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        Expression child = getStringExpression();
+        if (!child.evaluate(tuple, ptr)) {
+            return false;
+        }
+        if (ptr.getLength() == 0) {
+            ptr.set(ByteUtil.EMPTY_BYTE_ARRAY);
+            return true;
+        }
+        int len;
+        if (child.getDataType() == PDataType.CHAR) {
+            // Only single-byte characters allowed in CHAR
+            len = ptr.getLength();
+        } else {
+            try {
+                len = StringUtil.calculateUTF8Length(ptr.get(), ptr.getOffset(), ptr.getLength(), child.getColumnModifier());
+            } catch (UnsupportedEncodingException e) {
+                return false;
+            }
+        }
+        ptr.set(PDataType.INTEGER.toBytes(len));
+        return true;
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return PDataType.INTEGER;
+    }
+
+    @Override
+    public String getName() {
+        return NAME;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/MaxAggregateFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/MaxAggregateFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/MaxAggregateFunction.java
new file mode 100644
index 0000000..72e5984
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/MaxAggregateFunction.java
@@ -0,0 +1,75 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.aggregator.Aggregator;
+import org.apache.phoenix.expression.aggregator.MaxAggregator;
+import org.apache.phoenix.parse.FunctionParseNode.Argument;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.parse.MaxAggregateParseNode;
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.PDataType;
+
+
+
+/**
+ * Built-in function for finding MAX.
+ * 
+ * @author syyang
+ * @since 0.1
+ */
+@BuiltInFunction(name=MaxAggregateFunction.NAME, nodeClass=MaxAggregateParseNode.class, args= {@Argument()} )
+public class MaxAggregateFunction extends MinAggregateFunction {
+    public static final String NAME = "MAX";
+
+    public MaxAggregateFunction() {
+    }
+    
+    public MaxAggregateFunction(List<Expression> childExpressions, CountAggregateFunction delegate) {
+        super(childExpressions, delegate);
+    }
+
+    @Override 
+    public Aggregator newServerAggregator(Configuration conf) {
+        final PDataType type = getAggregatorExpression().getDataType();
+        ColumnModifier columnModifier = getAggregatorExpression().getColumnModifier();
+        return new MaxAggregator(columnModifier) {
+            @Override
+            public PDataType getDataType() {
+                return type;
+            }
+        };
+    }
+    
+    @Override
+    public String getName() {
+        return NAME;
+    }
+    
+    @Override
+    public ColumnModifier getColumnModifier() {
+       return getAggregatorExpression().getColumnModifier(); 
+    }    
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/MinAggregateFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/MinAggregateFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/MinAggregateFunction.java
new file mode 100644
index 0000000..26e8fd9
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/MinAggregateFunction.java
@@ -0,0 +1,89 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.aggregator.Aggregator;
+import org.apache.phoenix.expression.aggregator.MinAggregator;
+import org.apache.phoenix.parse.FunctionParseNode.Argument;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.parse.MinAggregateParseNode;
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+
+
+/**
+ * Built-in function for finding MIN.
+ * 
+ * @author syyang
+ * @since 0.1
+ */
+@BuiltInFunction(name=MinAggregateFunction.NAME, nodeClass=MinAggregateParseNode.class, args= {@Argument()} )
+public class MinAggregateFunction extends DelegateConstantToCountAggregateFunction {
+    public static final String NAME = "MIN";
+
+    public MinAggregateFunction() {
+    }
+    
+    public MinAggregateFunction(List<Expression> childExpressions, CountAggregateFunction delegate) {
+        super(childExpressions, delegate);
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        boolean wasEvaluated = super.evaluate(tuple, ptr);
+        if (!wasEvaluated) {
+            return false;
+        }
+        if (isConstantExpression()) {
+            getAggregatorExpression().evaluate(tuple, ptr);
+        }
+        return true;
+    }
+
+    @Override 
+    public Aggregator newServerAggregator(Configuration conf) {
+        final PDataType type = getAggregatorExpression().getDataType();
+        ColumnModifier columnModifier = getAggregatorExpression().getColumnModifier();
+        return new MinAggregator(columnModifier) {
+            @Override
+            public PDataType getDataType() {
+                return type;
+            }
+        };
+    }
+    
+    @Override
+    public ColumnModifier getColumnModifier() {
+       return getAggregatorExpression().getColumnModifier(); 
+    }
+    
+    @Override
+    public String getName() {
+        return NAME;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/PercentRankAggregateFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/PercentRankAggregateFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/PercentRankAggregateFunction.java
new file mode 100644
index 0000000..bf6c975
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/PercentRankAggregateFunction.java
@@ -0,0 +1,74 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.aggregator.Aggregator;
+import org.apache.phoenix.expression.aggregator.DistinctValueWithCountClientAggregator;
+import org.apache.phoenix.expression.aggregator.DistinctValueWithCountServerAggregator;
+import org.apache.phoenix.expression.aggregator.PercentRankClientAggregator;
+import org.apache.phoenix.parse.FunctionParseNode.Argument;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.schema.PDataType;
+
+/**
+ * 
+ * PERCENT_RANK(<expression>[,<expression>]) WITHIN GROUP (ORDER BY <expression>[,<expression>] ASC/DESC) aggregate function
+ *
+ * @author anoopsjohn
+ * @since 1.2.1
+ */
+@BuiltInFunction(name = PercentRankAggregateFunction.NAME, args = { @Argument(),
+        @Argument(allowedTypes = { PDataType.BOOLEAN }, isConstant = true), @Argument(isConstant = true) })
+public class PercentRankAggregateFunction extends DistinctValueWithCountAggregateFunction {
+    public static final String NAME = "PERCENT_RANK";
+
+    public PercentRankAggregateFunction() {
+
+    }
+
+    public PercentRankAggregateFunction(List<Expression> childern) {
+        super(childern);
+    }
+
+    @Override
+    public Aggregator newServerAggregator(Configuration conf) {
+        return new DistinctValueWithCountServerAggregator(conf);
+    }
+
+    @Override
+    public DistinctValueWithCountClientAggregator newClientAggregator() {
+        return new PercentRankClientAggregator(children, getAggregatorExpression().getColumnModifier());
+    }
+
+    @Override
+    public String getName() {
+        return NAME;
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return PDataType.DECIMAL;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/PercentileContAggregateFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/PercentileContAggregateFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/PercentileContAggregateFunction.java
new file mode 100644
index 0000000..703d619
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/PercentileContAggregateFunction.java
@@ -0,0 +1,75 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.aggregator.Aggregator;
+import org.apache.phoenix.expression.aggregator.DistinctValueWithCountClientAggregator;
+import org.apache.phoenix.expression.aggregator.DistinctValueWithCountServerAggregator;
+import org.apache.phoenix.expression.aggregator.PercentileClientAggregator;
+import org.apache.phoenix.parse.FunctionParseNode.Argument;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.schema.PDataType;
+
+/**
+ * 
+ * Built-in function for PERCENTILE_CONT(<expression>) WITHIN GROUP (ORDER BY <expression> ASC/DESC) aggregate function
+ *
+ * @author anoopsjohn
+ * @since 1.2.1
+ */
+@BuiltInFunction(name = PercentileContAggregateFunction.NAME, args = { @Argument(allowedTypes = { PDataType.DECIMAL }),
+        @Argument(allowedTypes = { PDataType.BOOLEAN }, isConstant = true),
+        @Argument(allowedTypes = { PDataType.DECIMAL }, isConstant = true, minValue = "0", maxValue = "1") })
+public class PercentileContAggregateFunction extends DistinctValueWithCountAggregateFunction {
+    public static final String NAME = "PERCENTILE_CONT";
+
+    public PercentileContAggregateFunction() {
+        
+    }
+    
+    public PercentileContAggregateFunction(List<Expression> childern) {
+        super(childern);
+    }
+
+    @Override
+    public Aggregator newServerAggregator(Configuration conf) {
+        return new DistinctValueWithCountServerAggregator(conf);
+    }
+
+    @Override
+    public DistinctValueWithCountClientAggregator newClientAggregator() {
+        return new PercentileClientAggregator(children, getAggregatorExpression().getColumnModifier());
+    }
+
+    @Override
+    public String getName() {
+        return NAME;
+    }
+    
+    @Override
+    public PDataType getDataType() {
+        return PDataType.DECIMAL;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/PercentileDiscAggregateFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/PercentileDiscAggregateFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/PercentileDiscAggregateFunction.java
new file mode 100644
index 0000000..7909c41
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/PercentileDiscAggregateFunction.java
@@ -0,0 +1,72 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.aggregator.Aggregator;
+import org.apache.phoenix.expression.aggregator.DistinctValueWithCountClientAggregator;
+import org.apache.phoenix.expression.aggregator.DistinctValueWithCountServerAggregator;
+import org.apache.phoenix.expression.aggregator.PercentileDiscClientAggregator;
+import org.apache.phoenix.parse.FunctionParseNode.Argument;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.schema.PDataType;
+
+
+/**
+ * 
+ * Built-in function for PERCENTILE_DISC(<expression>) WITHIN GROUP (ORDER BY <expression> ASC/DESC) aggregate function
+ *
+ * @author ramkrishna
+ * @since 1.2.1
+ */
+@BuiltInFunction(name = PercentileDiscAggregateFunction.NAME, args = { @Argument(allowedTypes = { PDataType.DECIMAL }),
+        @Argument(allowedTypes = { PDataType.BOOLEAN }, isConstant = true),
+        @Argument(allowedTypes = { PDataType.DECIMAL }, isConstant = true, minValue = "0", maxValue = "1") })
+public class PercentileDiscAggregateFunction extends DistinctValueWithCountAggregateFunction {
+
+	public static final String NAME = "PERCENTILE_DISC";
+
+	public PercentileDiscAggregateFunction() {
+	}
+
+	public PercentileDiscAggregateFunction(List<Expression> childern) {
+		super(childern);
+	}
+	
+	@Override
+	public Aggregator newServerAggregator(Configuration conf) {
+		return new DistinctValueWithCountServerAggregator(conf);
+	}
+	
+	@Override
+	public DistinctValueWithCountClientAggregator newClientAggregator() {
+		return new PercentileDiscClientAggregator(children, getAggregatorExpression().getColumnModifier());
+	}
+
+	@Override
+	public String getName() {
+		return NAME;
+	}
+	
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/PrefixFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/PrefixFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/PrefixFunction.java
new file mode 100644
index 0000000..2612d74
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/PrefixFunction.java
@@ -0,0 +1,83 @@
+package org.apache.phoenix.expression.function;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.compile.KeyPart;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.query.KeyRange;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.util.ByteUtil;
+
+abstract public class PrefixFunction extends ScalarFunction {
+    public PrefixFunction() {
+    }
+
+    public PrefixFunction(List<Expression> children) {
+        super(children);
+    }
+
+    @Override
+    public int getKeyFormationTraversalIndex() {
+        return preservesOrder() == OrderPreserving.NO ? NO_TRAVERSAL : 0;
+    }
+    
+    protected boolean extractNode() {
+        return false;
+    }
+
+    private static byte[] evaluateExpression(Expression rhs) {
+        ImmutableBytesWritable ptr = new ImmutableBytesWritable();
+        rhs.evaluate(null, ptr);
+        byte[] key = ByteUtil.copyKeyBytesIfNecessary(ptr);
+        return key;
+    }
+    
+    @Override
+    public KeyPart newKeyPart(final KeyPart childPart) {
+        return new KeyPart() {
+            private final List<Expression> extractNodes = extractNode() ? Collections.<Expression>singletonList(PrefixFunction.this) : Collections.<Expression>emptyList();
+
+            @Override
+            public PColumn getColumn() {
+                return childPart.getColumn();
+            }
+
+            @Override
+            public List<Expression> getExtractNodes() {
+                return extractNodes;
+            }
+
+            @Override
+            public KeyRange getKeyRange(CompareOp op, Expression rhs) {
+                byte[] key;
+                KeyRange range;
+                PDataType type = getColumn().getDataType();
+                switch (op) {
+                case EQUAL:
+                    key = evaluateExpression(rhs);
+                    range = type.getKeyRange(key, true, ByteUtil.nextKey(key), false);
+                    break;
+                case GREATER:
+                    key = evaluateExpression(rhs);
+                    range = type.getKeyRange(ByteUtil.nextKey(key), true, KeyRange.UNBOUND, false);
+                    break;
+                case LESS_OR_EQUAL:
+                    key = evaluateExpression(rhs);
+                    range = type.getKeyRange(KeyRange.UNBOUND, false, ByteUtil.nextKey(key), false);
+                    break;
+                default:
+                    return childPart.getKeyRange(op, rhs);
+                }
+                Integer length = getColumn().getByteSize();
+                return length == null ? range : range.fill(length);
+            }
+        };
+    }
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RTrimFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RTrimFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RTrimFunction.java
new file mode 100644
index 0000000..d7702a5
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RTrimFunction.java
@@ -0,0 +1,161 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.compile.KeyPart;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.parse.FunctionParseNode.Argument;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.query.KeyRange;
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.StringUtil;
+
+
+/**
+ * 
+ * Implementation of the RTrim(<string>) build-in function. It removes from the right end of
+ * <string> space character and other function bytes in single byte utf8 characters set 
+ * 
+ * @author zhuang
+ * @since 0.1
+ */
+@BuiltInFunction(name=RTrimFunction.NAME, args={
+    @Argument(allowedTypes={PDataType.VARCHAR})})
+public class RTrimFunction extends ScalarFunction {
+    public static final String NAME = "RTRIM";
+
+    private Integer byteSize;
+
+    public RTrimFunction() { }
+
+    public RTrimFunction(List<Expression> children) throws SQLException {
+        super(children);
+        if (getStringExpression().getDataType().isFixedWidth()) {
+            byteSize = getStringExpression().getByteSize();
+        }
+    }
+
+    private Expression getStringExpression() {
+        return children.get(0);
+    }
+
+    @Override
+    public ColumnModifier getColumnModifier() {
+        return children.get(0).getColumnModifier();
+    }    
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        // Starting from the end of the byte, look for all single bytes at the end of the string
+        // that is below SPACE_UTF8 (space and control characters) or above (control chars).
+        if (!getStringExpression().evaluate(tuple, ptr)) {
+            return false;
+        }
+        if (ptr.getLength() == 0) {
+            ptr.set(ByteUtil.EMPTY_BYTE_ARRAY);
+            return true;
+        }
+        byte[] string = ptr.get();
+        int offset = ptr.getOffset();
+        int length = ptr.getLength();
+        
+        ColumnModifier columnModifier = getStringExpression().getColumnModifier();
+        int i = StringUtil.getFirstNonBlankCharIdxFromEnd(string, offset, length, columnModifier);
+        if (i == offset - 1) {
+            ptr.set(ByteUtil.EMPTY_BYTE_ARRAY);
+            return true;
+            }
+        ptr.set(string, offset, i - offset + 1);
+        return true;
+    }
+
+    @Override
+    public OrderPreserving preservesOrder() {
+        return OrderPreserving.YES_IF_LAST;
+    }
+
+    @Override
+    public int getKeyFormationTraversalIndex() {
+        return 0;
+    }
+
+    @Override
+    public KeyPart newKeyPart(final KeyPart childPart) {
+        return new KeyPart() {
+            @Override
+            public KeyRange getKeyRange(CompareOp op, Expression rhs) {
+                ImmutableBytesWritable ptr = new ImmutableBytesWritable();
+                rhs.evaluate(null, ptr);
+                byte[] key = ByteUtil.copyKeyBytesIfNecessary(ptr);
+                PDataType type = getColumn().getDataType();
+                KeyRange range;
+                switch (op) {
+                case EQUAL:
+                    range = type.getKeyRange(key, true, ByteUtil.nextKey(ByteUtil.concat(key, new byte[] {StringUtil.SPACE_UTF8})), false);
+                    break;
+                case LESS_OR_EQUAL:
+                    range = type.getKeyRange(KeyRange.UNBOUND, false, ByteUtil.nextKey(ByteUtil.concat(key, new byte[] {StringUtil.SPACE_UTF8})), false);
+                    break;
+                default:
+                    range = childPart.getKeyRange(op, rhs);
+                    break;
+                }
+                Integer length = getColumn().getByteSize();
+                return length == null ? range : range.fill(length);
+            }
+
+            @Override
+            public List<Expression> getExtractNodes() {
+                return Collections.<Expression>emptyList();
+            }
+
+            @Override
+            public PColumn getColumn() {
+                return childPart.getColumn();
+            }
+        };
+    }
+
+    @Override
+    public Integer getByteSize() {
+        return byteSize;
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return PDataType.VARCHAR;
+    }
+
+    @Override
+    public String getName() {
+        return NAME;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RegexpReplaceFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RegexpReplaceFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RegexpReplaceFunction.java
new file mode 100644
index 0000000..1f0e28e
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RegexpReplaceFunction.java
@@ -0,0 +1,130 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.function;
+
+import java.io.DataInput;
+import java.io.IOException;
+import java.util.List;
+import java.util.regex.Pattern;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.LiteralExpression;
+import org.apache.phoenix.parse.FunctionParseNode.Argument;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+
+/**
+ * 
+ * Function similar to the regexp_replace function in Postgres, which is used to pattern
+ * match a segment of the string. Usage:
+ * REGEXP_REPLACE(<source_char>,<pattern>,<replace_string>)
+ * source_char is the string in which we want to perform string replacement. pattern is a
+ * Java compatible regular expression string, and we replace all the matching part with 
+ * replace_string. The first 2 arguments are required and are {@link org.apache.phoenix.schema.PDataType#VARCHAR},
+ * the replace_string is default to empty string.
+ * 
+ * The function returns a {@link org.apache.phoenix.schema.PDataType#VARCHAR}
+ * 
+ * @author zhuang
+ * @since 0.1
+ */
+@BuiltInFunction(name=RegexpReplaceFunction.NAME, args= {
+    @Argument(allowedTypes={PDataType.VARCHAR}),
+    @Argument(allowedTypes={PDataType.VARCHAR}),
+    @Argument(allowedTypes={PDataType.VARCHAR},defaultValue="null")} )
+public class RegexpReplaceFunction extends ScalarFunction {
+    public static final String NAME = "REGEXP_REPLACE";
+
+    private boolean hasReplaceStr;
+    private Pattern pattern;
+    
+    public RegexpReplaceFunction() { }
+
+    // Expect 1 arguments, the pattern. 
+    public RegexpReplaceFunction(List<Expression> children) {
+        super(children);
+        init();
+    }
+
+    private void init() {
+        hasReplaceStr = ((LiteralExpression)getReplaceStrExpression()).getValue() != null;
+        Object patternString = ((LiteralExpression)children.get(1)).getValue();
+        if (patternString != null) {
+            pattern = Pattern.compile((String)patternString);
+        }
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        // Can't parse if there is no replacement pattern.
+        if (pattern == null) {
+            return false;
+        }
+        Expression sourceStrExpression = getSourceStrExpression();
+        if (!sourceStrExpression.evaluate(tuple, ptr)) {
+            return false;
+        }
+        String sourceStr = (String)PDataType.VARCHAR.toObject(ptr, sourceStrExpression.getColumnModifier());
+        if (sourceStr == null) {
+            return false;
+        }
+        String replaceStr;
+        if (hasReplaceStr) {
+            Expression replaceStrExpression = this.getReplaceStrExpression();
+            if (!replaceStrExpression.evaluate(tuple, ptr)) {
+                return false;
+            }
+            replaceStr = (String)PDataType.VARCHAR.toObject(ptr, replaceStrExpression.getColumnModifier());
+        } else {
+            replaceStr = "";
+        }
+        String replacedStr = pattern.matcher(sourceStr).replaceAll(replaceStr);
+        ptr.set(PDataType.VARCHAR.toBytes(replacedStr));
+        return true;
+    }
+
+    private Expression getSourceStrExpression() {
+        return children.get(0);
+    }
+
+    private Expression getReplaceStrExpression() {
+        return children.get(2);
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return PDataType.VARCHAR;
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        super.readFields(input);
+        init();
+    }
+
+    @Override
+    public String getName() {
+        return NAME;
+    }
+}


[42/51] [partial] Initial commit of master branch from github

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..cd2b0e2
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionCompiler.java
@@ -0,0 +1,1391 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.compile;
+
+import java.math.BigDecimal;
+import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
+import java.sql.Types;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.compile.GroupByCompiler.GroupBy;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+import org.apache.phoenix.expression.AndExpression;
+import org.apache.phoenix.expression.ArrayConstructorExpression;
+import org.apache.phoenix.expression.CaseExpression;
+import org.apache.phoenix.expression.CoerceExpression;
+import org.apache.phoenix.expression.ComparisonExpression;
+import org.apache.phoenix.expression.DateAddExpression;
+import org.apache.phoenix.expression.DateSubtractExpression;
+import org.apache.phoenix.expression.DecimalAddExpression;
+import org.apache.phoenix.expression.DecimalDivideExpression;
+import org.apache.phoenix.expression.DecimalMultiplyExpression;
+import org.apache.phoenix.expression.DecimalSubtractExpression;
+import org.apache.phoenix.expression.DoubleAddExpression;
+import org.apache.phoenix.expression.DoubleDivideExpression;
+import org.apache.phoenix.expression.DoubleMultiplyExpression;
+import org.apache.phoenix.expression.DoubleSubtractExpression;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.InListExpression;
+import org.apache.phoenix.expression.IsNullExpression;
+import org.apache.phoenix.expression.LikeExpression;
+import org.apache.phoenix.expression.LiteralExpression;
+import org.apache.phoenix.expression.LongAddExpression;
+import org.apache.phoenix.expression.LongDivideExpression;
+import org.apache.phoenix.expression.LongMultiplyExpression;
+import org.apache.phoenix.expression.LongSubtractExpression;
+import org.apache.phoenix.expression.NotExpression;
+import org.apache.phoenix.expression.OrExpression;
+import org.apache.phoenix.expression.RowKeyColumnExpression;
+import org.apache.phoenix.expression.RowValueConstructorExpression;
+import org.apache.phoenix.expression.StringConcatExpression;
+import org.apache.phoenix.expression.TimestampAddExpression;
+import org.apache.phoenix.expression.TimestampSubtractExpression;
+import org.apache.phoenix.parse.AddParseNode;
+import org.apache.phoenix.parse.AndParseNode;
+import org.apache.phoenix.parse.ArithmeticParseNode;
+import org.apache.phoenix.parse.ArrayConstructorNode;
+import org.apache.phoenix.parse.BindParseNode;
+import org.apache.phoenix.parse.CaseParseNode;
+import org.apache.phoenix.parse.CastParseNode;
+import org.apache.phoenix.parse.ColumnParseNode;
+import org.apache.phoenix.parse.ComparisonParseNode;
+import org.apache.phoenix.parse.DivideParseNode;
+import org.apache.phoenix.parse.FunctionParseNode;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunctionInfo;
+import org.apache.phoenix.parse.InListParseNode;
+import org.apache.phoenix.parse.IsNullParseNode;
+import org.apache.phoenix.parse.LikeParseNode;
+import org.apache.phoenix.parse.LiteralParseNode;
+import org.apache.phoenix.parse.MultiplyParseNode;
+import org.apache.phoenix.parse.NotParseNode;
+import org.apache.phoenix.parse.OrParseNode;
+import org.apache.phoenix.parse.ParseNode;
+import org.apache.phoenix.parse.RowValueConstructorParseNode;
+import org.apache.phoenix.parse.SequenceValueParseNode;
+import org.apache.phoenix.parse.StringConcatParseNode;
+import org.apache.phoenix.parse.SubtractParseNode;
+import org.apache.phoenix.parse.UnsupportedAllParseNodeVisitor;
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.ColumnNotFoundException;
+import org.apache.phoenix.schema.ColumnRef;
+import org.apache.phoenix.schema.DelegateDatum;
+import org.apache.phoenix.schema.PArrayDataType;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.PDatum;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.schema.RowKeyValueAccessor;
+import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.schema.TypeMismatchException;
+import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.SchemaUtil;
+
+
+public class ExpressionCompiler extends UnsupportedAllParseNodeVisitor<Expression> {
+    private boolean isAggregate;
+    protected ParseNode aggregateFunction;
+    protected final StatementContext context;
+    protected final GroupBy groupBy;
+    private int nodeCount;
+    
+    ExpressionCompiler(StatementContext context) {
+        this(context,GroupBy.EMPTY_GROUP_BY);
+    }
+
+    ExpressionCompiler(StatementContext context, GroupBy groupBy) {
+        this.context = context;
+        this.groupBy = groupBy;
+    }
+
+    public boolean isAggregate() {
+        return isAggregate;
+    }
+
+    public boolean isTopLevel() {
+        return nodeCount == 0;
+    }
+    
+    public void reset() {
+        this.isAggregate = false;
+        this.nodeCount = 0;
+    }
+
+    @Override
+    public boolean visitEnter(ComparisonParseNode node) {
+        return true;
+    }
+    
+    private void addBindParamMetaData(ParseNode parentNode, ParseNode lhsNode, ParseNode rhsNode, Expression lhsExpr, Expression rhsExpr) throws SQLException {
+        if (lhsNode instanceof BindParseNode) {
+            context.getBindManager().addParamMetaData((BindParseNode)lhsNode, rhsExpr);
+        }
+        if (rhsNode instanceof BindParseNode) {
+            context.getBindManager().addParamMetaData((BindParseNode)rhsNode, lhsExpr);
+        }
+    }
+    
+    private static void checkComparability(ParseNode node, PDataType lhsDataType, PDataType rhsDataType) throws TypeMismatchException {
+        if(lhsDataType != null && rhsDataType != null && !lhsDataType.isComparableTo(rhsDataType)) {
+            throw TypeMismatchException.newException(lhsDataType, rhsDataType, node.toString());
+        }
+    }
+    
+    // TODO: this no longer needs to be recursive, as we flatten out rvc when we normalize the statement
+    private void checkComparability(ParseNode parentNode, ParseNode lhsNode, ParseNode rhsNode, Expression lhsExpr, Expression rhsExpr) throws SQLException {
+        if (lhsNode instanceof RowValueConstructorParseNode && rhsNode instanceof RowValueConstructorParseNode) {
+            int i = 0;
+            for (; i < Math.min(lhsExpr.getChildren().size(),rhsExpr.getChildren().size()); i++) {
+                checkComparability(parentNode, lhsNode.getChildren().get(i), rhsNode.getChildren().get(i), lhsExpr.getChildren().get(i), rhsExpr.getChildren().get(i));
+            }
+            for (; i < lhsExpr.getChildren().size(); i++) {
+                checkComparability(parentNode, lhsNode.getChildren().get(i), null, lhsExpr.getChildren().get(i), null);
+            }
+            for (; i < rhsExpr.getChildren().size(); i++) {
+                checkComparability(parentNode, null, rhsNode.getChildren().get(i), null, rhsExpr.getChildren().get(i));
+            }
+        } else if (lhsExpr instanceof RowValueConstructorExpression) {
+            checkComparability(parentNode, lhsNode.getChildren().get(0), rhsNode, lhsExpr.getChildren().get(0), rhsExpr);
+            for (int i = 1; i < lhsExpr.getChildren().size(); i++) {
+                checkComparability(parentNode, lhsNode.getChildren().get(i), null, lhsExpr.getChildren().get(i), null);
+            }
+        } else if (rhsExpr instanceof RowValueConstructorExpression) {
+            checkComparability(parentNode, lhsNode, rhsNode.getChildren().get(0), lhsExpr, rhsExpr.getChildren().get(0));
+            for (int i = 1; i < rhsExpr.getChildren().size(); i++) {
+                checkComparability(parentNode, null, rhsNode.getChildren().get(i), null, rhsExpr.getChildren().get(i));
+            }
+        } else if (lhsNode == null && rhsNode == null) { // null == null will end up making the query degenerate
+            
+        } else if (lhsNode == null) { // AND rhs IS NULL
+            addBindParamMetaData(parentNode, lhsNode, rhsNode, lhsExpr, rhsExpr);
+        } else if (rhsNode == null) { // AND lhs IS NULL
+            addBindParamMetaData(parentNode, lhsNode, rhsNode, lhsExpr, rhsExpr);
+        } else { // AND lhs = rhs
+            checkComparability(parentNode, lhsExpr.getDataType(), rhsExpr.getDataType());
+            addBindParamMetaData(parentNode, lhsNode, rhsNode, lhsExpr, rhsExpr);
+        }
+    }
+
+    @Override
+    public Expression visitLeave(ComparisonParseNode node, List<Expression> children) throws SQLException {
+        ParseNode lhsNode = node.getChildren().get(0);
+        ParseNode rhsNode = node.getChildren().get(1);
+        Expression lhsExpr = children.get(0);
+        Expression rhsExpr = children.get(1);
+        boolean isDeterministic = lhsExpr.isDeterministic() || rhsExpr.isDeterministic();
+
+        PDataType lhsExprDataType = lhsExpr.getDataType();
+        PDataType rhsExprDataType = rhsExpr.getDataType();
+        checkComparability(node, lhsNode, rhsNode, lhsExpr, rhsExpr);
+        // We don't yet support comparison between entire arrays
+        if ( ( (lhsExprDataType != null && lhsExprDataType.isArrayType()) || 
+               (rhsExprDataType != null && rhsExprDataType.isArrayType()) ) &&
+             ( node.getFilterOp() != CompareOp.EQUAL && node.getFilterOp() != CompareOp.NOT_EQUAL ) ) {
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode.NON_EQUALITY_ARRAY_COMPARISON)
+            .setMessage(ComparisonExpression.toString(node.getFilterOp(), children)).build().buildException();
+        }
+        
+        if (lhsExpr instanceof RowValueConstructorExpression || rhsExpr instanceof RowValueConstructorExpression) {
+            rhsExpr = RowValueConstructorExpression.coerce(lhsExpr, rhsExpr, node.getFilterOp());
+            // Always wrap both sides in row value constructor, so we don't have to consider comparing
+            // a non rvc with a rvc.
+            if ( ! ( lhsExpr instanceof RowValueConstructorExpression ) ) {
+                lhsExpr = new RowValueConstructorExpression(Collections.singletonList(lhsExpr), lhsExpr.isStateless());
+            }
+            children = Arrays.asList(lhsExpr, rhsExpr);
+        }
+        
+        Object lhsValue = null;
+        // Can't use lhsNode.isConstant(), because we have cases in which we don't know
+        // in advance if a function evaluates to null (namely when bind variables are used)
+        // TODO: use lhsExpr.isStateless instead
+        if (lhsExpr instanceof LiteralExpression) {
+            lhsValue = ((LiteralExpression)lhsExpr).getValue();
+            if (lhsValue == null) {
+                return LiteralExpression.newConstant(false, PDataType.BOOLEAN, lhsExpr.isDeterministic());
+            }
+        }
+        Object rhsValue = null;
+        // TODO: use lhsExpr.isStateless instead
+        if (rhsExpr instanceof LiteralExpression) {
+            rhsValue = ((LiteralExpression)rhsExpr).getValue();
+            if (rhsValue == null) {
+                return LiteralExpression.newConstant(false, PDataType.BOOLEAN, rhsExpr.isDeterministic());
+            }
+        }
+        if (lhsValue != null && rhsValue != null) {
+            return LiteralExpression.newConstant(ByteUtil.compare(node.getFilterOp(),lhsExprDataType.compareTo(lhsValue, rhsValue, rhsExprDataType)), isDeterministic);
+        }
+        // Coerce constant to match type of lhs so that we don't need to
+        // convert at filter time. Since we normalize the select statement
+        // to put constants on the LHS, we don't need to check the RHS.
+        if (rhsValue != null) {
+            // Comparing an unsigned int/long against a negative int/long would be an example. We just need to take
+            // into account the comparison operator.
+            if (rhsExprDataType != lhsExprDataType 
+                    || rhsExpr.getColumnModifier() != lhsExpr.getColumnModifier()
+                    || (rhsExpr.getMaxLength() != null && lhsExpr.getMaxLength() != null && rhsExpr.getMaxLength() < lhsExpr.getMaxLength())) {
+                // TODO: if lengths are unequal and fixed width?
+                if (rhsExprDataType.isCoercibleTo(lhsExprDataType, rhsValue)) { // will convert 2.0 -> 2
+                    children = Arrays.asList(children.get(0), LiteralExpression.newConstant(rhsValue, lhsExprDataType, 
+                            lhsExpr.getMaxLength(), null, lhsExpr.getColumnModifier(), isDeterministic));
+                } else if (node.getFilterOp() == CompareOp.EQUAL) {
+                    return LiteralExpression.newConstant(false, PDataType.BOOLEAN, true);
+                } else if (node.getFilterOp() == CompareOp.NOT_EQUAL) {
+                    return LiteralExpression.newConstant(true, PDataType.BOOLEAN, true);
+                } else { // TODO: generalize this with PDataType.getMinValue(), PDataTypeType.getMaxValue() methods
+                    switch(rhsExprDataType) {
+                    case DECIMAL:
+                        /*
+                         * We're comparing an int/long to a constant decimal with a fraction part.
+                         * We need the types to match in case this is used to form a key. To form the start/stop key,
+                         * we need to adjust the decimal by truncating it or taking its ceiling, depending on the comparison
+                         * operator, to get a whole number.
+                         */
+                        int increment = 0;
+                        switch (node.getFilterOp()) {
+                        case GREATER_OR_EQUAL: 
+                        case LESS: // get next whole number
+                            increment = 1;
+                        default: // Else, we truncate the value
+                            BigDecimal bd = (BigDecimal)rhsValue;
+                            rhsValue = bd.longValue() + increment;
+                            children = Arrays.asList(children.get(0), LiteralExpression.newConstant(rhsValue, lhsExprDataType, lhsExpr.getColumnModifier(), rhsExpr.isDeterministic()));
+                            break;
+                        }
+                        break;
+                    case LONG:
+                        /*
+                         * We are comparing an int, unsigned_int to a long, or an unsigned_long to a negative long.
+                         * int has range of -2147483648 to 2147483647, and unsigned_int has a value range of 0 to 4294967295.
+                         * 
+                         * If lhs is int or unsigned_int, since we already determined that we cannot coerce the rhs 
+                         * to become the lhs, we know the value on the rhs is greater than lhs if it's positive, or smaller than
+                         * lhs if it's negative.
+                         * 
+                         * If lhs is an unsigned_long, then we know the rhs is definitely a negative long. rhs in this case
+                         * will always be bigger than rhs.
+                         */
+                        if (lhsExprDataType == PDataType.INTEGER || 
+                        lhsExprDataType == PDataType.UNSIGNED_INT) {
+                            switch (node.getFilterOp()) {
+                            case LESS:
+                            case LESS_OR_EQUAL:
+                                if ((Long)rhsValue > 0) {
+                                    return LiteralExpression.newConstant(true, PDataType.BOOLEAN, isDeterministic);
+                                } else {
+                                    return LiteralExpression.newConstant(false, PDataType.BOOLEAN, isDeterministic);
+                                }
+                            case GREATER:
+                            case GREATER_OR_EQUAL:
+                                if ((Long)rhsValue > 0) {
+                                    return LiteralExpression.newConstant(false, PDataType.BOOLEAN, isDeterministic);
+                                } else {
+                                    return LiteralExpression.newConstant(true, PDataType.BOOLEAN, isDeterministic);
+                                }
+                            default:
+                                break;
+                            }
+                        } else if (lhsExprDataType == PDataType.UNSIGNED_LONG) {
+                            switch (node.getFilterOp()) {
+                            case LESS:
+                            case LESS_OR_EQUAL:
+                                return LiteralExpression.newConstant(false, PDataType.BOOLEAN, isDeterministic);
+                            case GREATER:
+                            case GREATER_OR_EQUAL:
+                                return LiteralExpression.newConstant(true, PDataType.BOOLEAN, isDeterministic);
+                            default:
+                                break;
+                            }
+                        }
+                        children = Arrays.asList(children.get(0), LiteralExpression.newConstant(rhsValue, rhsExprDataType, lhsExpr.getColumnModifier(), isDeterministic));
+                        break;
+                    }
+                }
+            }
+
+            // Determine if we know the expression must be TRUE or FALSE based on the max size of
+            // a fixed length expression.
+            if (children.get(1).getMaxLength() != null && lhsExpr.getMaxLength() != null && lhsExpr.getMaxLength() < children.get(1).getMaxLength()) {
+                switch (node.getFilterOp()) {
+                case EQUAL:
+                    return LiteralExpression.newConstant(false, PDataType.BOOLEAN, isDeterministic);
+                case NOT_EQUAL:
+                    return LiteralExpression.newConstant(true, PDataType.BOOLEAN, isDeterministic);
+                default:
+                    break;
+                }
+            }
+        }
+        return new ComparisonExpression(node.getFilterOp(), children);
+    }
+
+    @Override
+    public boolean visitEnter(AndParseNode node) throws SQLException {
+        return true;
+    }
+
+    @Override
+    public Expression visitLeave(AndParseNode node, List<Expression> children) throws SQLException {
+        boolean isDeterministic = true;
+        Iterator<Expression> iterator = children.iterator();
+        while (iterator.hasNext()) {
+            Expression child = iterator.next();
+            if (child.getDataType() != PDataType.BOOLEAN) {
+                throw TypeMismatchException.newException(PDataType.BOOLEAN, child.getDataType(), child.toString());
+            }
+            if (LiteralExpression.isFalse(child)) {
+                return child;
+            }
+            if (LiteralExpression.isTrue(child)) {
+                iterator.remove();
+            }
+            isDeterministic &= child.isDeterministic();
+        }
+        if (children.size() == 0) {
+            return LiteralExpression.newConstant(true, isDeterministic);
+        }
+        if (children.size() == 1) {
+            return children.get(0);
+        }
+        return new AndExpression(children);
+    }
+
+    @Override
+    public boolean visitEnter(OrParseNode node) throws SQLException {
+        return true;
+    }
+
+    private Expression orExpression(List<Expression> children) throws SQLException {
+        Iterator<Expression> iterator = children.iterator();
+        boolean isDeterministic = true;
+        while (iterator.hasNext()) {
+            Expression child = iterator.next();
+            if (child.getDataType() != PDataType.BOOLEAN) {
+                throw TypeMismatchException.newException(PDataType.BOOLEAN, child.getDataType(), child.toString());
+            }
+            if (LiteralExpression.isFalse(child)) {
+                iterator.remove();
+            }
+            if (LiteralExpression.isTrue(child)) {
+                return child;
+            }
+            isDeterministic &= child.isDeterministic();
+        }
+        if (children.size() == 0) {
+            return LiteralExpression.newConstant(true, isDeterministic);
+        }
+        if (children.size() == 1) {
+            return children.get(0);
+        }
+        return new OrExpression(children);
+    }
+
+    @Override
+    public Expression visitLeave(OrParseNode node, List<Expression> children) throws SQLException {
+        return orExpression(children);
+    }
+
+    @Override
+    public boolean visitEnter(FunctionParseNode node) throws SQLException {
+        // TODO: Oracle supports nested aggregate function while other DBs don't. Should we?
+        if (node.isAggregate()) {
+            if (aggregateFunction != null) {
+                throw new SQLFeatureNotSupportedException("Nested aggregate functions are not supported");
+            }
+            this.aggregateFunction = node;
+            this.isAggregate = true;
+
+        }
+        return true;
+    }
+
+    private Expression wrapGroupByExpression(Expression expression) {
+        // If we're in an aggregate function, don't wrap a group by expression,
+        // since in that case we're aggregating over the regular/ungrouped
+        // column.
+        if (aggregateFunction == null) {
+            int index = groupBy.getExpressions().indexOf(expression);
+            if (index >= 0) {
+                isAggregate = true;
+                RowKeyValueAccessor accessor = new RowKeyValueAccessor(groupBy.getKeyExpressions(), index);
+                expression = new RowKeyColumnExpression(expression, accessor, groupBy.getKeyExpressions().get(index).getDataType());
+            }
+        }
+        return expression;
+    }
+
+    /**
+     * Add expression to the expression manager, returning the same one if
+     * already used.
+     */
+    protected Expression addExpression(Expression expression) {
+        return context.getExpressionManager().addIfAbsent(expression);
+    }
+
+    @Override
+    /**
+     * @param node a function expression node
+     * @param children the child expression arguments to the function expression node.
+     */
+    public Expression visitLeave(FunctionParseNode node, List<Expression> children) throws SQLException {
+        children = node.validate(children, context);
+        Expression expression = node.create(children, context);
+        ImmutableBytesWritable ptr = context.getTempPtr();
+        if (node.isStateless()) {
+            Object value = null;
+            PDataType type = expression.getDataType();
+            if (expression.evaluate(null, ptr)) {
+                value = type.toObject(ptr);
+            }
+            return LiteralExpression.newConstant(value, type, expression.isDeterministic());
+        }
+        boolean isDeterministic = true;
+        BuiltInFunctionInfo info = node.getInfo();
+        for (int i = 0; i < info.getRequiredArgCount(); i++) { 
+            // Optimization to catch cases where a required argument is null resulting in the function
+            // returning null. We have to wait until after we create the function expression so that
+            // we can get the proper type to use.
+            if (node.evalToNullIfParamIsNull(context, i)) {
+                Expression child = children.get(i);
+                isDeterministic &= child.isDeterministic();
+                if (child.isStateless() && (!child.evaluate(null, ptr) || ptr.getLength() == 0)) {
+                    return LiteralExpression.newConstant(null, expression.getDataType(), isDeterministic);
+                }
+            }
+        }
+        expression = addExpression(expression);
+        expression = wrapGroupByExpression(expression);
+        if (aggregateFunction == node) {
+            aggregateFunction = null; // Turn back off on the way out
+        }
+        return expression;
+    }
+
+    /**
+     * Called by visitor to resolve a column expression node into a column reference.
+     * Derived classes may use this as a hook to trap all column resolves.
+     * @param node a column expression node
+     * @return a resolved ColumnRef
+     * @throws SQLException if the column expression node does not refer to a known/unambiguous column
+     */
+    protected ColumnRef resolveColumn(ColumnParseNode node) throws SQLException {
+        ColumnRef ref = context.getResolver().resolveColumn(node.getSchemaName(), node.getTableName(), node.getName());
+        PTable table = ref.getTable();
+        int pkPosition = ref.getPKSlotPosition();
+        // Disallow explicit reference to SALT or TENANT_ID columns
+        if (pkPosition >= 0) {
+            boolean isSalted = table.getBucketNum() != null;
+            boolean isMultiTenant = context.getConnection().getTenantId() != null && table.isMultiTenant();
+            int minPosition = (isSalted ? 1 : 0) + (isMultiTenant ? 1 : 0);
+            if (pkPosition < minPosition) {
+                throw new ColumnNotFoundException(table.getSchemaName().getString(), table.getTableName().getString(), null, ref.getColumn().getName().getString());
+            }
+        }
+        return ref;
+    }
+
+    @Override
+    public Expression visit(ColumnParseNode node) throws SQLException {
+        ColumnRef ref = resolveColumn(node);
+        TableRef tableRef = ref.getTableRef();
+        if (tableRef.equals(context.getCurrentTable()) 
+                && !SchemaUtil.isPKColumn(ref.getColumn())) { // project only kv columns
+            context.getScan().addColumn(ref.getColumn().getFamilyName().getBytes(), ref.getColumn().getName().getBytes());
+        }
+        Expression expression = ref.newColumnExpression();
+        Expression wrappedExpression = wrapGroupByExpression(expression);
+        // If we're in an aggregate expression
+        // and we're not in the context of an aggregate function
+        // and we didn't just wrap our column reference
+        // then we're mixing aggregate and non aggregate expressions in the same expression.
+        // This catches cases like this: SELECT sum(a_integer) + a_integer FROM atable GROUP BY a_string
+        if (isAggregate && aggregateFunction == null && wrappedExpression == expression) {
+            throwNonAggExpressionInAggException(expression.toString());
+        }
+        return wrappedExpression;
+    }
+
+    @Override
+    public Expression visit(BindParseNode node) throws SQLException {
+        Object value = context.getBindManager().getBindValue(node);
+        return LiteralExpression.newConstant(value, true);
+    }
+
+    @Override
+    public Expression visit(LiteralParseNode node) throws SQLException {
+        return LiteralExpression.newConstant(node.getValue(), node.getType(), true);
+    }
+
+    @Override
+    public List<Expression> newElementList(int size) {
+        nodeCount += size;
+        return new ArrayList<Expression>(size);
+    }
+
+    @Override
+    public void addElement(List<Expression> l, Expression element) {
+        nodeCount--;
+        l.add(element);
+    }
+
+    @Override
+    public boolean visitEnter(CaseParseNode node) throws SQLException {
+        return true;
+    }
+
+    private static boolean isDeterministic(List<Expression> l) {
+        for (Expression e : l) {
+            if (!e.isDeterministic()) {
+                return false;
+            }
+        }
+        return true;
+    }
+    
+    @Override
+    public Expression visitLeave(CaseParseNode node, List<Expression> l) throws SQLException {
+        final CaseExpression caseExpression = new CaseExpression(l);
+        for (int i = 0; i < node.getChildren().size(); i+=2) {
+            ParseNode childNode = node.getChildren().get(i);
+            if (childNode instanceof BindParseNode) {
+                context.getBindManager().addParamMetaData((BindParseNode)childNode, new DelegateDatum(caseExpression));
+            }
+        }
+        if (node.isStateless()) {
+            ImmutableBytesWritable ptr = context.getTempPtr();
+            int index = caseExpression.evaluateIndexOf(null, ptr);
+            if (index < 0) {
+                return LiteralExpression.newConstant(null, isDeterministic(l));
+            }
+            return caseExpression.getChildren().get(index);
+        }
+        return wrapGroupByExpression(caseExpression);
+    }
+
+    @Override
+    public boolean visitEnter(LikeParseNode node) throws SQLException {
+        return true;
+    }
+
+    @Override
+    public Expression visitLeave(LikeParseNode node, List<Expression> children) throws SQLException {
+        ParseNode lhsNode = node.getChildren().get(0);
+        ParseNode rhsNode = node.getChildren().get(1);
+        Expression lhs = children.get(0);
+        Expression rhs = children.get(1);
+        if ( rhs.getDataType() != null && lhs.getDataType() != null && 
+                !lhs.getDataType().isCoercibleTo(rhs.getDataType())  && 
+                !rhs.getDataType().isCoercibleTo(lhs.getDataType())) {
+            throw TypeMismatchException.newException(lhs.getDataType(), rhs.getDataType(), node.toString());
+        }
+        if (lhsNode instanceof BindParseNode) {
+            context.getBindManager().addParamMetaData((BindParseNode)lhsNode, rhs);
+        }
+        if (rhsNode instanceof BindParseNode) {
+            context.getBindManager().addParamMetaData((BindParseNode)rhsNode, lhs);
+        }
+        if (rhs instanceof LiteralExpression) {
+            String pattern = (String)((LiteralExpression)rhs).getValue();
+            if (pattern == null || pattern.length() == 0) {
+                return LiteralExpression.newConstant(null, rhs.isDeterministic());
+            }
+            // TODO: for pattern of '%' optimize to strlength(lhs) > 0
+            // We can't use lhs IS NOT NULL b/c if lhs is NULL we need
+            // to return NULL.
+            int index = LikeExpression.indexOfWildcard(pattern);
+            // Can't possibly be as long as the constant, then FALSE
+            Integer lhsByteSize = lhs.getByteSize();
+            if (lhsByteSize != null && lhsByteSize < index) {
+                return LiteralExpression.newConstant(false, rhs.isDeterministic());
+            }
+            if (index == -1) {
+                String rhsLiteral = LikeExpression.unescapeLike(pattern);
+                if (lhsByteSize != null && lhsByteSize != rhsLiteral.length()) {
+                    return LiteralExpression.newConstant(false, rhs.isDeterministic());
+                }
+                CompareOp op = node.isNegate() ? CompareOp.NOT_EQUAL : CompareOp.EQUAL;
+                if (pattern.equals(rhsLiteral)) {
+                    return new ComparisonExpression(op, children);
+                } else {
+                    rhs = LiteralExpression.newConstant(rhsLiteral, PDataType.CHAR, rhs.isDeterministic());
+                    return new ComparisonExpression(op, Arrays.asList(lhs,rhs));
+                }
+            }
+        }
+        Expression expression = new LikeExpression(children);
+        if (node.isStateless()) {
+            ImmutableBytesWritable ptr = context.getTempPtr();
+            if (!expression.evaluate(null, ptr)) {
+                return LiteralExpression.newConstant(null, expression.isDeterministic());
+            } else {
+                return LiteralExpression.newConstant(Boolean.TRUE.equals(PDataType.BOOLEAN.toObject(ptr)) ^ node.isNegate(), expression.isDeterministic());
+            }
+        }
+        if (node.isNegate()) {
+            expression = new NotExpression(expression);
+        }
+        return expression;
+    }
+
+
+    @Override
+    public boolean visitEnter(NotParseNode node) throws SQLException {
+        return true;
+    }
+
+    @Override
+    public Expression visitLeave(NotParseNode node, List<Expression> children) throws SQLException {
+        ParseNode childNode = node.getChildren().get(0);
+        Expression child = children.get(0);
+        if (!PDataType.BOOLEAN.isCoercibleTo(child.getDataType())) {
+            throw TypeMismatchException.newException(PDataType.BOOLEAN, child.getDataType(), node.toString());
+        }
+        if (childNode instanceof BindParseNode) { // TODO: valid/possibe?
+            context.getBindManager().addParamMetaData((BindParseNode)childNode, child);
+        }
+        ImmutableBytesWritable ptr = context.getTempPtr();
+        // Can't use child.isConstant(), because we have cases in which we don't know
+        // in advance if a function evaluates to null (namely when bind variables are used)
+        // TODO: use child.isStateless
+        if (child.isStateless()) {
+            if (!child.evaluate(null, ptr) || ptr.getLength() == 0) {
+                return LiteralExpression.newConstant(null, PDataType.BOOLEAN, child.isDeterministic());
+            }
+            return LiteralExpression.newConstant(!(Boolean)PDataType.BOOLEAN.toObject(ptr), PDataType.BOOLEAN, child.isDeterministic());
+        }
+        return new NotExpression(child);
+    }
+
+    @Override
+    public boolean visitEnter(CastParseNode node) throws SQLException {
+        return true;
+    }
+
+    @Override
+    public Expression visitLeave(CastParseNode node, List<Expression> children) throws SQLException {
+        ParseNode childNode = node.getChildren().get(0);
+        PDataType targetDataType = node.getDataType();
+        Expression childExpr = children.get(0);
+        PDataType fromDataType = childExpr.getDataType();
+        
+        if (childNode instanceof BindParseNode) {
+            context.getBindManager().addParamMetaData((BindParseNode)childNode, childExpr);
+        }
+        
+        Expression expr = childExpr;
+        if(fromDataType != null) {
+            /*
+             * IndexStatementRewriter creates a CAST parse node when rewriting the query to use
+             * indexed columns. Without this check present we wrongly and unnecessarily
+             * end up creating a RoundExpression. 
+             */
+            if (context.getResolver().getTables().get(0).getTable().getType() != PTableType.INDEX) {
+                expr =  CastParseNode.convertToRoundExpressionIfNeeded(fromDataType, targetDataType, children);
+            }
+        }
+        return CoerceExpression.create(expr, targetDataType); 
+    }
+    
+   @Override
+    public boolean visitEnter(InListParseNode node) throws SQLException {
+        return true;
+    }
+
+    @Override
+    public Expression visitLeave(InListParseNode node, List<Expression> l) throws SQLException {
+        List<Expression> inChildren = l;
+        Expression firstChild = inChildren.get(0);
+        ImmutableBytesWritable ptr = context.getTempPtr();
+        PDataType firstChildType = firstChild.getDataType();
+        ParseNode firstChildNode = node.getChildren().get(0);
+        
+        if (firstChildNode instanceof BindParseNode) {
+            PDatum datum = firstChild;
+            if (firstChildType == null) {
+                datum = inferBindDatum(inChildren);
+            }
+            context.getBindManager().addParamMetaData((BindParseNode)firstChildNode, datum);
+        }
+        for (int i = 1; i < l.size(); i++) {
+            ParseNode childNode = node.getChildren().get(i);
+            if (childNode instanceof BindParseNode) {
+                context.getBindManager().addParamMetaData((BindParseNode)childNode, firstChild);
+            }
+        }
+        if (firstChildNode.isStateless() && firstChild.evaluate(null, ptr) && ptr.getLength() == 0) {
+            return LiteralExpression.newConstant(null, PDataType.BOOLEAN, firstChild.isDeterministic());
+        }
+        
+        Expression e = InListExpression.create(inChildren, ptr);
+        if (node.isNegate()) {
+            e = new NotExpression(e);
+        }
+        if (node.isStateless()) {
+            if (!e.evaluate(null, ptr) || ptr.getLength() == 0) {
+                return LiteralExpression.newConstant(null, e.getDataType(), e.isDeterministic());
+            }
+            Object value = e.getDataType().toObject(ptr);
+            return LiteralExpression.newConstant(value, e.getDataType(), e.isDeterministic());
+        }
+        
+        return e;
+    }
+
+    private static final PDatum DECIMAL_DATUM = new PDatum() {
+        @Override
+        public boolean isNullable() {
+            return true;
+        }
+        @Override
+        public PDataType getDataType() {
+            return PDataType.DECIMAL;
+        }
+        @Override
+        public Integer getByteSize() {
+            return null;
+        }
+        @Override
+        public Integer getMaxLength() {
+            return null;
+        }
+        @Override
+        public Integer getScale() {
+            return PDataType.DEFAULT_SCALE;
+        }
+        @Override
+        public ColumnModifier getColumnModifier() {
+            return null;
+        }        
+    };
+
+    private static PDatum inferBindDatum(List<Expression> children) {
+        boolean isChildTypeUnknown = false;
+        PDatum datum = children.get(1);
+        for (int i = 2; i < children.size(); i++) {
+            Expression child = children.get(i);
+            PDataType childType = child.getDataType();
+            if (childType == null) {
+                isChildTypeUnknown = true;
+            } else if (datum.getDataType() == null) {
+                datum = child;
+                isChildTypeUnknown = true;
+            } else if (datum.getDataType() == childType || childType.isCoercibleTo(datum.getDataType())) {
+                continue;
+            } else if (datum.getDataType().isCoercibleTo(childType)) {
+                datum = child;
+            }
+        }
+        // If we found an "unknown" child type and the return type is a number
+        // make the return type be the most general number type of DECIMAL.
+        // TODO: same for TIMESTAMP for DATE/TIME?
+        if (isChildTypeUnknown && datum.getDataType() != null && datum.getDataType().isCoercibleTo(PDataType.DECIMAL)) {
+            return DECIMAL_DATUM;
+        }
+        return datum;
+    }
+
+    @Override
+    public boolean visitEnter(IsNullParseNode node) throws SQLException {
+        return true;
+    }
+
+    @Override
+    public Expression visitLeave(IsNullParseNode node, List<Expression> children) throws SQLException {
+        ParseNode childNode = node.getChildren().get(0);
+        Expression child = children.get(0);
+        if (childNode instanceof BindParseNode) { // TODO: valid/possibe?
+            context.getBindManager().addParamMetaData((BindParseNode)childNode, child);
+        }
+        Boolean value = null;
+        // Can't use child.isConstant(), because we have cases in which we don't know
+        // in advance if a function evaluates to null (namely when bind variables are used)
+        // TODO: review: have ParseNode.getValue()?
+        if (child instanceof LiteralExpression) {
+            value = (Boolean)((LiteralExpression)child).getValue();
+            return node.isNegate() ^ value == null ? LiteralExpression.newConstant(false, PDataType.BOOLEAN, child.isDeterministic()) : LiteralExpression.newConstant(true, PDataType.BOOLEAN, child.isDeterministic());
+        }
+        if (!child.isNullable()) { // If child expression is not nullable, we can rewrite this
+            return node.isNegate() ? LiteralExpression.newConstant(true, PDataType.BOOLEAN, child.isDeterministic()) : LiteralExpression.newConstant(false, PDataType.BOOLEAN, child.isDeterministic());
+        }
+        return new IsNullExpression(child, node.isNegate());
+    }
+
+    private static interface ArithmeticExpressionFactory {
+        Expression create(ArithmeticParseNode node, List<Expression> children) throws SQLException;
+    }
+
+    private static interface ArithmeticExpressionBinder {
+        PDatum getBindMetaData(int i, List<Expression> children, Expression expression);
+    }
+
+    private Expression visitLeave(ArithmeticParseNode node, List<Expression> children, ArithmeticExpressionBinder binder, ArithmeticExpressionFactory factory)
+            throws SQLException {
+
+        boolean isNull = false;
+        for (Expression child : children) {
+            boolean isChildLiteral = (child instanceof LiteralExpression);
+            isNull |= isChildLiteral && ((LiteralExpression)child).getValue() == null;
+        }
+
+        Expression expression = factory.create(node, children);
+
+        for (int i = 0; i < node.getChildren().size(); i++) {
+            ParseNode childNode = node.getChildren().get(i);
+            if (childNode instanceof BindParseNode) {
+                context.getBindManager().addParamMetaData((BindParseNode)childNode, binder == null ? expression : binder.getBindMetaData(i, children, expression));
+            }
+        }
+
+        ImmutableBytesWritable ptr = context.getTempPtr();
+
+        // If all children are literals, just evaluate now
+        if (expression.isStateless()) {
+            if (!expression.evaluate(null,ptr) || ptr.getLength() == 0) {
+                return LiteralExpression.newConstant(null, expression.getDataType(), expression.isDeterministic());
+            }
+            return LiteralExpression.newConstant(expression.getDataType().toObject(ptr), expression.getDataType(), expression.isDeterministic());
+        } else if (isNull) {
+            return LiteralExpression.newConstant(null, expression.getDataType(), expression.isDeterministic());
+        }
+        // Otherwise create and return the expression
+        return wrapGroupByExpression(expression);
+    }
+
+    @Override
+    public boolean visitEnter(SubtractParseNode node) throws SQLException {
+        return true;
+    }
+
+    @Override
+    public Expression visitLeave(SubtractParseNode node,
+            List<Expression> children) throws SQLException {
+        return visitLeave(node, children, new ArithmeticExpressionBinder() {
+            @Override
+            public PDatum getBindMetaData(int i, List<Expression> children,
+                    final Expression expression) {
+                final PDataType type;
+                // If we're binding the first parameter and the second parameter
+                // is a date
+                // we know that the first parameter must be a date type too.
+                if (i == 0 && (type = children.get(1).getDataType()) != null
+                        && type.isCoercibleTo(PDataType.DATE)) {
+                    return new PDatum() {
+                        @Override
+                        public boolean isNullable() {
+                            return expression.isNullable();
+                        }
+                        @Override
+                        public PDataType getDataType() {
+                            return type;
+                        }
+                        @Override
+                        public Integer getByteSize() {
+                            return type.getByteSize();
+                        }
+                        @Override
+                        public Integer getMaxLength() {
+                            return expression.getMaxLength();
+                        }
+                        @Override
+                        public Integer getScale() {
+                            return expression.getScale();
+                        }
+                        @Override
+                        public ColumnModifier getColumnModifier() {
+                            return expression.getColumnModifier();
+                        }                        
+                    };
+                } else if (expression.getDataType() != null
+                        && expression.getDataType().isCoercibleTo(
+                                PDataType.DATE)) {
+                    return new PDatum() { // Same as with addition
+                        @Override
+                        public boolean isNullable() {
+                            return expression.isNullable();
+                        }
+                        @Override
+                        public PDataType getDataType() {
+                            return PDataType.DECIMAL;
+                        }
+                        @Override
+                        public Integer getByteSize() {
+                            return null;
+                        }
+                        @Override
+                        public Integer getMaxLength() {
+                            return expression.getMaxLength();
+                        }
+                        @Override
+                        public Integer getScale() {
+                            return expression.getScale();
+                        }
+                        @Override
+                        public ColumnModifier getColumnModifier() {
+                            return expression.getColumnModifier();
+                        }
+                    };
+                }
+                // Otherwise just go with what was calculated for the expression
+                return expression;
+            }
+        }, new ArithmeticExpressionFactory() {
+            @Override
+            public Expression create(ArithmeticParseNode node,
+                    List<Expression> children) throws SQLException {
+                int i = 0;
+                PDataType theType = null;
+                Expression e1 = children.get(0);
+                Expression e2 = children.get(1);
+                boolean isDeterministic = e1.isDeterministic() && e2.isDeterministic();
+                PDataType type1 = e1.getDataType();
+                PDataType type2 = e2.getDataType();
+                // TODO: simplify this special case for DATE conversion
+                /**
+                 * For date1-date2, we want to coerce to a LONG because this
+                 * cannot be compared against another date. It has essentially
+                 * become a number. For date1-5, we want to preserve the DATE
+                 * type because this can still be compared against another date
+                 * and cannot be multiplied or divided. Any other time occurs is
+                 * an error. For example, 5-date1 is an error. The nulls occur if
+                 * we have bind variables.
+                 */
+                boolean isType1Date = 
+                        type1 != null 
+                        && type1 != PDataType.TIMESTAMP
+                        && type1 != PDataType.UNSIGNED_TIMESTAMP
+                        && type1.isCoercibleTo(PDataType.DATE);
+                boolean isType2Date = 
+                        type2 != null
+                        && type2 != PDataType.TIMESTAMP
+                        && type2 != PDataType.UNSIGNED_TIMESTAMP
+                        && type2.isCoercibleTo(PDataType.DATE);
+                if (isType1Date || isType2Date) {
+                    if (isType1Date && isType2Date) {
+                        i = 2;
+                        theType = PDataType.LONG;
+                    } else if (isType1Date && type2 != null
+                            && type2.isCoercibleTo(PDataType.DECIMAL)) {
+                        i = 2;
+                        theType = PDataType.DATE;
+                    } else if (type1 == null || type2 == null) {
+                        /*
+                         * FIXME: Could be either a Date or BigDecimal, but we
+                         * don't know if we're comparing to a date or a number
+                         * which would be disambiguate it.
+                         */
+                        i = 2;
+                        theType = null;
+                    }
+                } else if(type1 == PDataType.TIMESTAMP || type2 == PDataType.TIMESTAMP) {
+                    i = 2;
+                    theType = PDataType.TIMESTAMP;
+                } else if(type1 == PDataType.UNSIGNED_TIMESTAMP || type2 == PDataType.UNSIGNED_TIMESTAMP) {
+                    i = 2;
+                    theType = PDataType.UNSIGNED_TIMESTAMP;
+                }
+                
+                for (; i < children.size(); i++) {
+                    // This logic finds the common type to which all child types are coercible
+                    // without losing precision.
+                    Expression e = children.get(i);
+                    isDeterministic &= e.isDeterministic();
+                    PDataType type = e.getDataType();
+                    if (type == null) {
+                        continue;
+                    } else if (type.isCoercibleTo(PDataType.LONG)) {
+                        if (theType == null) {
+                            theType = PDataType.LONG;
+                        }
+                    } else if (type == PDataType.DECIMAL) {
+                        // Coerce return type to DECIMAL from LONG or DOUBLE if DECIMAL child found,
+                        // unless we're doing date arithmetic.
+                        if (theType == null
+                                || !theType.isCoercibleTo(PDataType.DATE)) {
+                            theType = PDataType.DECIMAL;
+                        }
+                    } else if (type.isCoercibleTo(PDataType.DOUBLE)) {
+                        // Coerce return type to DOUBLE from LONG if DOUBLE child found,
+                        // unless we're doing date arithmetic or we've found another child of type DECIMAL
+                        if (theType == null
+                                || (theType != PDataType.DECIMAL && !theType.isCoercibleTo(PDataType.DATE) )) {
+                            theType = PDataType.DOUBLE;
+                        }
+                    } else {
+                        throw TypeMismatchException.newException(type, node.toString());
+                    }
+                }
+                if (theType == PDataType.DECIMAL) {
+                    return new DecimalSubtractExpression(children);
+                } else if (theType == PDataType.LONG) {
+                    return new LongSubtractExpression(children);
+                } else if (theType == PDataType.DOUBLE) {
+                    return new DoubleSubtractExpression(children);
+                } else if (theType == null) {
+                    return LiteralExpression.newConstant(null, theType, isDeterministic);
+                } else if (theType == PDataType.TIMESTAMP || theType == PDataType.UNSIGNED_TIMESTAMP) {
+                    return new TimestampSubtractExpression(children);
+                } else if (theType.isCoercibleTo(PDataType.DATE)) {
+                    return new DateSubtractExpression(children);
+                } else {
+                    throw TypeMismatchException.newException(theType, node.toString());
+                }
+            }
+        });
+    }
+
+    @Override
+    public boolean visitEnter(AddParseNode node) throws SQLException {
+        return true;
+    }
+
+    @Override
+    public Expression visitLeave(AddParseNode node, List<Expression> children) throws SQLException {
+        return visitLeave(node, children,
+                new ArithmeticExpressionBinder() {
+            @Override
+            public PDatum getBindMetaData(int i, List<Expression> children, final Expression expression) {
+                PDataType type = expression.getDataType();
+                if (type != null && type.isCoercibleTo(PDataType.DATE)) {
+                    return new PDatum() {
+                        @Override
+                        public boolean isNullable() {
+                            return expression.isNullable();
+                        }
+                        @Override
+                        public PDataType getDataType() {
+                            return PDataType.DECIMAL;
+                        }
+                        @Override
+                        public Integer getByteSize() {
+                            return null;
+                        }
+                        @Override
+                        public Integer getMaxLength() {
+                            return expression.getMaxLength();
+                        }
+                        @Override
+                        public Integer getScale() {
+                            return expression.getScale();
+                        }
+                        @Override
+                        public ColumnModifier getColumnModifier() {
+                            return expression.getColumnModifier();
+                        }
+                    };
+                }
+                return expression;
+            }
+        },
+        new ArithmeticExpressionFactory() {
+            @Override
+            public Expression create(ArithmeticParseNode node, List<Expression> children) throws SQLException {
+                boolean foundDate = false;
+                boolean isDeterministic = true;
+                PDataType theType = null;
+                for(int i = 0; i < children.size(); i++) {
+                    Expression e = children.get(i);
+                    isDeterministic &= e.isDeterministic();
+                    PDataType type = e.getDataType();
+                    if (type == null) {
+                        continue; 
+                    } else if (type.isCoercibleTo(PDataType.TIMESTAMP)) {
+                        if (foundDate) {
+                            throw TypeMismatchException.newException(type, node.toString());
+                        }
+                        if (theType == null || (theType != PDataType.TIMESTAMP && theType != PDataType.UNSIGNED_TIMESTAMP)) {
+                            theType = type;
+                        }
+                        foundDate = true;
+                    }else if (type == PDataType.DECIMAL) {
+                        if (theType == null || !theType.isCoercibleTo(PDataType.TIMESTAMP)) {
+                            theType = PDataType.DECIMAL;
+                        }
+                    } else if (type.isCoercibleTo(PDataType.LONG)) {
+                        if (theType == null) {
+                            theType = PDataType.LONG;
+                        }
+                    } else if (type.isCoercibleTo(PDataType.DOUBLE)) {
+                        if (theType == null) {
+                            theType = PDataType.DOUBLE;
+                        }
+                    } else {
+                        throw TypeMismatchException.newException(type, node.toString());
+                    }
+                }
+                if (theType == PDataType.DECIMAL) {
+                    return new DecimalAddExpression(children);
+                } else if (theType == PDataType.LONG) {
+                    return new LongAddExpression(children);
+                } else if (theType == PDataType.DOUBLE) {
+                    return new DoubleAddExpression(children);
+                } else if (theType == null) {
+                    return LiteralExpression.newConstant(null, theType, isDeterministic);
+                } else if (theType == PDataType.TIMESTAMP || theType == PDataType.UNSIGNED_TIMESTAMP) {
+                    return new TimestampAddExpression(children);
+                } else if (theType.isCoercibleTo(PDataType.DATE)) {
+                    return new DateAddExpression(children);
+                } else {
+                    throw TypeMismatchException.newException(theType, node.toString());
+                }
+            }
+        });
+    }
+
+    @Override
+    public boolean visitEnter(MultiplyParseNode node) throws SQLException {
+        return true;
+    }
+
+    @Override
+    public Expression visitLeave(MultiplyParseNode node, List<Expression> children) throws SQLException {
+        return visitLeave(node, children, null, new ArithmeticExpressionFactory() {
+            @Override
+            public Expression create(ArithmeticParseNode node, List<Expression> children) throws SQLException {
+                PDataType theType = null;
+                boolean isDeterministic = true;
+                for(int i = 0; i < children.size(); i++) {
+                    Expression e = children.get(i);
+                    isDeterministic &= e.isDeterministic();
+                    PDataType type = e.getDataType();
+                    if (type == null) {
+                        continue;
+                    } else if (type == PDataType.DECIMAL) {
+                        theType = PDataType.DECIMAL;
+                    } else if (type.isCoercibleTo(PDataType.LONG)) {
+                        if (theType == null) {
+                            theType = PDataType.LONG;
+                        }
+                    } else if (type.isCoercibleTo(PDataType.DOUBLE)) {
+                        if (theType == null) {
+                            theType = PDataType.DOUBLE;
+                        }
+                    } else {
+                        throw TypeMismatchException.newException(type, node.toString());
+                    }
+                }
+                switch (theType) {
+                case DECIMAL:
+                    return new DecimalMultiplyExpression( children);
+                case LONG:
+                    return new LongMultiplyExpression( children);
+                case DOUBLE:
+                    return new DoubleMultiplyExpression( children);
+                default:
+                    return LiteralExpression.newConstant(null, theType, isDeterministic);
+                }
+            }
+        });
+    }
+
+    @Override
+    public boolean visitEnter(DivideParseNode node) throws SQLException {
+        return true;
+    }
+
+    @Override
+    public Expression visitLeave(DivideParseNode node, List<Expression> children) throws SQLException {
+        for (int i = 1; i < children.size(); i++) { // Compile time check for divide by zero and null
+            Expression child = children.get(i);
+                if (child.getDataType() != null && child instanceof LiteralExpression) {
+                    LiteralExpression literal = (LiteralExpression)child;
+                    if (literal.getDataType() == PDataType.DECIMAL) {
+                        if (PDataType.DECIMAL.compareTo(literal.getValue(), BigDecimal.ZERO) == 0) {
+                            throw new SQLExceptionInfo.Builder(SQLExceptionCode.DIVIDE_BY_ZERO).build().buildException();
+                        }
+                    } else {
+                        if (literal.getDataType().compareTo(literal.getValue(), 0L, PDataType.LONG) == 0) {
+                            throw new SQLExceptionInfo.Builder(SQLExceptionCode.DIVIDE_BY_ZERO).build().buildException();
+                        }
+                    }
+                }
+        }
+        return visitLeave(node, children, null, new ArithmeticExpressionFactory() {
+            @Override
+            public Expression create(ArithmeticParseNode node, List<Expression> children) throws SQLException {
+                PDataType theType = null;
+                boolean isDeterministic = true;
+                for(int i = 0; i < children.size(); i++) {
+                    Expression e = children.get(i);
+                    isDeterministic &= e.isDeterministic();
+                    PDataType type = e.getDataType();
+                    if (type == null) {
+                        continue;
+                    } else if (type == PDataType.DECIMAL) {
+                        theType = PDataType.DECIMAL;
+                    } else if (type.isCoercibleTo(PDataType.LONG)) {
+                        if (theType == null) {
+                            theType = PDataType.LONG;
+                        }
+                    } else if (type.isCoercibleTo(PDataType.DOUBLE)) {
+                        if (theType == null) {
+                            theType = PDataType.DOUBLE;
+                        }
+                    } else {
+                        throw TypeMismatchException.newException(type, node.toString());
+                    }
+                }
+                switch (theType) {
+                case DECIMAL:
+                    return new DecimalDivideExpression( children);
+                case LONG:
+                    return new LongDivideExpression( children);
+                case DOUBLE:
+                    return new DoubleDivideExpression(children);
+                default:
+                    return LiteralExpression.newConstant(null, theType, isDeterministic);
+                }
+            }
+        });
+    }
+
+    public static void throwNonAggExpressionInAggException(String nonAggregateExpression) throws SQLException {
+        throw new SQLExceptionInfo.Builder(SQLExceptionCode.AGGREGATE_WITH_NOT_GROUP_BY_COLUMN)
+        .setMessage(nonAggregateExpression).build().buildException();
+    }
+
+    @Override
+    public Expression visitLeave(StringConcatParseNode node, List<Expression> children) throws SQLException {
+        final StringConcatExpression expression=new StringConcatExpression(children);
+        for (int i = 0; i < children.size(); i++) {
+            ParseNode childNode=node.getChildren().get(i);
+            if(childNode instanceof BindParseNode) {
+                context.getBindManager().addParamMetaData((BindParseNode)childNode,expression);
+            }
+            PDataType type=children.get(i).getDataType();
+            if(type==PDataType.VARBINARY){
+                throw new SQLExceptionInfo.Builder(SQLExceptionCode.TYPE_NOT_SUPPORTED_FOR_OPERATOR)
+                .setMessage("Concatenation does not support "+ type +" in expression" + node).build().buildException();
+            }
+        }
+        ImmutableBytesWritable ptr = context.getTempPtr();
+        if (expression.isStateless()) {
+            if (!expression.evaluate(null,ptr) || ptr.getLength() == 0) {
+                return LiteralExpression.newConstant(null, expression.getDataType(), expression.isDeterministic());
+            }
+            return LiteralExpression.newConstant(expression.getDataType().toObject(ptr), expression.getDataType(), expression.isDeterministic());
+        }
+        return wrapGroupByExpression(expression);
+    }
+
+    @Override
+    public boolean visitEnter(StringConcatParseNode node) throws SQLException {
+        return true;
+    }
+
+    @Override
+    public boolean visitEnter(RowValueConstructorParseNode node) throws SQLException {
+        return true;
+    }
+
+    @Override
+    public Expression visitLeave(RowValueConstructorParseNode node, List<Expression> l) throws SQLException {
+        // Don't trim trailing nulls here, as we'd potentially be dropping bind
+        // variables that aren't bound yet.
+        return new RowValueConstructorExpression(l, node.isStateless());
+    }
+
+	@Override
+	public Expression visit(SequenceValueParseNode node)
+			throws SQLException {
+	    // NEXT VALUE FOR is only supported in SELECT expressions and UPSERT VALUES
+        throw new SQLExceptionInfo.Builder(SQLExceptionCode.INVALID_USE_OF_NEXT_VALUE_FOR)
+        .setSchemaName(node.getTableName().getSchemaName())
+        .setTableName(node.getTableName().getTableName()).build().buildException();
+	}
+
+    @Override
+    public Expression visitLeave(ArrayConstructorNode node, List<Expression> children) throws SQLException {
+        boolean isChildTypeUnknown = false;
+        Expression arrayElemChild = null;
+        PDataType arrayElemDataType = children.get(0).getDataType();
+        for (int i = 0; i < children.size(); i++) {
+            Expression child = children.get(i);
+            PDataType childType = child.getDataType();
+            if (childType == null) {
+                isChildTypeUnknown = true;
+            } else if (arrayElemDataType == null) {
+                arrayElemDataType = childType;
+                isChildTypeUnknown = true;
+                arrayElemChild = child;
+            } else if (arrayElemDataType == childType || childType.isCoercibleTo(arrayElemDataType)) {
+                continue;
+            } else if (arrayElemDataType.isCoercibleTo(childType)) {
+                arrayElemChild = child;
+                arrayElemDataType = childType;
+            } else {
+                throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_CONVERT_TYPE)
+                        .setMessage(
+                                "Case expressions must have common type: " + arrayElemDataType
+                                        + " cannot be coerced to " + childType).build().buildException();
+            }
+        }
+        // If we found an "unknown" child type and the return type is a number
+        // make the return type be the most general number type of DECIMAL.
+        if (isChildTypeUnknown && arrayElemDataType != null && arrayElemDataType.isCoercibleTo(PDataType.DECIMAL)) {
+            arrayElemDataType = PDataType.DECIMAL;
+        }
+        final PDataType theArrayElemDataType = arrayElemDataType;
+        for (int i = 0; i < node.getChildren().size(); i++) {
+            ParseNode childNode = node.getChildren().get(i);
+            if (childNode instanceof BindParseNode) {
+                context.getBindManager().addParamMetaData((BindParseNode)childNode,
+                        arrayElemDataType == arrayElemChild.getDataType() ? arrayElemChild :
+                            new DelegateDatum(arrayElemChild) {
+                    @Override
+                    public PDataType getDataType() {
+                        return theArrayElemDataType;
+                    }
+                });
+            }
+        }
+        ImmutableBytesWritable ptr = context.getTempPtr();
+        Object[] elements = new Object[children.size()];
+        if (node.isStateless()) {
+            boolean isDeterministic = true;
+            for (int i = 0; i < children.size(); i++) {
+                Expression child = children.get(i);
+                isDeterministic &= child.isDeterministic();
+                child.evaluate(null, ptr);
+                Object value = arrayElemDataType.toObject(ptr, child.getDataType(), child.getColumnModifier());
+                elements[i] = LiteralExpression.newConstant(value, child.getDataType(), child.isDeterministic()).getValue();
+            }
+            Object value = PArrayDataType.instantiatePhoenixArray(arrayElemDataType, elements);
+            return LiteralExpression.newConstant(value,
+                    PDataType.fromTypeId(arrayElemDataType.getSqlType() + Types.ARRAY), isDeterministic);
+        }
+        
+        ArrayConstructorExpression arrayExpression = new ArrayConstructorExpression(children, arrayElemDataType);
+        return wrapGroupByExpression(arrayExpression);
+    }
+
+    @Override
+    public boolean visitEnter(ArrayConstructorNode node) throws SQLException {
+        return true;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionManager.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionManager.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionManager.java
new file mode 100644
index 0000000..f66e61f
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionManager.java
@@ -0,0 +1,70 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.compile;
+
+import java.util.Iterator;
+import java.util.Map;
+
+
+import com.google.common.collect.Maps;
+import org.apache.phoenix.expression.Expression;
+
+/**
+ * 
+ * Class to manage list of expressions inside of a select statement by
+ * deduping them.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class ExpressionManager {
+    // Use a Map instead of a Set because we need to get and return
+    // the existing Expression
+    private final Map<Expression, Expression> expressionMap;
+    
+    public ExpressionManager() {
+        expressionMap = Maps.newHashMap();
+    }
+    
+    /**
+     * Add the expression to the set of known expressions for the select
+     * clause. If the expression is already in the set, then the new one
+     * passed in is ignored.
+     * @param expression the new expression to add
+     * @return the new expression if not already present in the set and
+     * the existing one otherwise.
+     */
+    public Expression addIfAbsent(Expression expression) {
+        Expression existingExpression = expressionMap.get(expression);
+        if (existingExpression == null) {
+            expressionMap.put(expression, expression);
+            return expression;
+        }
+        return existingExpression;
+    }
+    
+    public int getExpressionCount() {
+        return expressionMap.size();
+    }
+    
+    public Iterator<Expression> getExpressions() {
+        return expressionMap.keySet().iterator();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionProjector.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionProjector.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionProjector.java
new file mode 100644
index 0000000..8b30f0f
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionProjector.java
@@ -0,0 +1,93 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.compile;
+
+
+import java.sql.SQLException;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+
+
+/**
+ * 
+ * Projector for getting value from a select statement for an expression
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class ExpressionProjector implements ColumnProjector {
+    private final String name;
+    private final Expression expression;
+    private final String tableName;
+    private final boolean isCaseSensitive;
+    
+    public ExpressionProjector(String name, String tableName, Expression expression, boolean isCaseSensitive) {
+        this.name = name;
+        this.expression = expression;
+        this.tableName = tableName;
+        this.isCaseSensitive = isCaseSensitive;
+    }
+    
+    @Override
+    public String getTableName() {
+        return tableName;
+    }
+
+    @Override
+    public Expression getExpression() {
+        return expression;
+    }
+
+    @Override
+    public String getName() {
+        return name;
+    }
+
+    @Override
+    public final Object getValue(Tuple tuple, PDataType type, ImmutableBytesWritable ptr) throws SQLException {
+        try {
+            Expression expression = getExpression();
+            if (!expression.evaluate(tuple, ptr)) {
+                return null;
+            }
+            if (ptr.getLength() == 0) {
+                return null;
+            }        
+            return type.toObject(ptr, expression.getDataType(), expression.getColumnModifier());
+        } catch (RuntimeException e) {
+            // FIXME: Expression.evaluate does not throw SQLException
+            // so this will unwrap throws from that.
+            if (e.getCause() instanceof SQLException) {
+                throw (SQLException) e.getCause();
+            }
+            throw e;
+        }
+    }
+
+    @Override
+    public boolean isCaseSensitive() {
+        return isCaseSensitive;
+    }
+}


[36/51] [partial] Initial commit of master branch from github

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..3218539
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/HashJoinRegionScanner.java
@@ -0,0 +1,284 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.coprocessor;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Queue;
+
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.cache.GlobalCache;
+import org.apache.phoenix.cache.HashCache;
+import org.apache.phoenix.cache.TenantCache;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.join.HashJoinInfo;
+import org.apache.phoenix.join.ScanProjector;
+import org.apache.phoenix.join.ScanProjector.ProjectedValueTuple;
+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.ResultTuple;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.TupleUtil;
+
+public class HashJoinRegionScanner implements RegionScanner {
+    
+    private final RegionScanner scanner;
+    private final ScanProjector projector;
+    private final HashJoinInfo joinInfo;
+    private Queue<ProjectedValueTuple> resultQueue;
+    private boolean hasMore;
+    private HashCache[] hashCaches;
+    private List<Tuple>[] tempTuples;
+    private ValueBitSet tempDestBitSet;
+    private ValueBitSet[] tempSrcBitSet;
+    
+    @SuppressWarnings("unchecked")
+    public HashJoinRegionScanner(RegionScanner scanner, ScanProjector projector, HashJoinInfo joinInfo, ImmutableBytesWritable tenantId, RegionCoprocessorEnvironment env) throws IOException {
+        assert (projector != null);
+        this.scanner = scanner;
+        this.projector = projector;
+        this.joinInfo = joinInfo;
+        this.resultQueue = new LinkedList<ProjectedValueTuple>();
+        this.hasMore = true;
+        if (joinInfo != null) {
+            for (JoinType type : joinInfo.getJoinTypes()) {
+                if (type != JoinType.Inner && type != JoinType.Left)
+                    throw new IOException("Got join type '" + type + "'. Expect only INNER or LEFT with hash-joins.");
+            }
+            int count = joinInfo.getJoinIds().length;
+            this.tempTuples = new List[count];
+            this.tempDestBitSet = ValueBitSet.newInstance(joinInfo.getJoinedSchema());
+            this.hashCaches = new HashCache[count];
+            this.tempSrcBitSet = new ValueBitSet[count];
+            TenantCache cache = GlobalCache.getTenantCache(env, tenantId);
+            for (int i = 0; i < count; i++) {
+                ImmutableBytesPtr joinId = joinInfo.getJoinIds()[i];
+                HashCache hashCache = (HashCache)cache.getServerCache(joinId);
+                if (hashCache == null)
+                    throw new IOException("Could not find hash cache for joinId: " + Bytes.toString(joinId.get(), joinId.getOffset(), joinId.getLength()));
+                hashCaches[i] = hashCache;
+                tempSrcBitSet[i] = ValueBitSet.newInstance(joinInfo.getSchemas()[i]);
+            }
+            this.projector.setValueBitSet(tempDestBitSet);
+        }
+    }
+    
+    private void processResults(List<KeyValue> result, boolean hasLimit) throws IOException {
+        if (result.isEmpty())
+            return;
+        
+        ProjectedValueTuple tuple = projector.projectResults(new ResultTuple(new Result(result)));
+        if (joinInfo == null) {
+            resultQueue.offer(tuple);
+            return;
+        }
+        
+        if (hasLimit)
+            throw new UnsupportedOperationException("Cannot support join operations in scans with limit");
+
+        int count = joinInfo.getJoinIds().length;
+        boolean cont = true;
+        for (int i = 0; i < count; i++) {
+            if (!(joinInfo.earlyEvaluation()[i]))
+                continue;
+            ImmutableBytesPtr key = TupleUtil.getConcatenatedValue(tuple, joinInfo.getJoinExpressions()[i]);
+            tempTuples[i] = hashCaches[i].get(key);
+            JoinType type = joinInfo.getJoinTypes()[i];
+            if (type == JoinType.Inner && (tempTuples[i] == null || tempTuples[i].isEmpty())) {
+                cont = false;
+                break;
+            }
+        }
+        if (cont) {
+            KeyValueSchema schema = joinInfo.getJoinedSchema();
+            resultQueue.offer(tuple);
+            for (int i = 0; i < count; i++) {
+                boolean earlyEvaluation = joinInfo.earlyEvaluation()[i];
+                if (earlyEvaluation && 
+                        (tempTuples[i] == null || tempTuples[i].isEmpty()))
+                    continue;
+                int j = resultQueue.size();
+                while (j-- > 0) {
+                    ProjectedValueTuple lhs = resultQueue.poll();
+                    if (!earlyEvaluation) {
+                        ImmutableBytesPtr key = TupleUtil.getConcatenatedValue(lhs, joinInfo.getJoinExpressions()[i]);
+                        tempTuples[i] = hashCaches[i].get(key);                        	
+                        if (tempTuples[i] == null || tempTuples[i].isEmpty()) {
+                            if (joinInfo.getJoinTypes()[i] != JoinType.Inner) {
+                                resultQueue.offer(lhs);
+                            }
+                            continue;
+                        }
+                    }
+                    for (Tuple t : tempTuples[i]) {
+                        ProjectedValueTuple joined = tempSrcBitSet[i] == ValueBitSet.EMPTY_VALUE_BITSET ?
+                                lhs : ScanProjector.mergeProjectedValue(
+                                        lhs, schema, tempDestBitSet,
+                                        t, joinInfo.getSchemas()[i], tempSrcBitSet[i], 
+                                        joinInfo.getFieldPositions()[i]);
+                        resultQueue.offer(joined);
+                    }
+                }
+            }
+            // apply post-join filter
+            Expression postFilter = joinInfo.getPostJoinFilterExpression();
+            if (postFilter != null) {
+                for (Iterator<ProjectedValueTuple> iter = resultQueue.iterator(); iter.hasNext();) {
+                    ProjectedValueTuple t = iter.next();
+                    ImmutableBytesWritable tempPtr = new ImmutableBytesWritable();
+                    try {
+                        if (!postFilter.evaluate(t, tempPtr)) {
+                            iter.remove();
+                            continue;
+                        }
+                    } catch (IllegalDataException e) {
+                        iter.remove();
+                        continue;
+                    }
+                    Boolean b = (Boolean)postFilter.getDataType().toObject(tempPtr);
+                    if (!b.booleanValue()) {
+                        iter.remove();
+                    }
+                }
+            }
+        }
+    }
+    
+    private boolean shouldAdvance() {
+        if (!resultQueue.isEmpty())
+            return false;
+        
+        return hasMore;
+    }
+    
+    private boolean nextInQueue(List<KeyValue> results) {
+        if (resultQueue.isEmpty())
+            return false;
+        
+        results.add(resultQueue.poll().getValue(0));
+        return resultQueue.isEmpty() ? hasMore : true;
+    }
+
+    @Override
+    public long getMvccReadPoint() {
+        return scanner.getMvccReadPoint();
+    }
+
+    @Override
+    public HRegionInfo getRegionInfo() {
+        return scanner.getRegionInfo();
+    }
+
+    @Override
+    public boolean isFilterDone() {
+        return scanner.isFilterDone() && resultQueue.isEmpty();
+    }
+
+    @Override
+    public boolean nextRaw(List<KeyValue> result, String metric) throws IOException {
+        while (shouldAdvance()) {
+            hasMore = scanner.nextRaw(result, metric);
+            processResults(result, false);
+            result.clear();
+        }
+        
+        return nextInQueue(result);
+    }
+
+    @Override
+    public boolean nextRaw(List<KeyValue> result, int limit, String metric)
+            throws IOException {
+        while (shouldAdvance()) {
+            hasMore = scanner.nextRaw(result, limit, metric);
+            processResults(result, true);
+            result.clear();
+        }
+        
+        return nextInQueue(result);
+    }
+
+    @Override
+    public boolean reseek(byte[] row) throws IOException {
+        return scanner.reseek(row);
+    }
+
+    @Override
+    public void close() throws IOException {
+        scanner.close();
+    }
+
+    @Override
+    public boolean next(List<KeyValue> result) throws IOException {
+        while (shouldAdvance()) {
+            hasMore = scanner.next(result);
+            processResults(result, false);
+            result.clear();
+        }
+        
+        return nextInQueue(result);
+    }
+
+    @Override
+    public boolean next(List<KeyValue> result, String metric) throws IOException {
+        while (shouldAdvance()) {
+            hasMore = scanner.next(result, metric);
+            processResults(result, false);
+            result.clear();
+        }
+        
+        return nextInQueue(result);
+    }
+
+    @Override
+    public boolean next(List<KeyValue> result, int limit) throws IOException {
+        while (shouldAdvance()) {
+            hasMore = scanner.next(result, limit);
+            processResults(result, true);
+            result.clear();
+        }
+        
+        return nextInQueue(result);
+    }
+
+    @Override
+    public boolean next(List<KeyValue> result, int limit, String metric)
+            throws IOException {
+        while (shouldAdvance()) {
+            hasMore = scanner.next(result, limit, metric);
+            processResults(result, true);
+            result.clear();
+        }
+        
+        return nextInQueue(result);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..ec0d9f5
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
@@ -0,0 +1,1220 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.coprocessor;
+
+import static com.google.common.collect.Lists.newArrayList;
+import static org.apache.hadoop.hbase.filter.CompareFilter.CompareOp.EQUAL;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ARRAY_SIZE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_COUNT_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_MODIFIER;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_NAME_INDEX;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_SIZE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DATA_TABLE_NAME_BYTES;
+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_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DISABLE_WAL_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_STATE_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.LINK_TYPE_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.MULTI_TENANT_BYTES;
+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_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.TABLE_FAMILY_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_NAME_INDEX;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_SEQ_NUM_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_TYPE_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TENANT_ID_INDEX;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_STATEMENT_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_TYPE_BYTES;
+import static org.apache.phoenix.schema.PTableType.INDEX;
+import static org.apache.phoenix.util.SchemaUtil.getVarCharLength;
+import static org.apache.phoenix.util.SchemaUtil.getVarChars;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.coprocessor.BaseEndpointCoprocessor;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterBase;
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
+import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+import org.apache.hadoop.hbase.index.util.IndexManagementUtil;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.phoenix.cache.GlobalCache;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.AmbiguousColumnException;
+import org.apache.phoenix.schema.ColumnFamilyNotFoundException;
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.ColumnNotFoundException;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PColumnFamily;
+import org.apache.phoenix.schema.PColumnImpl;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.PIndexState;
+import org.apache.phoenix.schema.PName;
+import org.apache.phoenix.schema.PNameFactory;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTable.LinkType;
+import org.apache.phoenix.schema.PTable.ViewType;
+import org.apache.phoenix.schema.PTableImpl;
+import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.schema.TableNotFoundException;
+import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.IndexUtil;
+import org.apache.phoenix.util.KeyValueUtil;
+import org.apache.phoenix.util.MetaDataUtil;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.ServerUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Lists;
+
+/**
+ * 
+ * Endpoint co-processor through which all Phoenix metadata mutations flow.
+ * We only allow mutations to the latest version of a Phoenix table (i.e. the
+ * timeStamp must be increasing).
+ * For adding/dropping columns use a sequence number on the table to ensure that
+ * the client has the latest version.
+ * The timeStamp on the table correlates with the timeStamp on the data row.
+ * TODO: we should enforce that a metadata mutation uses a timeStamp bigger than
+ * any in use on the data table, b/c otherwise we can end up with data rows that
+ * are not valid against a schema row.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class MetaDataEndpointImpl extends BaseEndpointCoprocessor implements MetaDataProtocol {
+    private static final Logger logger = LoggerFactory.getLogger(MetaDataEndpointImpl.class);
+
+    // KeyValues for Table
+    private static final KeyValue TABLE_TYPE_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, TABLE_TYPE_BYTES);
+    private static final KeyValue TABLE_SEQ_NUM_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, TABLE_SEQ_NUM_BYTES);
+    private static final KeyValue COLUMN_COUNT_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, COLUMN_COUNT_BYTES);
+    private static final KeyValue SALT_BUCKETS_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, SALT_BUCKETS_BYTES);
+    private static final KeyValue PK_NAME_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, PK_NAME_BYTES);
+    private static final KeyValue DATA_TABLE_NAME_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, DATA_TABLE_NAME_BYTES);
+    private static final KeyValue INDEX_STATE_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, INDEX_STATE_BYTES);
+    private static final KeyValue IMMUTABLE_ROWS_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, IMMUTABLE_ROWS_BYTES);
+    private static final KeyValue VIEW_EXPRESSION_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, VIEW_STATEMENT_BYTES);
+    private static final KeyValue DEFAULT_COLUMN_FAMILY_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, DEFAULT_COLUMN_FAMILY_NAME_BYTES);
+    private static final KeyValue DISABLE_WAL_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, DISABLE_WAL_BYTES);
+    private static final KeyValue MULTI_TENANT_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, MULTI_TENANT_BYTES);
+    private static final KeyValue VIEW_TYPE_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, VIEW_TYPE_BYTES);
+    private static final List<KeyValue> TABLE_KV_COLUMNS = Arrays.<KeyValue>asList(
+            TABLE_TYPE_KV,
+            TABLE_SEQ_NUM_KV,
+            COLUMN_COUNT_KV,
+            SALT_BUCKETS_KV,
+            PK_NAME_KV,
+            DATA_TABLE_NAME_KV,
+            INDEX_STATE_KV,
+            IMMUTABLE_ROWS_KV,
+            VIEW_EXPRESSION_KV,
+            DEFAULT_COLUMN_FAMILY_KV,
+            DISABLE_WAL_KV,
+            MULTI_TENANT_KV,
+            VIEW_TYPE_KV
+            );
+    static {
+        Collections.sort(TABLE_KV_COLUMNS, KeyValue.COMPARATOR);
+    }
+    private static final int TABLE_TYPE_INDEX = TABLE_KV_COLUMNS.indexOf(TABLE_TYPE_KV);
+    private static final int TABLE_SEQ_NUM_INDEX = TABLE_KV_COLUMNS.indexOf(TABLE_SEQ_NUM_KV);
+    private static final int COLUMN_COUNT_INDEX = TABLE_KV_COLUMNS.indexOf(COLUMN_COUNT_KV);
+    private static final int SALT_BUCKETS_INDEX = TABLE_KV_COLUMNS.indexOf(SALT_BUCKETS_KV);
+    private static final int PK_NAME_INDEX = TABLE_KV_COLUMNS.indexOf(PK_NAME_KV);
+    private static final int DATA_TABLE_NAME_INDEX = TABLE_KV_COLUMNS.indexOf(DATA_TABLE_NAME_KV);
+    private static final int INDEX_STATE_INDEX = TABLE_KV_COLUMNS.indexOf(INDEX_STATE_KV);
+    private static final int IMMUTABLE_ROWS_INDEX = TABLE_KV_COLUMNS.indexOf(IMMUTABLE_ROWS_KV);
+    private static final int VIEW_STATEMENT_INDEX = TABLE_KV_COLUMNS.indexOf(VIEW_EXPRESSION_KV);
+    private static final int DEFAULT_COLUMN_FAMILY_INDEX = TABLE_KV_COLUMNS.indexOf(DEFAULT_COLUMN_FAMILY_KV);
+    private static final int DISABLE_WAL_INDEX = TABLE_KV_COLUMNS.indexOf(DISABLE_WAL_KV);
+    private static final int MULTI_TENANT_INDEX = TABLE_KV_COLUMNS.indexOf(MULTI_TENANT_KV);
+    private static final int VIEW_TYPE_INDEX = TABLE_KV_COLUMNS.indexOf(VIEW_TYPE_KV);
+    
+    // KeyValues for Column
+    private static final KeyValue DECIMAL_DIGITS_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, Bytes.toBytes(DECIMAL_DIGITS));
+    private static final KeyValue COLUMN_SIZE_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, Bytes.toBytes(COLUMN_SIZE));
+    private static final KeyValue NULLABLE_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, Bytes.toBytes(NULLABLE));
+    private static final KeyValue DATA_TYPE_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, Bytes.toBytes(DATA_TYPE));
+    private static final KeyValue ORDINAL_POSITION_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, Bytes.toBytes(ORDINAL_POSITION));
+    private static final KeyValue COLUMN_MODIFIER_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, Bytes.toBytes(COLUMN_MODIFIER));
+    private static final KeyValue ARRAY_SIZE_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, Bytes.toBytes(ARRAY_SIZE));
+    private static final List<KeyValue> COLUMN_KV_COLUMNS = Arrays.<KeyValue>asList(
+            DECIMAL_DIGITS_KV,
+            COLUMN_SIZE_KV,
+            NULLABLE_KV,
+            DATA_TYPE_KV,
+            ORDINAL_POSITION_KV,
+            COLUMN_MODIFIER_KV,
+            DATA_TABLE_NAME_KV, // included in both column and table row for metadata APIs
+            ARRAY_SIZE_KV
+            );
+    static {
+        Collections.sort(COLUMN_KV_COLUMNS, KeyValue.COMPARATOR);
+    }
+    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);
+    private static final int SQL_DATA_TYPE_INDEX = COLUMN_KV_COLUMNS.indexOf(DATA_TYPE_KV);
+    private static final int ORDINAL_POSITION_INDEX = COLUMN_KV_COLUMNS.indexOf(ORDINAL_POSITION_KV);
+    private static final int COLUMN_MODIFIER_INDEX = COLUMN_KV_COLUMNS.indexOf(COLUMN_MODIFIER_KV);
+    private static final int ARRAY_SIZE_INDEX = COLUMN_KV_COLUMNS.indexOf(ARRAY_SIZE_KV);
+    
+    private static final int LINK_TYPE_INDEX = 0;
+
+    private static PName newPName(byte[] keyBuffer, int keyOffset, int keyLength) {
+        if (keyLength <= 0) {
+            return null;
+        }
+        int length = getVarCharLength(keyBuffer, keyOffset, keyLength);
+        // TODO: PNameImpl that doesn't need to copy the bytes
+        byte[] pnameBuf = new byte[length];
+        System.arraycopy(keyBuffer, keyOffset, pnameBuf, 0, length);
+        return PNameFactory.newName(pnameBuf);
+    }
+    
+    private static Scan newTableRowsScan(byte[] key, long startTimeStamp, long stopTimeStamp) throws IOException {
+        Scan scan = new Scan();
+        scan.setTimeRange(startTimeStamp, stopTimeStamp);
+        scan.setStartRow(key);
+        byte[] stopKey = ByteUtil.concat(key, QueryConstants.SEPARATOR_BYTE_ARRAY);
+        ByteUtil.nextKey(stopKey, stopKey.length);
+        scan.setStopRow(stopKey);
+        return scan;
+    }
+
+    @Override
+    public RegionCoprocessorEnvironment getEnvironment() {
+        return (RegionCoprocessorEnvironment)super.getEnvironment();
+    }
+    
+    private PTable buildTable(byte[] key, ImmutableBytesPtr cacheKey, HRegion region, long clientTimeStamp) throws IOException, SQLException {
+        Scan scan = newTableRowsScan(key, MIN_TABLE_TIMESTAMP, clientTimeStamp);
+        RegionScanner scanner = region.getScanner(scan);
+        Map<ImmutableBytesPtr,PTable> metaDataCache = GlobalCache.getInstance(this.getEnvironment()).getMetaDataCache();
+        try {
+            PTable oldTable = metaDataCache.get(cacheKey);
+            long tableTimeStamp = oldTable == null ? MIN_TABLE_TIMESTAMP-1 : oldTable.getTimeStamp();
+            PTable newTable;
+            newTable = getTable(scanner, clientTimeStamp, tableTimeStamp);
+            if (newTable == null) {
+                return null;
+            }
+            if (oldTable == null || tableTimeStamp < newTable.getTimeStamp()) {
+                if (logger.isDebugEnabled()) {
+                    logger.debug("Caching table " + Bytes.toStringBinary(cacheKey.get(), cacheKey.getOffset(), cacheKey.getLength()) + " at seqNum " + newTable.getSequenceNumber() + " with newer timestamp " + newTable.getTimeStamp() + " versus " + tableTimeStamp);
+                }
+                oldTable = metaDataCache.put(cacheKey, newTable);
+                if (logger.isDebugEnabled()) {
+                    if (oldTable == null) {
+                        logger.debug("No previously cached table " + Bytes.toStringBinary(cacheKey.get(), cacheKey.getOffset(), cacheKey.getLength()));
+                    } else {
+                        logger.debug("Previously cached table " + Bytes.toStringBinary(cacheKey.get(), cacheKey.getOffset(), cacheKey.getLength()) + " was at seqNum " + oldTable.getSequenceNumber() + " with timestamp " + oldTable.getTimeStamp());
+                    }
+                }
+            }
+            return newTable;
+        } finally {
+            scanner.close();
+        }
+    }
+
+    private void addIndexToTable(PName tenantId, PName schemaName, PName indexName, PName tableName, long clientTimeStamp, List<PTable> indexes) throws IOException, SQLException {
+        byte[] key = SchemaUtil.getTableKey(tenantId == null ? ByteUtil.EMPTY_BYTE_ARRAY : tenantId.getBytes(), schemaName.getBytes(), indexName.getBytes());
+        PTable indexTable = doGetTable(key, clientTimeStamp);
+        if (indexTable == null) {
+            ServerUtil.throwIOException("Index not found", new TableNotFoundException(schemaName.getString(), indexName.getString()));
+            return;
+        }
+        indexes.add(indexTable);
+    }
+
+    private void addColumnToTable(List<KeyValue> results, PName colName, PName famName, KeyValue[] colKeyValues, List<PColumn> columns) {
+        int i = 0;
+        int j = 0;
+        while (i < results.size() && j < COLUMN_KV_COLUMNS.size()) {
+            KeyValue kv = results.get(i);
+            KeyValue searchKv = COLUMN_KV_COLUMNS.get(j);
+            int cmp = Bytes.compareTo(kv.getBuffer(), kv.getQualifierOffset(), kv.getQualifierLength(), 
+                    searchKv.getBuffer(), searchKv.getQualifierOffset(), searchKv.getQualifierLength());
+            if (cmp == 0) {
+                colKeyValues[j++] = kv;
+                i++;
+            } else if (cmp > 0) {
+                colKeyValues[j++] = null;
+            } else {
+                i++; // shouldn't happen - means unexpected KV in system table column row
+            }
+        }
+        // COLUMN_SIZE and DECIMAL_DIGIT are optional. NULLABLE, DATA_TYPE and ORDINAL_POSITION_KV are required.
+        if (colKeyValues[SQL_DATA_TYPE_INDEX] == null || colKeyValues[NULLABLE_INDEX] == null
+                || colKeyValues[ORDINAL_POSITION_INDEX] == null) {
+            throw new IllegalStateException("Didn't find all required key values in '" + colName.getString() + "' column metadata row");
+        }
+        KeyValue columnSizeKv = colKeyValues[COLUMN_SIZE_INDEX];
+        Integer maxLength = columnSizeKv == null ? null : PDataType.INTEGER.getCodec().decodeInt(columnSizeKv.getBuffer(), columnSizeKv.getValueOffset(), null);
+        KeyValue decimalDigitKv = colKeyValues[DECIMAL_DIGITS_INDEX];
+        Integer scale = decimalDigitKv == null ? null : PDataType.INTEGER.getCodec().decodeInt(decimalDigitKv.getBuffer(), decimalDigitKv.getValueOffset(), null);
+        KeyValue ordinalPositionKv = colKeyValues[ORDINAL_POSITION_INDEX];
+        int position = PDataType.INTEGER.getCodec().decodeInt(ordinalPositionKv.getBuffer(), ordinalPositionKv.getValueOffset(), null);
+        KeyValue nullableKv = colKeyValues[NULLABLE_INDEX];
+        boolean isNullable = PDataType.INTEGER.getCodec().decodeInt(nullableKv.getBuffer(), nullableKv.getValueOffset(), null) != ResultSetMetaData.columnNoNulls;
+        KeyValue sqlDataTypeKv = colKeyValues[SQL_DATA_TYPE_INDEX];
+        PDataType dataType = PDataType.fromTypeId(PDataType.INTEGER.getCodec().decodeInt(sqlDataTypeKv.getBuffer(), sqlDataTypeKv.getValueOffset(), null));
+        if (maxLength == null && dataType == PDataType.BINARY) dataType = PDataType.VARBINARY; // For backward compatibility.
+        KeyValue columnModifierKv = colKeyValues[COLUMN_MODIFIER_INDEX];
+        ColumnModifier sortOrder = columnModifierKv == null ? null : ColumnModifier.fromSystemValue(PDataType.INTEGER.getCodec().decodeInt(columnModifierKv.getBuffer(), columnModifierKv.getValueOffset(), null));
+        KeyValue arraySizeKv = colKeyValues[ARRAY_SIZE_INDEX];
+        Integer arraySize = arraySizeKv == null ? null : PDataType.INTEGER.getCodec().decodeInt(arraySizeKv.getBuffer(), arraySizeKv.getValueOffset(), null);
+        PColumn column = new PColumnImpl(colName, famName, dataType, maxLength, scale, isNullable, position-1, sortOrder, arraySize);
+        columns.add(column);
+    }
+
+    private PTable getTable(RegionScanner scanner, long clientTimeStamp, long tableTimeStamp) throws IOException, SQLException {
+        List<KeyValue> results = Lists.newArrayList();
+        scanner.next(results);
+        if (results.isEmpty()) {
+            return null;
+        }
+        KeyValue[] tableKeyValues = new KeyValue[TABLE_KV_COLUMNS.size()];
+        KeyValue[] colKeyValues = new KeyValue[COLUMN_KV_COLUMNS.size()];
+        
+        // Create PTable based on KeyValues from scan
+        KeyValue keyValue = results.get(0);
+        byte[] keyBuffer = keyValue.getBuffer();
+        int keyLength = keyValue.getRowLength();
+        int keyOffset = keyValue.getRowOffset();
+        PName tenantId = newPName(keyBuffer, keyOffset, keyLength);
+        int tenantIdLength = tenantId.getBytes().length;
+        PName schemaName = newPName(keyBuffer, keyOffset+tenantIdLength+1, keyLength);
+        int schemaNameLength = schemaName.getBytes().length;
+        int tableNameLength = keyLength-schemaNameLength-1-tenantIdLength-1;
+        byte[] tableNameBytes = new byte[tableNameLength];
+        System.arraycopy(keyBuffer, keyOffset+schemaNameLength+1+tenantIdLength+1, tableNameBytes, 0, tableNameLength);
+        PName tableName = PNameFactory.newName(tableNameBytes);
+        
+        int offset = tenantIdLength + schemaNameLength + tableNameLength + 3;
+        // This will prevent the client from continually looking for the current
+        // table when we know that there will never be one since we disallow updates
+        // unless the table is the latest
+        // If we already have a table newer than the one we just found and
+        // the client timestamp is less that the existing table time stamp,
+        // bump up the timeStamp to right before the client time stamp, since
+        // we know it can't possibly change.
+        long timeStamp = keyValue.getTimestamp();
+//        long timeStamp = tableTimeStamp > keyValue.getTimestamp() && 
+//                         clientTimeStamp < tableTimeStamp
+//                         ? clientTimeStamp-1 
+//                         : keyValue.getTimestamp();
+
+        int i = 0;
+        int j = 0;
+        while (i < results.size() && j < TABLE_KV_COLUMNS.size()) {
+            KeyValue kv = results.get(i);
+            KeyValue searchKv = TABLE_KV_COLUMNS.get(j);
+            int cmp = Bytes.compareTo(kv.getBuffer(), kv.getQualifierOffset(), kv.getQualifierLength(), 
+                    searchKv.getBuffer(), searchKv.getQualifierOffset(), searchKv.getQualifierLength());
+            if (cmp == 0) {
+                timeStamp = Math.max(timeStamp, kv.getTimestamp()); // Find max timestamp of table header row
+                tableKeyValues[j++] = kv;
+                i++;
+            } else if (cmp > 0) {
+                tableKeyValues[j++] = null;
+            } else {
+                i++; // shouldn't happen - means unexpected KV in system table header row
+            }
+        }
+        // TABLE_TYPE, TABLE_SEQ_NUM and COLUMN_COUNT are required.
+        if (tableKeyValues[TABLE_TYPE_INDEX] == null || tableKeyValues[TABLE_SEQ_NUM_INDEX] == null
+                || tableKeyValues[COLUMN_COUNT_INDEX] == null) {
+            throw new IllegalStateException("Didn't find expected key values for table row in metadata row");
+        }
+        KeyValue tableTypeKv = tableKeyValues[TABLE_TYPE_INDEX];
+        PTableType tableType = PTableType.fromSerializedValue(tableTypeKv.getBuffer()[tableTypeKv.getValueOffset()]);
+        KeyValue tableSeqNumKv = tableKeyValues[TABLE_SEQ_NUM_INDEX];
+        long tableSeqNum = PDataType.LONG.getCodec().decodeLong(tableSeqNumKv.getBuffer(), tableSeqNumKv.getValueOffset(), null);
+        KeyValue columnCountKv = tableKeyValues[COLUMN_COUNT_INDEX];
+        int columnCount = PDataType.INTEGER.getCodec().decodeInt(columnCountKv.getBuffer(), columnCountKv.getValueOffset(), null);
+        KeyValue pkNameKv = tableKeyValues[PK_NAME_INDEX];
+        PName pkName = pkNameKv != null ? newPName(pkNameKv.getBuffer(), pkNameKv.getValueOffset(), pkNameKv.getValueLength()) : null;
+        KeyValue saltBucketNumKv = tableKeyValues[SALT_BUCKETS_INDEX];
+        Integer saltBucketNum = saltBucketNumKv != null ? (Integer)PDataType.INTEGER.getCodec().decodeInt(saltBucketNumKv.getBuffer(), saltBucketNumKv.getValueOffset(), null) : null;
+        KeyValue dataTableNameKv = tableKeyValues[DATA_TABLE_NAME_INDEX];
+        PName dataTableName = dataTableNameKv != null ? newPName(dataTableNameKv.getBuffer(), dataTableNameKv.getValueOffset(), dataTableNameKv.getValueLength()) : null;
+        KeyValue indexStateKv = tableKeyValues[INDEX_STATE_INDEX];
+        PIndexState indexState = indexStateKv == null ? null : PIndexState.fromSerializedValue(indexStateKv.getBuffer()[indexStateKv.getValueOffset()]);
+        KeyValue immutableRowsKv = tableKeyValues[IMMUTABLE_ROWS_INDEX];
+        boolean isImmutableRows = immutableRowsKv == null ? false : (Boolean)PDataType.BOOLEAN.toObject(immutableRowsKv.getBuffer(), immutableRowsKv.getValueOffset(), immutableRowsKv.getValueLength());
+        KeyValue defaultFamilyNameKv = tableKeyValues[DEFAULT_COLUMN_FAMILY_INDEX];
+        PName defaultFamilyName = defaultFamilyNameKv != null ? newPName(defaultFamilyNameKv.getBuffer(), defaultFamilyNameKv.getValueOffset(), defaultFamilyNameKv.getValueLength()) : null;
+        KeyValue viewStatementKv = tableKeyValues[VIEW_STATEMENT_INDEX];
+        String viewStatement = viewStatementKv != null ? (String)PDataType.VARCHAR.toObject(viewStatementKv.getBuffer(), viewStatementKv.getValueOffset(), viewStatementKv.getValueLength()) : null;
+        KeyValue disableWALKv = tableKeyValues[DISABLE_WAL_INDEX];
+        boolean disableWAL = disableWALKv == null ? PTable.DEFAULT_DISABLE_WAL : Boolean.TRUE.equals(PDataType.BOOLEAN.toObject(disableWALKv.getBuffer(), disableWALKv.getValueOffset(), disableWALKv.getValueLength()));
+        KeyValue multiTenantKv = tableKeyValues[MULTI_TENANT_INDEX];
+        boolean multiTenant = multiTenantKv == null ? false : Boolean.TRUE.equals(PDataType.BOOLEAN.toObject(multiTenantKv.getBuffer(), multiTenantKv.getValueOffset(), multiTenantKv.getValueLength()));
+        KeyValue viewTypeKv = tableKeyValues[VIEW_TYPE_INDEX];
+        ViewType viewType = viewTypeKv == null ? null : ViewType.fromSerializedValue(viewTypeKv.getBuffer()[viewTypeKv.getValueOffset()]);
+        
+        List<PColumn> columns = Lists.newArrayListWithExpectedSize(columnCount);
+        List<PTable> indexes = new ArrayList<PTable>();
+        List<PName> physicalTables = new ArrayList<PName>();
+        while (true) {
+            results.clear();
+            scanner.next(results);
+            if (results.isEmpty()) {
+                break;
+            }
+            KeyValue colKv = results.get(LINK_TYPE_INDEX);
+            int colKeyLength = colKv.getRowLength();
+            PName colName = newPName(colKv.getBuffer(), colKv.getRowOffset() + offset, colKeyLength-offset);
+            int colKeyOffset = offset + colName.getBytes().length + 1;
+            PName famName = newPName(colKv.getBuffer(), colKv.getRowOffset() + colKeyOffset, colKeyLength-colKeyOffset);
+            if (colName.getString().isEmpty() && famName != null) {
+                LinkType linkType = LinkType.fromSerializedValue(colKv.getBuffer()[colKv.getValueOffset()]);
+                if (linkType == LinkType.INDEX_TABLE) {
+                    addIndexToTable(tenantId, schemaName, famName, tableName, clientTimeStamp, indexes);
+                } else if (linkType == LinkType.PHYSICAL_TABLE) {
+                    physicalTables.add(famName);
+                } else {
+                    logger.warn("Unknown link type: " + colKv.getBuffer()[colKv.getValueOffset()] + " for " + SchemaUtil.getTableName(schemaName.getString(), tableName.getString()));
+                }
+            } else {
+                addColumnToTable(results, colName, famName, colKeyValues, columns);
+            }
+        }
+        
+        return PTableImpl.makePTable(schemaName, tableName, tableType, indexState, timeStamp, tableSeqNum, pkName, saltBucketNum, columns, tableType == INDEX ? dataTableName : null, 
+                indexes, isImmutableRows, physicalTables, defaultFamilyName, viewStatement, disableWAL, multiTenant, viewType);
+    }
+
+    private PTable buildDeletedTable(byte[] key, ImmutableBytesPtr cacheKey, HRegion region, long clientTimeStamp) throws IOException {
+        if (clientTimeStamp == HConstants.LATEST_TIMESTAMP) {
+            return null;
+        }
+        
+        Scan scan = newTableRowsScan(key, clientTimeStamp, HConstants.LATEST_TIMESTAMP);
+        scan.setFilter(new FirstKeyOnlyFilter());
+        scan.setRaw(true);
+        RegionScanner scanner = region.getScanner(scan);
+        List<KeyValue> results = Lists.<KeyValue>newArrayList();
+        scanner.next(results);
+        // HBase ignores the time range on a raw scan (HBASE-7362)
+        if (!results.isEmpty() && results.get(0).getTimestamp() > clientTimeStamp) {
+            KeyValue kv = results.get(0);
+            if (kv.isDelete()) {
+                Map<ImmutableBytesPtr,PTable> metaDataCache = GlobalCache.getInstance(this.getEnvironment()).getMetaDataCache();
+                PTable table = newDeletedTableMarker(kv.getTimestamp());
+                metaDataCache.put(cacheKey, table);
+                return table;
+            }
+        }
+        return null;
+    }
+
+    private static PTable newDeletedTableMarker(long timestamp) {
+        return new PTableImpl(timestamp);
+    }
+
+    private static boolean isTableDeleted(PTable table) {
+        return table.getName() == null;
+    }
+
+    private PTable loadTable(RegionCoprocessorEnvironment env, byte[] key, ImmutableBytesPtr cacheKey, long clientTimeStamp, long asOfTimeStamp) throws IOException, SQLException {
+        HRegion region = env.getRegion();
+        Map<ImmutableBytesPtr,PTable> metaDataCache = GlobalCache.getInstance(this.getEnvironment()).getMetaDataCache();
+        PTable table = metaDataCache.get(cacheKey);
+        // We always cache the latest version - fault in if not in cache
+        if (table != null || (table = buildTable(key, cacheKey, region, asOfTimeStamp)) != null) {
+            return table;
+        }
+        // if not found then check if newer table already exists and add delete marker for timestamp found
+        if (table == null && (table=buildDeletedTable(key, cacheKey, region, clientTimeStamp)) != null) {
+            return table;
+        }
+        return null;
+    }
+    
+    
+    @Override
+    public MetaDataMutationResult createTable(List<Mutation> tableMetadata) throws IOException {
+        byte[][] rowKeyMetaData = new byte[3][];
+        MetaDataUtil.getTenantIdAndSchemaAndTableName(tableMetadata,rowKeyMetaData);
+        byte[] tenantIdBytes = rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
+        byte[] schemaName = rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
+        byte[] tableName = rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
+        
+        try {
+            byte[] parentTableName = MetaDataUtil.getParentTableName(tableMetadata);
+            byte[] lockTableName = parentTableName == null ? tableName : parentTableName;
+            byte[] lockKey = SchemaUtil.getTableKey(tenantIdBytes, schemaName, lockTableName);
+            byte[] key = parentTableName == null ? lockKey : SchemaUtil.getTableKey(tenantIdBytes, schemaName, tableName);
+            byte[] parentKey = parentTableName == null ? null : lockKey;
+            
+            RegionCoprocessorEnvironment env = getEnvironment();
+            HRegion region = env.getRegion();
+            MetaDataMutationResult result = checkTableKeyInRegion(lockKey, region);
+            if (result != null) {
+                return result; 
+            }
+            List<Integer> lids = Lists.newArrayList(5);
+            long clientTimeStamp = MetaDataUtil.getClientTimeStamp(tableMetadata);
+            try {
+                acquireLock(region, lockKey, lids);
+                if (key != lockKey) {
+                    acquireLock(region, key, lids);
+                }
+                // Load parent table first
+                PTable parentTable = null;
+                ImmutableBytesPtr parentCacheKey = null;
+                if (parentKey != null) {
+                    parentCacheKey = new ImmutableBytesPtr(parentKey);
+                    parentTable = loadTable(env, parentKey, parentCacheKey, clientTimeStamp, clientTimeStamp);
+                    if (parentTable == null || isTableDeleted(parentTable)) {
+                        return new MetaDataMutationResult(MutationCode.PARENT_TABLE_NOT_FOUND, EnvironmentEdgeManager.currentTimeMillis(), parentTable);
+                    }
+                    // If parent table isn't at the expected sequence number, then return
+                    if (parentTable.getSequenceNumber() != MetaDataUtil.getParentSequenceNumber(tableMetadata)) {
+                        return new MetaDataMutationResult(MutationCode.CONCURRENT_TABLE_MUTATION, EnvironmentEdgeManager.currentTimeMillis(), parentTable);
+                    }
+                }
+                // Load child table next
+                ImmutableBytesPtr cacheKey = new ImmutableBytesPtr(key);
+                // Get as of latest timestamp so we can detect if we have a newer table that already exists
+                // without making an additional query
+                PTable table = loadTable(env, key, cacheKey, clientTimeStamp, HConstants.LATEST_TIMESTAMP);
+                if (table != null) {
+                    if (table.getTimeStamp() < clientTimeStamp) {
+                        // If the table is older than the client time stamp and it's deleted, continue
+                        if (!isTableDeleted(table)) {
+                            return new MetaDataMutationResult(MutationCode.TABLE_ALREADY_EXISTS, EnvironmentEdgeManager.currentTimeMillis(), table);
+                        }
+                    } else {
+                        return new MetaDataMutationResult(MutationCode.NEWER_TABLE_FOUND, EnvironmentEdgeManager.currentTimeMillis(), table);
+                    }
+                }
+                
+                // TODO: Switch this to HRegion#batchMutate when we want to support indexes on the system
+                // table. Basically, we get all the locks that we don't already hold for all the
+                // tableMetadata rows. This ensures we don't have deadlock situations (ensuring primary and
+                // then index table locks are held, in that order). For now, we just don't support indexing
+                // on the system table. This is an issue because of the way we manage batch mutation in the
+                // Indexer.
+                region.mutateRowsWithLocks(tableMetadata, Collections.<byte[]>emptySet());
+                
+                // Invalidate the cache - the next getTable call will add it
+                // TODO: consider loading the table that was just created here, patching up the parent table, and updating the cache
+                Map<ImmutableBytesPtr,PTable> metaDataCache = GlobalCache.getInstance(this.getEnvironment()).getMetaDataCache();
+                if (parentCacheKey != null) {
+                    metaDataCache.remove(parentCacheKey);
+                }
+                metaDataCache.remove(cacheKey);
+                // Get timeStamp from mutations - the above method sets it if it's unset
+                long currentTimeStamp = MetaDataUtil.getClientTimeStamp(tableMetadata);
+                return new MetaDataMutationResult(MutationCode.TABLE_NOT_FOUND, currentTimeStamp, null);
+            } finally {
+                releaseLocks(region, lids);
+            }
+        } catch (Throwable t) {
+            ServerUtil.throwIOException(SchemaUtil.getTableName(schemaName, tableName), t);
+            return null; // impossible
+        }
+    }
+
+    private static void acquireLock(HRegion region, byte[] key, List<Integer> lids) throws IOException {
+        Integer lid = region.getLock(null, key, true);
+        if (lid == null) {
+            throw new IOException("Failed to acquire lock on " + Bytes.toStringBinary(key));
+        }
+        lids.add(lid);
+    }
+    
+    private static void releaseLocks(HRegion region, List<Integer> lids) {
+        for (Integer lid : lids) {
+            region.releaseRowLock(lid);
+        }
+    }
+    
+    private static final byte[] PHYSICAL_TABLE_BYTES = new byte[] {PTable.LinkType.PHYSICAL_TABLE.getSerializedValue()};
+    /**
+     * @param tableName parent table's name
+     * @return true if there exist a table that use this table as their base table.
+     * TODO: should we pass a timestamp here?
+     */
+    private boolean hasViews(HRegion region, byte[] tenantId, PTable table) throws IOException {
+        byte[] schemaName = table.getSchemaName().getBytes();
+        byte[] tableName = table.getTableName().getBytes();
+        Scan scan = new Scan();
+        // If the table is multi-tenant, we need to check across all tenant_ids,
+        // so we can't constrain the row key. Otherwise, any views would have
+        // the same tenantId.
+        if (!table.isMultiTenant()) {
+            byte[] startRow = ByteUtil.concat(tenantId, QueryConstants.SEPARATOR_BYTE_ARRAY);
+            byte[] stopRow = ByteUtil.nextKey(startRow);
+            scan.setStartRow(startRow);
+            scan.setStopRow(stopRow);
+        }
+        SingleColumnValueFilter linkFilter = new SingleColumnValueFilter(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES, EQUAL, PHYSICAL_TABLE_BYTES);
+        linkFilter.setFilterIfMissing(true);
+        byte[] suffix = ByteUtil.concat(QueryConstants.SEPARATOR_BYTE_ARRAY, SchemaUtil.getTableNameAsBytes(schemaName, tableName));
+        SuffixFilter rowFilter = new SuffixFilter(suffix);
+        Filter filter = new FilterList(linkFilter, rowFilter);
+        scan.setFilter(filter);
+        scan.addColumn(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES);
+        RegionScanner scanner = region.getScanner(scan);
+        try {
+            List<KeyValue> results = newArrayList();
+            scanner.next(results);
+            return results.size() > 0;
+        }
+        finally {
+            scanner.close();
+        }
+    }
+    
+    @Override
+    public MetaDataMutationResult dropTable(List<Mutation> tableMetadata, String tableType) throws IOException {
+        byte[][] rowKeyMetaData = new byte[3][];
+        MetaDataUtil.getTenantIdAndSchemaAndTableName(tableMetadata,rowKeyMetaData);
+        byte[] tenantIdBytes = rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
+        byte[] schemaName = rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
+        byte[] tableName = rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
+        // Disallow deletion of a system table
+        if (tableType.equals(PTableType.SYSTEM.getSerializedValue())) {
+            return new MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION, EnvironmentEdgeManager.currentTimeMillis(), null);
+        }
+        List<byte[]> tableNamesToDelete = Lists.newArrayList();
+        try {
+            byte[] parentTableName = MetaDataUtil.getParentTableName(tableMetadata);
+            byte[] lockTableName = parentTableName == null ? tableName : parentTableName;
+            byte[] lockKey = SchemaUtil.getTableKey(tenantIdBytes, schemaName, lockTableName);
+            byte[] key = parentTableName == null ? lockKey : SchemaUtil.getTableKey(tenantIdBytes, schemaName, tableName);
+            
+            RegionCoprocessorEnvironment env = getEnvironment();
+            HRegion region = env.getRegion();
+            MetaDataMutationResult result = checkTableKeyInRegion(key, region);
+            if (result != null) {
+                return result; 
+            }
+            List<Integer> lids = Lists.newArrayList(5);
+            try {
+                acquireLock(region, lockKey, lids);
+                if (key != lockKey) {
+                    acquireLock(region, key, lids);
+                }
+                List<ImmutableBytesPtr> invalidateList = new ArrayList<ImmutableBytesPtr>();
+                result = doDropTable(key, tenantIdBytes, schemaName, tableName, PTableType.fromSerializedValue(tableType), tableMetadata, invalidateList, lids, tableNamesToDelete);
+                if (result.getMutationCode() != MutationCode.TABLE_ALREADY_EXISTS || result.getTable() == null) {
+                    return result;
+                }
+                Map<ImmutableBytesPtr,PTable> metaDataCache = GlobalCache.getInstance(this.getEnvironment()).getMetaDataCache();
+                // Commit the list of deletion.
+                region.mutateRowsWithLocks(tableMetadata, Collections.<byte[]>emptySet());
+                long currentTime = MetaDataUtil.getClientTimeStamp(tableMetadata);
+                for (ImmutableBytesPtr ckey: invalidateList) {
+                    metaDataCache.put(ckey, newDeletedTableMarker(currentTime));
+                }
+                if (parentTableName != null) {
+                    ImmutableBytesPtr parentCacheKey = new ImmutableBytesPtr(lockKey);
+                    metaDataCache.remove(parentCacheKey);
+                }
+                return result;
+            } finally {
+                releaseLocks(region, lids);
+            }
+        } catch (Throwable t) {
+            ServerUtil.throwIOException(SchemaUtil.getTableName(schemaName, tableName), t);
+            return null; // impossible
+        }
+    }
+
+    private MetaDataMutationResult doDropTable(byte[] key, byte[] tenantId, byte[] schemaName, byte[] tableName, PTableType tableType, 
+            List<Mutation> rowsToDelete, List<ImmutableBytesPtr> invalidateList, List<Integer> lids, List<byte[]> tableNamesToDelete) throws IOException, SQLException {
+        long clientTimeStamp = MetaDataUtil.getClientTimeStamp(rowsToDelete);
+
+        RegionCoprocessorEnvironment env = getEnvironment();
+        HRegion region = env.getRegion();
+        ImmutableBytesPtr cacheKey = new ImmutableBytesPtr(key);
+        
+        Map<ImmutableBytesPtr,PTable> metaDataCache = GlobalCache.getInstance(this.getEnvironment()).getMetaDataCache();
+        PTable table = metaDataCache.get(cacheKey);
+        
+        // We always cache the latest version - fault in if not in cache
+        if (table != null || (table = buildTable(key, cacheKey, region, HConstants.LATEST_TIMESTAMP)) != null) {
+            if (table.getTimeStamp() < clientTimeStamp) {
+                // If the table is older than the client time stamp and its deleted, continue
+                if (isTableDeleted(table)) {
+                    return new MetaDataMutationResult(MutationCode.TABLE_ALREADY_EXISTS, EnvironmentEdgeManager.currentTimeMillis(), null);
+                }
+                if ( tableType != table.getType())  {
+                    // We said to drop a table, but found a view or visa versa
+                    return new MetaDataMutationResult(MutationCode.TABLE_NOT_FOUND, EnvironmentEdgeManager.currentTimeMillis(), null);
+                }
+            } else {
+                return new MetaDataMutationResult(MutationCode.NEWER_TABLE_FOUND, EnvironmentEdgeManager.currentTimeMillis(), table);
+            }
+        }
+        if (table == null && buildDeletedTable(key, cacheKey, region, clientTimeStamp) != null) {
+            return new MetaDataMutationResult(MutationCode.NEWER_TABLE_FOUND, EnvironmentEdgeManager.currentTimeMillis(), null);
+        }
+        // Get mutations for main table.
+        Scan scan = newTableRowsScan(key, MIN_TABLE_TIMESTAMP, clientTimeStamp);
+        RegionScanner scanner = region.getScanner(scan);
+        List<KeyValue> results = Lists.newArrayList();
+        scanner.next(results);
+        if (results.isEmpty()) {
+            return new MetaDataMutationResult(MutationCode.TABLE_NOT_FOUND, EnvironmentEdgeManager.currentTimeMillis(), null);
+        }
+        KeyValue typeKeyValue = KeyValueUtil.getColumnLatest(results, PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES, PhoenixDatabaseMetaData.TABLE_TYPE_BYTES);
+        assert(typeKeyValue != null && typeKeyValue.getValueLength() == 1);
+        if ( tableType != PTableType.fromSerializedValue(typeKeyValue.getBuffer()[typeKeyValue.getValueOffset()]))  {
+            // We said to drop a table, but found a view or visa versa
+            return new MetaDataMutationResult(MutationCode.TABLE_NOT_FOUND, EnvironmentEdgeManager.currentTimeMillis(), null);
+        }
+        // Don't allow a table with views to be deleted
+        // TODO: support CASCADE with DROP
+        if (tableType == PTableType.TABLE && hasViews(region, tenantId, table)) {
+            return new MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION, EnvironmentEdgeManager.currentTimeMillis(), null);
+        }
+        if (table.getType() != PTableType.VIEW) { // Add to list of HTables to delete, unless it's a view
+            tableNamesToDelete.add(table.getName().getBytes());
+        }
+        List<byte[]> indexNames = Lists.newArrayList();
+        invalidateList.add(cacheKey);
+        byte[][] rowKeyMetaData = new byte[5][];
+        byte[] rowKey;
+        do {
+            KeyValue kv = results.get(LINK_TYPE_INDEX);
+            rowKey = kv.getRow();
+            int nColumns = getVarChars(rowKey, rowKeyMetaData);
+            if (nColumns == 5 && rowKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX].length == 0 && rowKeyMetaData[PhoenixDatabaseMetaData.INDEX_NAME_INDEX].length > 0
+                    && Bytes.compareTo(kv.getBuffer(), kv.getQualifierOffset(), kv.getQualifierLength(), LINK_TYPE_BYTES, 0, LINK_TYPE_BYTES.length) == 0
+                    && LinkType.fromSerializedValue(kv.getBuffer()[kv.getValueOffset()]) == LinkType.INDEX_TABLE) {
+                indexNames.add(rowKeyMetaData[PhoenixDatabaseMetaData.INDEX_NAME_INDEX]);
+            }
+            @SuppressWarnings("deprecation") // FIXME: Remove when unintentionally deprecated method is fixed (HBASE-7870).
+            // FIXME: the version of the Delete constructor without the lock args was introduced
+            // in 0.94.4, thus if we try to use it here we can no longer use the 0.94.2 version
+            // of the client.
+            Delete delete = new Delete(rowKey, clientTimeStamp, null);
+            rowsToDelete.add(delete);
+            results.clear();
+            scanner.next(results);
+        } while (!results.isEmpty());
+        
+        // Recursively delete indexes
+        for (byte[] indexName : indexNames) {
+            byte[] indexKey = SchemaUtil.getTableKey(tenantId, schemaName, indexName);
+            @SuppressWarnings("deprecation") // FIXME: Remove when unintentionally deprecated method is fixed (HBASE-7870).
+            // FIXME: the version of the Delete constructor without the lock args was introduced
+            // in 0.94.4, thus if we try to use it here we can no longer use the 0.94.2 version
+            // of the client.
+            Delete delete = new Delete(indexKey, clientTimeStamp, null);
+            rowsToDelete.add(delete);
+            acquireLock(region, indexKey, lids);
+            MetaDataMutationResult result = doDropTable(indexKey, tenantId, schemaName, indexName, PTableType.INDEX, rowsToDelete, invalidateList, lids, tableNamesToDelete);
+            if (result.getMutationCode() != MutationCode.TABLE_ALREADY_EXISTS || result.getTable() == null) {
+                return result;
+            }
+        }
+        
+        return new MetaDataMutationResult(MutationCode.TABLE_ALREADY_EXISTS, EnvironmentEdgeManager.currentTimeMillis(), table, tableNamesToDelete);
+    }
+
+    private static interface ColumnMutator {
+        MetaDataMutationResult updateMutation(PTable table, byte[][] rowKeyMetaData, List<Mutation> tableMetadata, HRegion region, List<ImmutableBytesPtr> invalidateList, List<Integer> lids) throws IOException, SQLException;
+    }
+
+    private MetaDataMutationResult mutateColumn(List<Mutation> tableMetadata, ColumnMutator mutator) throws IOException {
+        byte[][] rowKeyMetaData = new byte[5][];
+        MetaDataUtil.getTenantIdAndSchemaAndTableName(tableMetadata,rowKeyMetaData);
+        byte[] tenantId = rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
+        byte[] schemaName = rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
+        byte[] tableName = rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
+        try {
+            RegionCoprocessorEnvironment env = getEnvironment();
+            byte[] key = SchemaUtil.getTableKey(tenantId, schemaName, tableName);
+            HRegion region = env.getRegion();
+            MetaDataMutationResult result = checkTableKeyInRegion(key, region);
+            if (result != null) {
+                return result; 
+            }
+            List<Integer> lids = Lists.newArrayList(5);
+            try {
+                acquireLock(region, key, lids);
+                ImmutableBytesPtr cacheKey = new ImmutableBytesPtr(key);
+                List<ImmutableBytesPtr> invalidateList = new ArrayList<ImmutableBytesPtr>();
+                invalidateList.add(cacheKey);
+                Map<ImmutableBytesPtr,PTable> metaDataCache = GlobalCache.getInstance(this.getEnvironment()).getMetaDataCache();
+                PTable table = metaDataCache.get(cacheKey);
+                if (logger.isDebugEnabled()) {
+                    if (table == null) {
+                        logger.debug("Table " + Bytes.toStringBinary(key) + " not found in cache. Will build through scan");
+                    } else {
+                        logger.debug("Table " + Bytes.toStringBinary(key) + " found in cache with timestamp " + table.getTimeStamp() + " seqNum " + table.getSequenceNumber());
+                    }
+                }
+                // Get client timeStamp from mutations
+                long clientTimeStamp = MetaDataUtil.getClientTimeStamp(tableMetadata);
+                if (table == null && (table = buildTable(key, cacheKey, region, HConstants.LATEST_TIMESTAMP)) == null) {
+                    // if not found then call newerTableExists and add delete marker for timestamp found
+                    if (buildDeletedTable(key, cacheKey, region, clientTimeStamp) != null) {
+                        return new MetaDataMutationResult(MutationCode.NEWER_TABLE_FOUND, EnvironmentEdgeManager.currentTimeMillis(), null);
+                    }
+                    return new MetaDataMutationResult(MutationCode.TABLE_NOT_FOUND, EnvironmentEdgeManager.currentTimeMillis(), null);
+                }
+                if (table.getTimeStamp() >= clientTimeStamp) {
+                    return new MetaDataMutationResult(MutationCode.NEWER_TABLE_FOUND, EnvironmentEdgeManager.currentTimeMillis(), table);
+                } else if (isTableDeleted(table)) {
+                    return new MetaDataMutationResult(MutationCode.TABLE_NOT_FOUND, EnvironmentEdgeManager.currentTimeMillis(), null);
+                }
+                    
+                long expectedSeqNum = MetaDataUtil.getSequenceNumber(tableMetadata) - 1; // lookup TABLE_SEQ_NUM in tableMetaData
+                if (logger.isDebugEnabled()) {
+                    logger.debug("For table " + Bytes.toStringBinary(key) + " expecting seqNum " + expectedSeqNum + " and found seqNum " + table.getSequenceNumber() + " with " + table.getColumns().size() + " columns: " + table.getColumns());
+                }
+                if (expectedSeqNum != table.getSequenceNumber()) {
+                    if (logger.isDebugEnabled()) {
+                        logger.debug("For table " + Bytes.toStringBinary(key) + " returning CONCURRENT_TABLE_MUTATION due to unexpected seqNum");
+                    }
+                    return new MetaDataMutationResult(MutationCode.CONCURRENT_TABLE_MUTATION, EnvironmentEdgeManager.currentTimeMillis(), table);
+                }
+                
+                PTableType type = table.getType();
+                if (type == PTableType.INDEX) { 
+                    // Disallow mutation of an index table
+                    return new MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION, EnvironmentEdgeManager.currentTimeMillis(), null);
+                } else {
+                    PTableType expectedType = MetaDataUtil.getTableType(tableMetadata);
+                    // We said to drop a table, but found a view or visa versa
+                    if (type != expectedType) {
+                        return new MetaDataMutationResult(MutationCode.TABLE_NOT_FOUND, EnvironmentEdgeManager.currentTimeMillis(), null);
+                    }
+                    if (hasViews(region, tenantId, table)) {
+                        // Disallow any column mutations for parents of tenant tables
+                        return new MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION, EnvironmentEdgeManager.currentTimeMillis(), null);
+                    }
+                }
+                result = mutator.updateMutation(table, rowKeyMetaData, tableMetadata, region, invalidateList, lids);
+                if (result != null) {
+                    return result;
+                }
+                
+                region.mutateRowsWithLocks(tableMetadata, Collections.<byte[]>emptySet());
+                // Invalidate from cache
+                for (ImmutableBytesPtr invalidateKey : invalidateList) {
+                    PTable invalidatedTable = metaDataCache.remove(invalidateKey);
+                    if (logger.isDebugEnabled()) {
+                        if (invalidatedTable == null) {
+                            logger.debug("Attempted to invalidated table key " + Bytes.toStringBinary(cacheKey.get(),cacheKey.getOffset(),cacheKey.getLength()) + " but found no cached table");
+                        } else {
+                            logger.debug("Invalidated table key " + Bytes.toStringBinary(cacheKey.get(),cacheKey.getOffset(),cacheKey.getLength()) + " with timestamp " + invalidatedTable.getTimeStamp() + " and seqNum " + invalidatedTable.getSequenceNumber());
+                        }
+                    }
+                }
+                // Get client timeStamp from mutations, since it may get updated by the mutateRowsWithLocks call
+                long currentTime = MetaDataUtil.getClientTimeStamp(tableMetadata);
+                return new MetaDataMutationResult(MutationCode.TABLE_ALREADY_EXISTS, currentTime, null);
+            } finally {
+                releaseLocks(region,lids);
+            }
+        } catch (Throwable t) {
+            ServerUtil.throwIOException(SchemaUtil.getTableName(schemaName, tableName), t);
+            return null; // impossible
+        }
+    }
+
+    @Override
+    public MetaDataMutationResult addColumn(List<Mutation> tableMetaData) throws IOException {
+        return mutateColumn(tableMetaData, new ColumnMutator() {
+            @Override
+            public MetaDataMutationResult updateMutation(PTable table, byte[][] rowKeyMetaData, List<Mutation> tableMetaData, HRegion region, List<ImmutableBytesPtr> invalidateList, List<Integer> lids) {
+                byte[] tenantId = rowKeyMetaData[TENANT_ID_INDEX];
+                byte[] schemaName = rowKeyMetaData[SCHEMA_NAME_INDEX];
+                byte[] tableName = rowKeyMetaData[TABLE_NAME_INDEX];
+                for (Mutation m : tableMetaData) {
+                    byte[] key = m.getRow();
+                    boolean addingPKColumn = false;
+                    int pkCount = getVarChars(key, rowKeyMetaData);
+                    if (pkCount > COLUMN_NAME_INDEX 
+                            && Bytes.compareTo(schemaName, rowKeyMetaData[SCHEMA_NAME_INDEX]) == 0 
+                            && Bytes.compareTo(tableName, rowKeyMetaData[TABLE_NAME_INDEX]) == 0 ) {
+                        try {
+                            if (pkCount > FAMILY_NAME_INDEX && rowKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX].length > 0) {
+                                PColumnFamily family = table.getColumnFamily(rowKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX]);
+                                family.getColumn(rowKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX]);
+                            } else if (pkCount > COLUMN_NAME_INDEX && rowKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX].length > 0) {
+                                addingPKColumn = true;
+                                table.getPKColumn(new String(rowKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX]));
+                            } else {
+                                continue;
+                            }
+                            return new MetaDataMutationResult(MutationCode.COLUMN_ALREADY_EXISTS, EnvironmentEdgeManager.currentTimeMillis(), table);
+                        } catch (ColumnFamilyNotFoundException e) {
+                            continue;
+                        } catch (ColumnNotFoundException e) {
+                            if (addingPKColumn) {
+                                // Add all indexes to invalidate list, as they will all be adding the same PK column
+                                // No need to lock them, as we have the parent table lock at this point
+                                for (PTable index : table.getIndexes()) {
+                                    invalidateList.add(new ImmutableBytesPtr(SchemaUtil.getTableKey(tenantId, index.getSchemaName().getBytes(),index.getTableName().getBytes())));
+                                }
+                            }
+                            continue;
+                        }
+                    }
+                }
+                return null;
+            }
+        });
+    }
+    
+    @Override
+    public MetaDataMutationResult dropColumn(List<Mutation> tableMetaData) throws IOException {
+        final long clientTimeStamp = MetaDataUtil.getClientTimeStamp(tableMetaData);
+        final List<byte[]> tableNamesToDelete = Lists.newArrayList();
+        return mutateColumn(tableMetaData, new ColumnMutator() {
+            @SuppressWarnings("deprecation")
+            @Override
+            public MetaDataMutationResult updateMutation(PTable table, byte[][] rowKeyMetaData, List<Mutation> tableMetaData, HRegion region, List<ImmutableBytesPtr> invalidateList, List<Integer> lids) throws IOException, SQLException {
+                byte[] tenantId = rowKeyMetaData[TENANT_ID_INDEX];
+                byte[] schemaName = rowKeyMetaData[SCHEMA_NAME_INDEX];
+                byte[] tableName = rowKeyMetaData[TABLE_NAME_INDEX];
+                boolean deletePKColumn = false;
+                List<Mutation> additionalTableMetaData = Lists.newArrayList();
+                for (Mutation m : tableMetaData) {
+                    if (m instanceof Delete) {
+                        byte[] key = m.getRow();
+                        int pkCount = getVarChars(key, rowKeyMetaData);
+                        if (pkCount > COLUMN_NAME_INDEX 
+                                && Bytes.compareTo(schemaName, rowKeyMetaData[SCHEMA_NAME_INDEX]) == 0 
+                                && Bytes.compareTo(tableName, rowKeyMetaData[TABLE_NAME_INDEX]) == 0 ) {
+                            PColumn columnToDelete = null;
+                            try {
+                                if (pkCount > FAMILY_NAME_INDEX && rowKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX].length > 0) {
+                                    PColumnFamily family = table.getColumnFamily(rowKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX]);
+                                    columnToDelete = family.getColumn(rowKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX]);
+                                } else if (pkCount > COLUMN_NAME_INDEX && rowKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX].length > 0) {
+                                    deletePKColumn = true;
+                                    columnToDelete = table.getPKColumn(new String(rowKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX]));
+                                } else {
+                                    continue;
+                                }
+                                // Look for columnToDelete in any indexes. If found as PK column, get lock and drop the index. If found as covered column, delete from index (do this client side?).
+                                // In either case, invalidate index if the column is in it
+                                for (PTable index : table.getIndexes()) {
+                                    try {
+                                        String indexColumnName = IndexUtil.getIndexColumnName(columnToDelete);
+                                        PColumn indexColumn = index.getColumn(indexColumnName);
+                                        byte[] indexKey = SchemaUtil.getTableKey(tenantId, index.getSchemaName().getBytes(), index.getTableName().getBytes());
+                                        // If index contains the column in it's PK, then drop it
+                                        if (SchemaUtil.isPKColumn(indexColumn)) {
+                                            // Since we're dropping the index, lock it to ensure that a change in index state doesn't
+                                            // occur while we're dropping it.
+                                            acquireLock(region, indexKey, lids);
+                                            // Drop the index table. The doDropTable will expand this to all of the table rows and invalidate the index table
+                                            additionalTableMetaData.add(new Delete(indexKey, clientTimeStamp, null));
+                                            byte[] linkKey = MetaDataUtil.getParentLinkKey(tenantId, schemaName, tableName, index.getTableName().getBytes());
+                                            // Drop the link between the data table and the index table
+                                            additionalTableMetaData.add(new Delete(linkKey, clientTimeStamp, null));
+                                            doDropTable(indexKey, tenantId, index.getSchemaName().getBytes(), index.getTableName().getBytes(), index.getType(), additionalTableMetaData, invalidateList, lids, tableNamesToDelete);
+                                            // TODO: return in result?
+                                        } else {
+                                            invalidateList.add(new ImmutableBytesPtr(indexKey));
+                                        }
+                                    } catch (ColumnNotFoundException e) {
+                                    } catch (AmbiguousColumnException e) {
+                                    }
+                                }
+                            } catch (ColumnFamilyNotFoundException e) {
+                                return new MetaDataMutationResult(MutationCode.COLUMN_NOT_FOUND, EnvironmentEdgeManager.currentTimeMillis(), table, columnToDelete);
+                            } catch (ColumnNotFoundException e) {
+                                return new MetaDataMutationResult(MutationCode.COLUMN_NOT_FOUND, EnvironmentEdgeManager.currentTimeMillis(), table, columnToDelete);
+                            }
+                        }
+                    }
+                }
+                if (deletePKColumn) {
+                    if (table.getPKColumns().size() == 1) {
+                        return new MetaDataMutationResult(MutationCode.NO_PK_COLUMNS, EnvironmentEdgeManager.currentTimeMillis(), null);
+                    }
+                }
+                tableMetaData.addAll(additionalTableMetaData);
+                return null;
+            }
+        });
+        
+    }
+    
+    private static MetaDataMutationResult checkTableKeyInRegion(byte[] key, HRegion region) {
+        byte[] startKey = region.getStartKey();
+        byte[] endKey = region.getEndKey();
+        if (Bytes.compareTo(startKey, key) <= 0 && (Bytes.compareTo(HConstants.LAST_ROW, endKey) == 0 || Bytes.compareTo(key, endKey) < 0)) {
+            return null; // normal case;
+        }
+        return new MetaDataMutationResult(MutationCode.TABLE_NOT_IN_REGION, EnvironmentEdgeManager.currentTimeMillis(), null);
+    }
+
+    @Override
+    public MetaDataMutationResult getTable(byte[] tenantId, byte[] schemaName, byte[] tableName, long tableTimeStamp, long clientTimeStamp) throws IOException {
+        try {
+            byte[] key = SchemaUtil.getTableKey(tenantId, schemaName, tableName);
+            
+            // get the co-processor environment
+            RegionCoprocessorEnvironment env = getEnvironment();
+            // TODO: check that key is within region.getStartKey() and region.getEndKey()
+            // and return special code to force client to lookup region from meta.
+            HRegion region = env.getRegion();
+            MetaDataMutationResult result = checkTableKeyInRegion(key, region);
+            if (result != null) {
+                return result; 
+            }
+            
+            long currentTime = EnvironmentEdgeManager.currentTimeMillis();
+            PTable table = doGetTable(key, clientTimeStamp);
+            if (table == null) {
+                return new MetaDataMutationResult(MutationCode.TABLE_NOT_FOUND, currentTime, null);
+            }
+            return new MetaDataMutationResult(MutationCode.TABLE_ALREADY_EXISTS, currentTime, table.getTimeStamp() != tableTimeStamp ? table : null);
+        } catch (Throwable t) {
+            ServerUtil.throwIOException(SchemaUtil.getTableName(schemaName, tableName), t);
+            return null; // impossible
+        }
+    }
+
+    private PTable doGetTable(byte[] key, long clientTimeStamp) throws IOException, SQLException {
+        ImmutableBytesPtr cacheKey = new ImmutableBytesPtr(key);
+        Map<ImmutableBytesPtr,PTable> metaDataCache = GlobalCache.getInstance(this.getEnvironment()).getMetaDataCache();
+        PTable table = metaDataCache.get(cacheKey);
+        // We only cache the latest, so we'll end up building the table with every call if the client connection has specified an SCN.
+        // TODO: If we indicate to the client that we're returning an older version, but there's a newer version available, the client
+        // can safely not call this, since we only allow modifications to the latest.
+        if (table != null && table.getTimeStamp() < clientTimeStamp) {
+            // Table on client is up-to-date with table on server, so just return
+            if (isTableDeleted(table)) {
+                return null;
+            }
+            return table;
+        }
+        // Ask Lars about the expense of this call - if we don't take the lock, we still won't get partial results
+        // get the co-processor environment
+        RegionCoprocessorEnvironment env = getEnvironment();
+        // TODO: check that key is within region.getStartKey() and region.getEndKey()
+        // and return special code to force client to lookup region from meta.
+        HRegion region = env.getRegion();
+        /*
+         * Lock directly on key, though it may be an index table.
+         * This will just prevent a table from getting rebuilt
+         * too often.
+         */
+        Integer lid = region.getLock(null, key, true);
+        if (lid == null) {
+            throw new IOException("Failed to acquire lock on " + Bytes.toStringBinary(key));
+        }
+        try {
+            // Try cache again in case we were waiting on a lock
+            table = metaDataCache.get(cacheKey);
+            // We only cache the latest, so we'll end up building the table with every call if the client connection has specified an SCN.
+            // TODO: If we indicate to the client that we're returning an older version, but there's a newer version available, the client
+            // can safely not call this, since we only allow modifications to the latest.
+            if (table != null && table.getTimeStamp() < clientTimeStamp) {
+                // Table on client is up-to-date with table on server, so just return
+                if (isTableDeleted(table)) {
+                    return null;
+                }
+                return table;
+            }
+            // Query for the latest table first, since it's not cached
+            table = buildTable(key, cacheKey, region, HConstants.LATEST_TIMESTAMP);
+            if (table != null && table.getTimeStamp() < clientTimeStamp) {
+                return table;
+            }
+            // Otherwise, query for an older version of the table - it won't be cached 
+            return buildTable(key, cacheKey, region, clientTimeStamp);
+        } finally {
+            if (lid != null) region.releaseRowLock(lid);
+        }
+    }
+
+    @Override
+    public void clearCache() {
+        Map<ImmutableBytesPtr,PTable> metaDataCache = GlobalCache.getInstance(this.getEnvironment()).getMetaDataCache();
+        metaDataCache.clear();
+    }
+
+    @Override
+    public long getVersion() {
+        // The first 3 bytes of the long is used to encoding the HBase version as major.minor.patch.
+        // The next 4 bytes of the value is used to encode the Phoenix version as major.minor.patch.
+        long version = MetaDataUtil.encodeHBaseAndPhoenixVersions(this.getEnvironment().getHBaseVersion());
+        
+        // The last byte is used to communicate whether or not mutable secondary indexing
+        // was configured properly.
+        RegionCoprocessorEnvironment env = getEnvironment();
+        version = MetaDataUtil.encodeMutableIndexConfiguredProperly(
+                version, 
+                IndexManagementUtil.isWALEditCodecSet(env.getConfiguration()));
+        return version;
+    }
+
+    @Override
+    public MetaDataMutationResult updateIndexState(List<Mutation> tableMetadata) throws IOException {
+        byte[][] rowKeyMetaData = new byte[3][];
+        MetaDataUtil.getTenantIdAndSchemaAndTableName(tableMetadata,rowKeyMetaData);
+        byte[] tenantId = rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
+        byte[] schemaName = rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
+        byte[] tableName = rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
+        try {
+            RegionCoprocessorEnvironment env = getEnvironment();
+            byte[] key = SchemaUtil.getTableKey(tenantId, schemaName, tableName);
+            HRegion region = env.getRegion();
+            MetaDataMutationResult result = checkTableKeyInRegion(key, region);
+            if (result != null) {
+                return result; 
+            }
+            long timeStamp = MetaDataUtil.getClientTimeStamp(tableMetadata);
+            ImmutableBytesPtr cacheKey = new ImmutableBytesPtr(key);
+            List<KeyValue> newKVs = tableMetadata.get(0).getFamilyMap().get(TABLE_FAMILY_BYTES);
+            KeyValue newKV = newKVs.get(0);
+            PIndexState newState =  PIndexState.fromSerializedValue(newKV.getBuffer()[newKV.getValueOffset()]);
+            Integer lid = region.getLock(null, key, true);
+            if (lid == null) {
+                throw new IOException("Failed to acquire lock on " + Bytes.toStringBinary(key));
+            }
+            try {
+                Get get = new Get(key);
+                get.setTimeRange(PTable.INITIAL_SEQ_NUM, timeStamp);
+                get.addColumn(TABLE_FAMILY_BYTES, INDEX_STATE_BYTES);
+                Result currentResult = region.get(get);
+                if (currentResult.raw().length == 0) {
+                    return new MetaDataMutationResult(MutationCode.TABLE_NOT_FOUND, EnvironmentEdgeManager.currentTimeMillis(), null);
+                }
+                KeyValue currentStateKV = currentResult.raw()[0];
+                PIndexState currentState = PIndexState.fromSerializedValue(currentStateKV.getBuffer()[currentStateKV.getValueOffset()]);
+                // Detect invalid transitions
+                if (currentState == PIndexState.BUILDING) {
+                    if (newState == PIndexState.USABLE) {
+                        return new MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION, EnvironmentEdgeManager.currentTimeMillis(), null);
+                    }
+                } else if (currentState == PIndexState.DISABLE) {
+                    if (newState != PIndexState.BUILDING && newState != PIndexState.DISABLE) {
+                        return new MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION, EnvironmentEdgeManager.currentTimeMillis(), null);
+                    }
+                    // Done building, but was disable before that, so that in disabled state
+                    if (newState == PIndexState.ACTIVE) {
+                        newState = PIndexState.DISABLE;
+                    }
+                }
+
+                if (currentState == PIndexState.BUILDING && newState != PIndexState.ACTIVE) {
+                    timeStamp = currentStateKV.getTimestamp();
+                }
+                if ((currentState == PIndexState.UNUSABLE && newState == PIndexState.ACTIVE) || (currentState == PIndexState.ACTIVE && newState == PIndexState.UNUSABLE)) {
+                    newState = PIndexState.INACTIVE;
+                    newKVs.set(0, KeyValueUtil.newKeyValue(key, TABLE_FAMILY_BYTES, INDEX_STATE_BYTES, timeStamp, Bytes.toBytes(newState.getSerializedValue())));
+                } else if (currentState == PIndexState.INACTIVE && newState == PIndexState.USABLE) {
+                    newState = PIndexState.ACTIVE;
+                    newKVs.set(0, KeyValueUtil.newKeyValue(key, TABLE_FAMILY_BYTES, INDEX_STATE_BYTES, timeStamp, Bytes.toBytes(newState.getSerializedValue())));
+                }
+                if (currentState != newState) {
+                    region.mutateRowsWithLocks(tableMetadata, Collections.<byte[]>emptySet());
+                    // Invalidate from cache
+                    Map<ImmutableBytesPtr,PTable> metaDataCache = GlobalCache.getInstance(this.getEnvironment()).getMetaDataCache();
+                    metaDataCache.remove(cacheKey);
+                }
+                // Get client timeStamp from mutations, since it may get updated by the mutateRowsWithLocks call
+                long currentTime = MetaDataUtil.getClientTimeStamp(tableMetadata);
+                return new MetaDataMutationResult(MutationCode.TABLE_ALREADY_EXISTS, currentTime, null);
+            } finally {
+                region.releaseRowLock(lid);
+            }
+        } catch (Throwable t) {
+            ServerUtil.throwIOException(SchemaUtil.getTableName(schemaName, tableName), t);
+            return null; // impossible
+        }
+    }
+    
+    /**
+     * 
+     * Matches rows that end with a given byte array suffix
+     *
+     * @author jtaylor
+     * @since 3.0
+     */
+    private static class SuffixFilter extends FilterBase {
+        protected byte[] suffix = null;
+
+        public SuffixFilter(final byte[] suffix) {
+            this.suffix = suffix;
+        }
+
+        @Override
+        public boolean filterRowKey(byte[] buffer, int offset, int length) {
+            if (buffer == null || this.suffix == null) return true;
+            if (length < suffix.length) return true;
+            // if they are equal, return false => pass row
+            // else return true, filter row
+            // if we are passed the suffix, set flag
+            int cmp = Bytes.compareTo(buffer, offset + (length - this.suffix.length),
+                    this.suffix.length, this.suffix, 0, this.suffix.length);
+            return cmp != 0;
+        }
+
+        @Override
+        public void readFields(DataInput arg0) throws IOException {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public void write(DataOutput arg0) throws IOException {
+            throw new UnsupportedOperationException();
+        }
+    }
+}


[37/51] [partial] Initial commit of master branch from github

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereOptimizer.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereOptimizer.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereOptimizer.java
new file mode 100644
index 0000000..552ff95
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereOptimizer.java
@@ -0,0 +1,1135 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.compile;
+
+import static java.util.Collections.singletonList;
+
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Lists;
+import org.apache.phoenix.expression.AndExpression;
+import org.apache.phoenix.expression.BaseTerminalExpression;
+import org.apache.phoenix.expression.CoerceExpression;
+import org.apache.phoenix.expression.ComparisonExpression;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.InListExpression;
+import org.apache.phoenix.expression.IsNullExpression;
+import org.apache.phoenix.expression.LikeExpression;
+import org.apache.phoenix.expression.LiteralExpression;
+import org.apache.phoenix.expression.OrExpression;
+import org.apache.phoenix.expression.RowKeyColumnExpression;
+import org.apache.phoenix.expression.RowValueConstructorExpression;
+import org.apache.phoenix.expression.RowValueConstructorExpression.ExpressionComparabilityWrapper;
+import org.apache.phoenix.expression.function.FunctionExpression.OrderPreserving;
+import org.apache.phoenix.expression.function.ScalarFunction;
+import org.apache.phoenix.expression.visitor.TraverseNoExpressionVisitor;
+import org.apache.phoenix.parse.FilterableStatement;
+import org.apache.phoenix.parse.HintNode.Hint;
+import org.apache.phoenix.query.KeyRange;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.PName;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.RowKeySchema;
+import org.apache.phoenix.schema.SaltingUtil;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.ScanUtil;
+import org.apache.phoenix.util.SchemaUtil;
+
+
+/**
+ *
+ * Class that pushes row key expressions from the where clause to form the start/stop
+ * key of the scan and removes the expressions from the where clause when possible.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class WhereOptimizer {
+    private static final List<KeyRange> SALT_PLACEHOLDER = Collections.singletonList(PDataType.CHAR.getKeyRange(QueryConstants.SEPARATOR_BYTE_ARRAY));
+    private WhereOptimizer() {
+    }
+
+    /**
+     * Pushes row key expressions from the where clause into the start/stop key of the scan.
+     * @param context the shared context during query compilation
+     * @param statement the statement being compiled
+     * @param whereClause the where clause expression
+     * @return the new where clause with the key expressions removed
+     */
+    public static Expression pushKeyExpressionsToScan(StatementContext context, FilterableStatement statement, Expression whereClause) {
+        return pushKeyExpressionsToScan(context, statement, whereClause, null);
+    }
+
+    // For testing so that the extractedNodes can be verified
+    public static Expression pushKeyExpressionsToScan(StatementContext context, FilterableStatement statement,
+            Expression whereClause, Set<Expression> extractNodes) {
+        PName tenantId = context.getConnection().getTenantId();
+        PTable table = context.getResolver().getTables().get(0).getTable();
+        if (whereClause == null && (tenantId == null || !table.isMultiTenant())) {
+            context.setScanRanges(ScanRanges.EVERYTHING);
+            return whereClause;
+        }
+        if (LiteralExpression.isFalse(whereClause)) {
+            context.setScanRanges(ScanRanges.NOTHING);
+            return null;
+        }
+        KeyExpressionVisitor visitor = new KeyExpressionVisitor(context, table);
+        KeyExpressionVisitor.KeySlots keySlots = null;
+        if (whereClause != null) {
+            // TODO:: When we only have one where clause, the keySlots returns as a single slot object,
+            // instead of an array of slots for the corresponding column. Change the behavior so it
+            // becomes consistent.
+            keySlots = whereClause.accept(visitor);
+    
+            if (keySlots == null && (tenantId == null || !table.isMultiTenant())) {
+                context.setScanRanges(ScanRanges.EVERYTHING);
+                return whereClause;
+            }
+            // If a parameter is bound to null (as will be the case for calculating ResultSetMetaData and
+            // ParameterMetaData), this will be the case. It can also happen for an equality comparison
+            // for unequal lengths.
+            if (keySlots == KeyExpressionVisitor.DEGENERATE_KEY_PARTS) {
+                context.setScanRanges(ScanRanges.NOTHING);
+                return null;
+            }
+        }
+        if (keySlots == null) {
+            keySlots = KeyExpressionVisitor.DEGENERATE_KEY_PARTS;
+        }
+        
+        if (extractNodes == null) {
+            extractNodes = new HashSet<Expression>(table.getPKColumns().size());
+        }
+
+        // We're fully qualified if all columns except the salt column are specified
+        int fullyQualifiedColumnCount = table.getPKColumns().size() - (table.getBucketNum() == null ? 0 : 1);
+        int pkPos = table.getBucketNum() == null ? -1 : 0;
+        LinkedList<List<KeyRange>> cnf = new LinkedList<List<KeyRange>>();
+        RowKeySchema schema = table.getRowKeySchema();
+        boolean forcedSkipScan = statement.getHint().hasHint(Hint.SKIP_SCAN);
+        boolean forcedRangeScan = statement.getHint().hasHint(Hint.RANGE_SCAN);
+        boolean hasUnboundedRange = false;
+        boolean hasAnyRange = false;
+        
+        Iterator<KeyExpressionVisitor.KeySlot> iterator = keySlots.iterator();
+        // add tenant data isolation for tenant-specific tables
+        if (tenantId != null && table.isMultiTenant()) {
+            KeyRange tenantIdKeyRange = KeyRange.getKeyRange(tenantId.getBytes());
+            cnf.add(singletonList(tenantIdKeyRange));
+            if (iterator.hasNext()) iterator.next();
+            pkPos++;
+        }
+        // Concat byte arrays of literals to form scan start key
+        while (iterator.hasNext()) {
+            KeyExpressionVisitor.KeySlot slot = iterator.next();
+            // If the position of the pk columns in the query skips any part of the row k
+            // then we have to handle in the next phase through a key filter.
+            // If the slot is null this means we have no entry for this pk position.
+            if (slot == null || slot.getKeyRanges().isEmpty())  {
+                if (!forcedSkipScan) break;
+                continue;
+            }
+            if (slot.getPKPosition() != pkPos + 1) {
+                if (!forcedSkipScan) break;
+                for (int i=pkPos + 1; i < slot.getPKPosition(); i++) {
+                    cnf.add(Collections.singletonList(KeyRange.EVERYTHING_RANGE));
+                }
+            }
+            // We support (a,b) IN ((1,2),(3,4), so in this case we switch to a flattened schema
+            if (fullyQualifiedColumnCount > 1 && slot.getPKSpan() == fullyQualifiedColumnCount && slot.getKeyRanges().size() > 1) {
+                schema = SchemaUtil.VAR_BINARY_SCHEMA;
+            }
+            KeyPart keyPart = slot.getKeyPart();
+            pkPos = slot.getPKPosition();
+            List<KeyRange> keyRanges = slot.getKeyRanges();
+            cnf.add(keyRanges);
+            for (KeyRange range : keyRanges) {
+                hasUnboundedRange |= range.isUnbound();
+            }
+            
+            // Will be null in cases for which only part of the expression was factored out here
+            // to set the start/end key. An example would be <column> LIKE 'foo%bar' where we can
+            // set the start key to 'foo' but still need to match the regex at filter time.
+            // Don't extract expressions if we're forcing a range scan and we've already come
+            // across a range for a prior slot. The reason is that we have an inexact range after
+            // that, so must filter on the remaining conditions (see issue #467).
+            if (!forcedRangeScan || !hasAnyRange) {
+                List<Expression> nodesToExtract = keyPart.getExtractNodes();
+                extractNodes.addAll(nodesToExtract);
+            }
+            // Stop building start/stop key once we encounter a non single key range.
+            if (hasUnboundedRange && !forcedSkipScan) {
+                // TODO: when stats are available, we may want to continue this loop if the
+                // cardinality of this slot is low. We could potentially even continue this
+                // loop in the absence of a range for a key slot.
+                break;
+            }
+            hasAnyRange |= keyRanges.size() > 1 || (keyRanges.size() == 1 && !keyRanges.get(0).isSingleKey());
+        }
+        List<List<KeyRange>> ranges = cnf;
+        if (table.getBucketNum() != null) {
+            if (!cnf.isEmpty()) {
+                // If we have all single keys, we can optimize by adding the salt byte up front
+                if (schema == SchemaUtil.VAR_BINARY_SCHEMA) {
+                    ranges = SaltingUtil.setSaltByte(ranges, table.getBucketNum());
+                } else if (ScanUtil.isAllSingleRowScan(cnf, table.getRowKeySchema())) {
+                    cnf.addFirst(SALT_PLACEHOLDER);
+                    ranges = SaltingUtil.flattenRanges(cnf, table.getRowKeySchema(), table.getBucketNum());
+                    schema = SchemaUtil.VAR_BINARY_SCHEMA;
+                } else {
+                    cnf.addFirst(SaltingUtil.generateAllSaltingRanges(table.getBucketNum()));
+                }
+            }
+        }
+        context.setScanRanges(
+                ScanRanges.create(ranges, schema, statement.getHint().hasHint(Hint.RANGE_SCAN)),
+                keySlots.getMinMaxRange());
+        if (whereClause == null) {
+            return null;
+        } else {
+            return whereClause.accept(new RemoveExtractedNodesVisitor(extractNodes));
+        }
+    }
+
+    private static class RemoveExtractedNodesVisitor extends TraverseNoExpressionVisitor<Expression> {
+        private final Set<Expression> nodesToRemove;
+
+        private RemoveExtractedNodesVisitor(Set<Expression> nodesToRemove) {
+            this.nodesToRemove = nodesToRemove;
+        }
+
+        @Override
+        public Expression defaultReturn(Expression node, List<Expression> e) {
+            return nodesToRemove.contains(node) ? null : node;
+        }
+
+        @Override
+        public Iterator<Expression> visitEnter(OrExpression node) {
+            return node.getChildren().iterator();
+        }
+
+        @Override
+        public Iterator<Expression> visitEnter(AndExpression node) {
+            return node.getChildren().iterator();
+        }
+
+        @Override
+        public Expression visitLeave(AndExpression node, List<Expression> l) {
+            if (l.size() != node.getChildren().size()) {
+                if (l.isEmpty()) {
+                    // Don't return null here, because then our defaultReturn will kick in
+                    return LiteralExpression.newConstant(true, true);
+                }
+                if (l.size() == 1) {
+                    return l.get(0);
+                }
+                return new AndExpression(l);
+            }
+            return node;
+        }
+    }
+
+    /*
+     * TODO: We could potentially rewrite simple expressions to move constants to the RHS
+     * such that we can form a start/stop key for a scan. For example, rewrite this:
+     *     WHEREH a + 1 < 5
+     * to this instead:
+     *     WHERE a < 5 - 1
+     * Currently the first case would not be optimized. This includes other arithmetic
+     * operators, CASE statements, and string concatenation.
+     */
+    public static class KeyExpressionVisitor extends TraverseNoExpressionVisitor<KeyExpressionVisitor.KeySlots> {
+        private static final List<KeyRange> EVERYTHING_RANGES = Collections.<KeyRange>singletonList(KeyRange.EVERYTHING_RANGE);
+        private static final KeySlots DEGENERATE_KEY_PARTS = new KeySlots() {
+            @Override
+            public Iterator<KeySlot> iterator() {
+                return Iterators.emptyIterator();
+            }
+
+            @Override
+            public KeyRange getMinMaxRange() {
+                return null;
+            }
+        };
+
+        private static boolean isDegenerate(List<KeyRange> keyRanges) {
+            return keyRanges == null || keyRanges.isEmpty() || (keyRanges.size() == 1 && keyRanges.get(0) == KeyRange.EMPTY_RANGE);
+        }
+        
+        private static KeySlots newKeyParts(KeySlot slot, Expression extractNode, KeyRange keyRange) {
+            if (keyRange == null) {
+                return DEGENERATE_KEY_PARTS;
+            }
+            
+            List<KeyRange> keyRanges = slot.getPKSpan() == 1 ? Collections.<KeyRange>singletonList(keyRange) : EVERYTHING_RANGES;
+            KeyRange minMaxRange = slot.getPKSpan() == 1 ? null : keyRange;
+            return newKeyParts(slot, extractNode, keyRanges, minMaxRange);
+        }
+
+        private static KeySlots newKeyParts(KeySlot slot, Expression extractNode, List<KeyRange> keyRanges, KeyRange minMaxRange) {
+            if (isDegenerate(keyRanges)) {
+                return DEGENERATE_KEY_PARTS;
+            }
+            
+            List<Expression> extractNodes = extractNode == null || slot.getKeyPart().getExtractNodes().isEmpty()
+                  ? Collections.<Expression>emptyList()
+                  : Collections.<Expression>singletonList(extractNode);
+            return new SingleKeySlot(new BaseKeyPart(slot.getKeyPart().getColumn(), extractNodes), slot.getPKPosition(), slot.getPKSpan(), keyRanges, minMaxRange, slot.getOrderPreserving());
+        }
+
+        private static KeySlots newKeyParts(KeySlot slot, List<Expression> extractNodes, List<KeyRange> keyRanges, KeyRange minMaxRange) {
+            if (isDegenerate(keyRanges)) {
+                return DEGENERATE_KEY_PARTS;
+            }
+            
+            return new SingleKeySlot(new BaseKeyPart(slot.getKeyPart().getColumn(), extractNodes), slot.getPKPosition(), slot.getPKSpan(), keyRanges, minMaxRange, slot.getOrderPreserving());
+        }
+
+        private KeySlots newRowValueConstructorKeyParts(RowValueConstructorExpression rvc, List<KeySlots> childSlots) {
+            if (childSlots.isEmpty() || rvc.isStateless()) {
+                return null;
+            }
+            
+            int positionOffset = table.getBucketNum() == null ? 0 : 1;
+            int position = 0;
+            for (KeySlots slots : childSlots) {
+                KeySlot keySlot = slots.iterator().next();
+                List<Expression> childExtractNodes = keySlot.getKeyPart().getExtractNodes();
+                // If columns are not in PK order, then stop iteration
+                if (childExtractNodes.size() != 1 
+                        || childExtractNodes.get(0) != rvc.getChildren().get(position) 
+                        || keySlot.getPKPosition() != position + positionOffset) {
+                    break;
+                }
+                position++;
+                
+                // If we come to a point where we're not preserving order completely
+                // then stop. We will never get a NO here, but we might get a YES_IF_LAST
+                // if the child expression is only using part of the underlying pk column.
+                // (for example, in the case of SUBSTR). In this case, we must stop building
+                // the row key constructor at that point.
+                assert(keySlot.getOrderPreserving() != OrderPreserving.NO);
+                if (keySlot.getOrderPreserving() == OrderPreserving.YES_IF_LAST) {
+                    break;
+                }
+            }
+            if (position > 0) {
+                int span = position;
+                return new SingleKeySlot(new RowValueConstructorKeyPart(table.getPKColumns().get(positionOffset), rvc, span, childSlots), positionOffset, span, EVERYTHING_RANGES);
+            }
+            return null;
+        }
+
+        private static KeySlots newScalarFunctionKeyPart(KeySlot slot, ScalarFunction node) {
+            if (isDegenerate(slot.getKeyRanges())) {
+                return DEGENERATE_KEY_PARTS;
+            }
+            KeyPart part = node.newKeyPart(slot.getKeyPart());
+            if (part == null) {
+                return null;
+            }
+            
+            // Scalar function always returns primitive and never a row value constructor, so span is always 1
+            return new SingleKeySlot(part, slot.getPKPosition(), slot.getKeyRanges(), node.preservesOrder());
+        }
+
+        private KeySlots newCoerceKeyPart(KeySlot slot, final CoerceExpression node) {
+            if (isDegenerate(slot.getKeyRanges())) {
+                return DEGENERATE_KEY_PARTS;
+            }
+            final KeyPart childPart = slot.getKeyPart();
+            final ImmutableBytesWritable ptr = context.getTempPtr();
+            return new SingleKeySlot(new KeyPart() {
+
+                @Override
+                public KeyRange getKeyRange(CompareOp op, Expression rhs) {
+                    KeyRange range = childPart.getKeyRange(op, rhs);
+                    byte[] lower = range.getLowerRange();
+                    if (!range.lowerUnbound()) {
+                        ptr.set(lower);
+                        // Do the reverse translation so we can optimize out the coerce expression
+                        // For the actual type of the coerceBytes call, we use the node type instead of the rhs type, because
+                        // for IN, the rhs type will be VARBINARY and no coerce will be done in that case (and we need it to
+                        // be done).
+                        node.getChild().getDataType().coerceBytes(ptr, node.getDataType(), rhs.getColumnModifier(), node.getChild().getColumnModifier());
+                        lower = ByteUtil.copyKeyBytesIfNecessary(ptr);
+                    }
+                    byte[] upper = range.getUpperRange();
+                    if (!range.upperUnbound()) {
+                        ptr.set(upper);
+                        // Do the reverse translation so we can optimize out the coerce expression
+                        node.getChild().getDataType().coerceBytes(ptr, node.getDataType(), rhs.getColumnModifier(), node.getChild().getColumnModifier());
+                        upper = ByteUtil.copyKeyBytesIfNecessary(ptr);
+                    }
+                    return KeyRange.getKeyRange(lower, range.isLowerInclusive(), upper, range.isUpperInclusive());
+                }
+
+                @Override
+                public List<Expression> getExtractNodes() {
+                    return childPart.getExtractNodes();
+                }
+
+                @Override
+                public PColumn getColumn() {
+                    return childPart.getColumn();
+                }
+            }, slot.getPKPosition(), slot.getKeyRanges());
+        }
+
+        private KeySlots andKeySlots(AndExpression andExpression, List<KeySlots> childSlots) {
+            int nColumns = table.getPKColumns().size();
+            KeySlot[] keySlot = new KeySlot[nColumns];
+            KeyRange minMaxRange = KeyRange.EVERYTHING_RANGE;
+            List<Expression> minMaxExtractNodes = Lists.<Expression>newArrayList();
+            int initPosition = (table.getBucketNum() ==null ? 0 : 1);
+            for (KeySlots childSlot : childSlots) {
+                if (childSlot == DEGENERATE_KEY_PARTS) {
+                    return DEGENERATE_KEY_PARTS;
+                }
+                if (childSlot.getMinMaxRange() != null) {
+                    // TODO: potentially use KeySlot.intersect here. However, we can't intersect the key ranges in the slot
+                    // with our minMaxRange, since it spans columns and this would mess up our skip scan.
+                    minMaxRange = minMaxRange.intersect(childSlot.getMinMaxRange());
+                    for (KeySlot slot : childSlot) {
+                        minMaxExtractNodes.addAll(slot.getKeyPart().getExtractNodes());
+                    }
+                } else {
+                    for (KeySlot slot : childSlot) {
+                        // We have a nested AND with nothing for this slot, so continue
+                        if (slot == null) {
+                            continue;
+                        }
+                        int position = slot.getPKPosition();
+                        KeySlot existing = keySlot[position];
+                        if (existing == null) {
+                            keySlot[position] = slot;
+                        } else {
+                            keySlot[position] = existing.intersect(slot);
+                            if (keySlot[position] == null) {
+                                return DEGENERATE_KEY_PARTS;
+                            }
+                        }
+                    }
+                }
+            }
+
+            if (!minMaxExtractNodes.isEmpty()) {
+                if (keySlot[initPosition] == null) {
+                    keySlot[initPosition] = new KeySlot(new BaseKeyPart(table.getPKColumns().get(initPosition), minMaxExtractNodes), initPosition, 1, EVERYTHING_RANGES, null);
+                } else {
+                    keySlot[initPosition] = keySlot[initPosition].concatExtractNodes(minMaxExtractNodes);
+                }
+            }
+            List<KeySlot> keySlots = Arrays.asList(keySlot);
+            // If we have a salt column, skip that slot because
+            // they'll never be an expression contained by it.
+            keySlots = keySlots.subList(initPosition, keySlots.size());
+            return new MultiKeySlot(keySlots, minMaxRange == KeyRange.EVERYTHING_RANGE ? null : minMaxRange);
+        }
+
+        private KeySlots orKeySlots(OrExpression orExpression, List<KeySlots> childSlots) {
+            // If any children were filtered out, filter out the entire
+            // OR expression because we don't have enough information to
+            // constraint the scan start/stop key. An example would be:
+            // WHERE organization_id=? OR key_value_column = 'x'
+            // In this case, we cannot simply filter the key_value_column,
+            // because we end up bubbling up only the organization_id=?
+            // expression to form the start/stop key which is obviously wrong.
+            // For an OR expression, you need to be able to extract
+            // everything or nothing.
+            if (orExpression.getChildren().size() != childSlots.size()) {
+                return null;
+            }
+            int initialPos = (table.getBucketNum() == null ? 0 : 1);
+            KeySlot theSlot = null;
+            List<Expression> slotExtractNodes = Lists.<Expression>newArrayList();
+            int thePosition = -1;
+            boolean extractAll = true;
+            // TODO: Have separate list for single span versus multi span
+            // For multi-span, we only need to keep a single range.
+            List<KeyRange> slotRanges = Lists.newArrayList();
+            KeyRange minMaxRange = KeyRange.EMPTY_RANGE;
+            for (KeySlots childSlot : childSlots) {
+                if (childSlot == DEGENERATE_KEY_PARTS) {
+                    // TODO: can this ever happen and can we safely filter the expression tree?
+                    continue;
+                }
+                if (childSlot.getMinMaxRange() != null) {
+                    if (!slotRanges.isEmpty() && thePosition != initialPos) { // ORing together rvc in initial slot with other slots
+                        return null;
+                    }
+                    minMaxRange = minMaxRange.union(childSlot.getMinMaxRange());
+                    thePosition = initialPos;
+                    for (KeySlot slot : childSlot) {
+                        List<Expression> extractNodes = slot.getKeyPart().getExtractNodes();
+                        extractAll &= !extractNodes.isEmpty();
+                        slotExtractNodes.addAll(extractNodes);
+                    }
+                } else {
+                    // TODO: Do the same optimization that we do for IN if the childSlots specify a fully qualified row key
+                    for (KeySlot slot : childSlot) {
+                        // We have a nested OR with nothing for this slot, so continue
+                        if (slot == null) {
+                            continue; // FIXME: I don't think this is ever necessary
+                        }
+                        /*
+                         * If we see a different PK column than before, we can't
+                         * optimize it because our SkipScanFilter only handles
+                         * top level expressions that are ANDed together (where in
+                         * the same column expressions may be ORed together).
+                         * For example, WHERE a=1 OR b=2 cannot be handled, while
+                         *  WHERE (a=1 OR a=2) AND (b=2 OR b=3) can be handled.
+                         * TODO: We could potentially handle these cases through
+                         * multiple, nested SkipScanFilters, where each OR expression
+                         * is handled by its own SkipScanFilter and the outer one
+                         * increments the child ones and picks the one with the smallest
+                         * key.
+                         */
+                        if (thePosition == -1) {
+                            theSlot = slot;
+                            thePosition = slot.getPKPosition();
+                        } else if (thePosition != slot.getPKPosition()) {
+                            return null;
+                        }
+                        List<Expression> extractNodes = slot.getKeyPart().getExtractNodes();
+                        extractAll &= !extractNodes.isEmpty();
+                        slotExtractNodes.addAll(extractNodes);
+                        slotRanges.addAll(slot.getKeyRanges());
+                    }
+                }
+            }
+
+            if (thePosition == -1) {
+                return null;
+            }
+            // With a mix of both, we can't use skip scan, so empty out the union
+            // and only extract the min/max nodes.
+            if (!slotRanges.isEmpty() && minMaxRange != KeyRange.EMPTY_RANGE) {
+                boolean clearExtracts = false;
+                // Union the minMaxRanges together with the slotRanges.
+                for (KeyRange range : slotRanges) {
+                    if (!clearExtracts) {
+                        /*
+                         * Detect when to clear the extract nodes by determining if there
+                         * are gaps left by combining the ranges. If there are gaps, we
+                         * cannot extract the nodes, but must them as filters instead.
+                         */
+                        KeyRange intersection = minMaxRange.intersect(range);
+                        if (intersection == KeyRange.EMPTY_RANGE 
+                                || !range.equals(intersection.union(range)) 
+                                || !minMaxRange.equals(intersection.union(minMaxRange))) {
+                            clearExtracts = true;
+                        }
+                    }
+                    minMaxRange = minMaxRange.union(range);
+                }
+                if (clearExtracts) {
+                    extractAll = false;
+                    slotExtractNodes = Collections.emptyList();
+                }
+                slotRanges = Collections.emptyList();
+            }
+            if (theSlot == null) {
+                theSlot = new KeySlot(new BaseKeyPart(table.getPKColumns().get(initialPos), slotExtractNodes), initialPos, 1, EVERYTHING_RANGES, null);
+            } else if (minMaxRange != KeyRange.EMPTY_RANGE && !slotExtractNodes.isEmpty()) {
+                theSlot = theSlot.concatExtractNodes(slotExtractNodes);
+            }
+            return newKeyParts(
+                    theSlot, 
+                    extractAll ? Collections.<Expression>singletonList(orExpression) : slotExtractNodes, 
+                    slotRanges.isEmpty() ? EVERYTHING_RANGES : KeyRange.coalesce(slotRanges), 
+                    minMaxRange == KeyRange.EMPTY_RANGE ? null : minMaxRange);
+        }
+
+        private final PTable table;
+        private final StatementContext context;
+
+        public KeyExpressionVisitor(StatementContext context, PTable table) {
+            this.context = context;
+            this.table = table;
+        }
+
+        private boolean isFullyQualified(int pkSpan) {
+            int nPKColumns = table.getPKColumns().size();
+            return table.getBucketNum() == null ? pkSpan == nPKColumns : pkSpan == nPKColumns-1;
+        }
+        @Override
+        public KeySlots defaultReturn(Expression node, List<KeySlots> l) {
+            // Passes the CompositeKeyExpression up the tree
+            return l.size() == 1 ? l.get(0) : null;
+        }
+
+
+        @Override
+        public Iterator<Expression> visitEnter(CoerceExpression node) {
+            return node.getChildren().iterator();
+        }
+
+        @Override
+        public KeySlots visitLeave(CoerceExpression node, List<KeySlots> childParts) {
+            if (childParts.isEmpty()) {
+                return null;
+            }
+            return newCoerceKeyPart(childParts.get(0).iterator().next(), node);
+        }
+
+        @Override
+        public Iterator<Expression> visitEnter(AndExpression node) {
+            return node.getChildren().iterator();
+        }
+
+        @Override
+        public KeySlots visitLeave(AndExpression node, List<KeySlots> l) {
+            KeySlots keyExpr = andKeySlots(node, l);
+            return keyExpr;
+        }
+
+        @Override
+        public Iterator<Expression> visitEnter(OrExpression node) {
+            return node.getChildren().iterator();
+        }
+
+        @Override
+        public KeySlots visitLeave(OrExpression node, List<KeySlots> l) {
+            KeySlots keySlots = orKeySlots(node, l);
+            if (keySlots == null) {
+                // If we don't clear the child list, we end up passing some of
+                // the child expressions of the OR up the tree, causing only
+                // those expressions to form the scan start/stop key.
+                l.clear();
+                return null;
+            }
+            return keySlots;
+        }
+
+        @Override
+        public Iterator<Expression> visitEnter(RowValueConstructorExpression node) {
+            return node.getChildren().iterator();
+        }
+
+        @Override
+        public KeySlots visitLeave(RowValueConstructorExpression node, List<KeySlots> childSlots) {
+            return newRowValueConstructorKeyParts(node, childSlots);
+        }
+
+        @Override
+        public KeySlots visit(RowKeyColumnExpression node) {
+            PColumn column = table.getPKColumns().get(node.getPosition());
+            return new SingleKeySlot(new BaseKeyPart(column, Collections.<Expression>singletonList(node)), node.getPosition(), 1, EVERYTHING_RANGES);
+        }
+
+        @Override
+        public Iterator<Expression> visitEnter(ComparisonExpression node) {
+            Expression rhs = node.getChildren().get(1);
+            if (!rhs.isStateless() || node.getFilterOp() == CompareOp.NOT_EQUAL) {
+                return Iterators.emptyIterator();
+            }
+            return Iterators.singletonIterator(node.getChildren().get(0));
+        }
+
+        @Override
+        public KeySlots visitLeave(ComparisonExpression node, List<KeySlots> childParts) {
+            // Delay adding to extractedNodes, until we're done traversing,
+            // since we can't yet tell whether or not the PK column references
+            // are contiguous
+            if (childParts.isEmpty()) {
+                return null;
+            }
+            Expression rhs = node.getChildren().get(1);
+            KeySlots childSlots = childParts.get(0);
+            KeySlot childSlot = childSlots.iterator().next();
+            KeyPart childPart = childSlot.getKeyPart();
+            ColumnModifier modifier = childPart.getColumn().getColumnModifier();
+            CompareOp op = node.getFilterOp();
+            // For descending columns, the operator needs to be transformed to
+            // it's opposite, since the range is backwards.
+            if (modifier != null) {
+                op = modifier.transform(op);
+            }
+            KeyRange keyRange = childPart.getKeyRange(op, rhs);
+            return newKeyParts(childSlot, node, keyRange);
+        }
+
+        // TODO: consider supporting expression substitution in the PK for pre-joined tables
+        // You'd need to register the expression for a given PK and substitute with a column
+        // reference for this during ExpressionBuilder.
+        @Override
+        public Iterator<Expression> visitEnter(ScalarFunction node) {
+            int index = node.getKeyFormationTraversalIndex();
+            if (index < 0) {
+                return Iterators.emptyIterator();
+            }
+            return Iterators.singletonIterator(node.getChildren().get(index));
+        }
+
+        @Override
+        public KeySlots visitLeave(ScalarFunction node, List<KeySlots> childParts) {
+            if (childParts.isEmpty()) {
+                return null;
+            }
+            return newScalarFunctionKeyPart(childParts.get(0).iterator().next(), node);
+        }
+
+        @Override
+        public Iterator<Expression> visitEnter(LikeExpression node) {
+            // TODO: can we optimize something that starts with '_' like this: foo LIKE '_a%' ?
+            if (! (node.getChildren().get(1) instanceof LiteralExpression) || node.startsWithWildcard()) {
+                return Iterators.emptyIterator();
+            }
+
+            return Iterators.singletonIterator(node.getChildren().get(0));
+        }
+
+        @Override
+        public KeySlots visitLeave(LikeExpression node, List<KeySlots> childParts) {
+            if (childParts.isEmpty()) {
+                return null;
+            }
+            // for SUBSTR(<column>,1,3) LIKE 'foo%'
+            KeySlots childSlots = childParts.get(0);
+            KeySlot childSlot = childSlots.iterator().next();
+            final String startsWith = node.getLiteralPrefix();
+            byte[] key = PDataType.CHAR.toBytes(startsWith, node.getChildren().get(0).getColumnModifier());
+            // If the expression is an equality expression against a fixed length column
+            // and the key length doesn't match the column length, the expression can
+            // never be true.
+            // An zero length byte literal is null which can never be compared against as true
+            Integer childNodeFixedLength = node.getChildren().get(0).getByteSize();
+            if (childNodeFixedLength != null && key.length > childNodeFixedLength) {
+                return DEGENERATE_KEY_PARTS;
+            }
+            // TODO: is there a case where we'd need to go through the childPart to calculate the key range?
+            PColumn column = childSlot.getKeyPart().getColumn();
+            PDataType type = column.getDataType();
+            KeyRange keyRange = type.getKeyRange(key, true, ByteUtil.nextKey(key), false);
+            Integer columnFixedLength = column.getByteSize();
+            if (columnFixedLength != null) {
+                keyRange = keyRange.fill(columnFixedLength);
+            }
+            // Only extract LIKE expression if pattern ends with a wildcard and everything else was extracted
+            return newKeyParts(childSlot, node.endsWithOnlyWildcard() ? node : null, keyRange);
+        }
+
+        @Override
+        public Iterator<Expression> visitEnter(InListExpression node) {
+            return Iterators.singletonIterator(node.getChildren().get(0));
+        }
+
+        @Override
+        public KeySlots visitLeave(InListExpression node, List<KeySlots> childParts) {
+            if (childParts.isEmpty()) {
+                return null;
+            }
+
+            List<Expression> keyExpressions = node.getKeyExpressions();
+            List<KeyRange> ranges = Lists.newArrayListWithExpectedSize(keyExpressions.size());
+            KeySlot childSlot = childParts.get(0).iterator().next();
+            KeyPart childPart = childSlot.getKeyPart();
+            ColumnModifier mod = node.getChildren().get(0).getColumnModifier();
+            // We can only optimize a row value constructor that is fully qualified
+            if (childSlot.getPKSpan() > 1 && !isFullyQualified(childSlot.getPKSpan())) {
+                // Just return a key part that has the min/max of the IN list, but doesn't
+                // extract the IN list expression.
+                return newKeyParts(childSlot, (Expression)null, Collections.singletonList(
+                        KeyRange.getKeyRange(
+                                ByteUtil.copyKeyBytesIfNecessary(node.getMinKey()), true,
+                                ByteUtil.copyKeyBytesIfNecessary(node.getMaxKey()), true)), null);
+            }
+            // Handles cases like WHERE substr(foo,1,3) IN ('aaa','bbb')
+            for (Expression key : keyExpressions) {
+                KeyRange range = childPart.getKeyRange(CompareOp.EQUAL, key);
+                if (range != KeyRange.EMPTY_RANGE) { // null means it can't possibly be in range
+                    if (mod != null) {
+                        range = range.invert();
+                    }
+                    ranges.add(range);
+                }
+            }
+            return newKeyParts(childSlot, node, ranges, null);
+        }
+
+        @Override
+        public Iterator<Expression> visitEnter(IsNullExpression node) {
+            return Iterators.singletonIterator(node.getChildren().get(0));
+        }
+
+        @Override
+        public KeySlots visitLeave(IsNullExpression node, List<KeySlots> childParts) {
+            if (childParts.isEmpty()) {
+                return null;
+            }
+            KeySlots childSlots = childParts.get(0);
+            KeySlot childSlot = childSlots.iterator().next();
+            PColumn column = childSlot.getKeyPart().getColumn();
+            PDataType type = column.getDataType();
+            boolean isFixedWidth = type.isFixedWidth();
+            if (isFixedWidth) { // if column can't be null
+                return node.isNegate() ? null : 
+                    newKeyParts(childSlot, node, type.getKeyRange(new byte[column.getByteSize()], true,
+                                                                  KeyRange.UNBOUND, true));
+            } else {
+                KeyRange keyRange = node.isNegate() ? KeyRange.IS_NOT_NULL_RANGE : KeyRange.IS_NULL_RANGE;
+                return newKeyParts(childSlot, node, keyRange);
+            }
+        }
+
+        private static interface KeySlots extends Iterable<KeySlot> {
+            @Override public Iterator<KeySlot> iterator();
+            public KeyRange getMinMaxRange();
+        }
+
+        private static final class KeySlot {
+            private final int pkPosition;
+            private final int pkSpan;
+            private final KeyPart keyPart;
+            private final List<KeyRange> keyRanges;
+            private final OrderPreserving orderPreserving;
+
+            private KeySlot(KeyPart keyPart, int pkPosition, int pkSpan, List<KeyRange> keyRanges) {
+                this (keyPart, pkPosition, pkSpan, keyRanges, OrderPreserving.YES);
+            }
+            
+            private KeySlot(KeyPart keyPart, int pkPosition, int pkSpan, List<KeyRange> keyRanges, OrderPreserving orderPreserving) {
+                this.pkPosition = pkPosition;
+                this.pkSpan = pkSpan;
+                this.keyPart = keyPart;
+                this.keyRanges = keyRanges;
+                this.orderPreserving = orderPreserving;
+            }
+
+            public KeyPart getKeyPart() {
+                return keyPart;
+            }
+
+            public int getPKPosition() {
+                return pkPosition;
+            }
+
+            public int getPKSpan() {
+                return pkSpan;
+            }
+
+            public List<KeyRange> getKeyRanges() {
+                return keyRanges;
+            }
+
+            public final KeySlot concatExtractNodes(List<Expression> extractNodes) {
+                return new KeySlot(
+                        new BaseKeyPart(this.getKeyPart().getColumn(),
+                                    SchemaUtil.concat(this.getKeyPart().getExtractNodes(),extractNodes)),
+                        this.getPKPosition(),
+                        this.getPKSpan(),
+                        this.getKeyRanges(),
+                        this.getOrderPreserving());
+            }
+            
+            public final KeySlot intersect(KeySlot that) {
+                if (this.getPKPosition() != that.getPKPosition()) {
+                    throw new IllegalArgumentException("Position must be equal for intersect");
+                }
+                Preconditions.checkArgument(!this.keyRanges.isEmpty());
+                Preconditions.checkArgument(!that.keyRanges.isEmpty());
+
+                List<KeyRange> keyRanges = KeyRange.intersect(this.getKeyRanges(), that.getKeyRanges());
+                if (isDegenerate(keyRanges)) {
+                    return null;
+                }
+                return new KeySlot(
+                        new BaseKeyPart(this.getKeyPart().getColumn(),
+                                    SchemaUtil.concat(this.getKeyPart().getExtractNodes(),
+                                                      that.getKeyPart().getExtractNodes())),
+                        this.getPKPosition(),
+                        this.getPKSpan(),
+                        keyRanges,
+                        this.getOrderPreserving());
+            }
+
+            public OrderPreserving getOrderPreserving() {
+                return orderPreserving;
+            }
+        }
+
+        private static class MultiKeySlot implements KeySlots {
+            private final List<KeySlot> childSlots;
+            private final KeyRange minMaxRange;
+
+            private MultiKeySlot(List<KeySlot> childSlots, KeyRange minMaxRange) {
+                this.childSlots = childSlots;
+                this.minMaxRange = minMaxRange;
+            }
+
+            @Override
+            public Iterator<KeySlot> iterator() {
+                return childSlots.iterator();
+            }
+
+            @Override
+            public KeyRange getMinMaxRange() {
+                return minMaxRange;
+            }
+        }
+
+        private static class SingleKeySlot implements KeySlots {
+            private final KeySlot slot;
+            private final KeyRange minMaxRange;
+            
+            private SingleKeySlot(KeyPart part, int pkPosition, List<KeyRange> ranges) {
+                this(part, pkPosition, 1, ranges);
+            }
+            
+            private SingleKeySlot(KeyPart part, int pkPosition, List<KeyRange> ranges, OrderPreserving orderPreserving) {
+                this(part, pkPosition, 1, ranges, orderPreserving);
+            }
+            
+            private SingleKeySlot(KeyPart part, int pkPosition, int pkSpan, List<KeyRange> ranges) {
+                this(part,pkPosition,pkSpan,ranges, null, null);
+            }
+            
+            private SingleKeySlot(KeyPart part, int pkPosition, int pkSpan, List<KeyRange> ranges, OrderPreserving orderPreserving) {
+                this(part,pkPosition,pkSpan,ranges, null, orderPreserving);
+            }
+            
+            private SingleKeySlot(KeyPart part, int pkPosition, int pkSpan, List<KeyRange> ranges, KeyRange minMaxRange, OrderPreserving orderPreserving) {
+                this.slot = new KeySlot(part, pkPosition, pkSpan, ranges, orderPreserving);
+                this.minMaxRange = minMaxRange;
+            }
+            
+            @Override
+            public Iterator<KeySlot> iterator() {
+                return Iterators.<KeySlot>singletonIterator(slot);
+            }
+
+            @Override
+            public KeyRange getMinMaxRange() {
+                return minMaxRange;
+            }
+            
+        }
+        
+        private static class BaseKeyPart implements KeyPart {
+            @Override
+            public KeyRange getKeyRange(CompareOp op, Expression rhs) {
+                ImmutableBytesWritable ptr = new ImmutableBytesWritable();
+                rhs.evaluate(null, ptr);
+                byte[] key = ByteUtil.copyKeyBytesIfNecessary(ptr);
+                // If the column is fixed width, fill is up to it's byte size
+                PDataType type = getColumn().getDataType();
+                if (type.isFixedWidth()) {
+                    Integer length = getColumn().getByteSize();
+                    if (length != null) {
+                        key = ByteUtil.fillKey(key, length);
+                    }
+                }
+                return ByteUtil.getKeyRange(key, op, type);
+            }
+
+            private final PColumn column;
+            private final List<Expression> nodes;
+
+            private BaseKeyPart(PColumn column, List<Expression> nodes) {
+                this.column = column;
+                this.nodes = nodes;
+            }
+
+            @Override
+            public List<Expression> getExtractNodes() {
+                return nodes;
+            }
+
+            @Override
+            public PColumn getColumn() {
+                return column;
+            }
+        }
+        
+        private  class RowValueConstructorKeyPart implements KeyPart {
+            private final RowValueConstructorExpression rvc;
+            private final PColumn column;
+            private final List<Expression> nodes;
+            private final List<KeySlots> childSlots;
+
+            private RowValueConstructorKeyPart(PColumn column, RowValueConstructorExpression rvc, int span, List<KeySlots> childSlots) {
+                this.column = column;
+                if (span == rvc.getChildren().size()) {
+                    this.rvc = rvc;
+                    this.nodes = Collections.<Expression>singletonList(rvc);
+                    this.childSlots = childSlots;
+                } else {
+                    this.rvc = new RowValueConstructorExpression(rvc.getChildren().subList(0, span),rvc.isStateless());
+                    this.nodes = Collections.<Expression>emptyList();
+                    this.childSlots = childSlots.subList(0,  span);
+                }
+            }
+
+            @Override
+            public List<Expression> getExtractNodes() {
+                return nodes;
+            }
+
+            @Override
+            public PColumn getColumn() {
+                return column;
+            }
+           @Override
+            public KeyRange getKeyRange(CompareOp op, Expression rhs) {
+               // With row value constructors, we need to convert the operator for any transformation we do on individual values
+               // to prevent keys from being increased to the next key as would be done for fixed width values. The next key is
+               // done to compensate for the start key (lower range) always being inclusive (thus we convert > to >=) and the
+               // end key (upper range) always being exclusive (thus we convert <= to <). 
+               final CompareOp rvcElementOp = op == CompareOp.LESS_OR_EQUAL ? CompareOp.LESS : op == CompareOp.GREATER ? CompareOp.GREATER_OR_EQUAL : op;
+                if (op != CompareOp.EQUAL) {
+                    boolean usedAllOfLHS = !nodes.isEmpty();
+                    // We need to transform the comparison operator for a LHS row value constructor
+                    // that is shorter than a RHS row value constructor when we're extracting it.
+                    // For example: a < (1,2) is true if a = 1, so we need to switch
+                    // the compare op to <= like this: a <= 1. Since we strip trailing nulls
+                    // in the rvc, we don't need to worry about the a < (1,null) case.
+                    if (usedAllOfLHS && rvc.getChildren().size() < rhs.getChildren().size()) {
+                        if (op == CompareOp.LESS) {
+                            op = CompareOp.LESS_OR_EQUAL;
+                        } else if (op == CompareOp.GREATER_OR_EQUAL) {
+                            op = CompareOp.GREATER;
+                        }
+                    }
+                    if (!usedAllOfLHS || rvc.getChildren().size() != rhs.getChildren().size()) {
+                        // We know that rhs was converted to a row value constructor and that it's a constant
+                        rhs= new RowValueConstructorExpression(rhs.getChildren().subList(0, Math.min(rvc.getChildren().size(), rhs.getChildren().size())), rhs.isStateless());
+                    }
+                }
+                /*
+                 * Recursively transform the RHS row value constructor by applying the same logic as
+                 * is done elsewhere during WHERE optimization: optimizing out LHS functions by applying
+                 * the appropriate transformation to the RHS key.
+                 */
+                // Child slot iterator parallel with child expressions of the LHS row value constructor 
+                final Iterator<KeySlots> keySlotsIterator = childSlots.iterator();
+                try {
+                    // Call our static row value expression constructor with the current LHS row value constructor and
+                    // the current RHS (which has already been coerced to match the LHS expression). We pass through an
+                    // implementation of ExpressionComparabilityWrapper that transforms the RHS key to match the row key
+                    // structure of the LHS column. This is essentially optimizing out the expressions on the LHS by
+                    // applying the appropriate transformations to the RHS (through the KeyPart#getKeyRange method).
+                    // For example, with WHERE (invert(a),b) < ('abc',5), the 'abc' would be inverted by going through the
+                    // childPart.getKeyRange defined for the invert function.
+                    rhs = RowValueConstructorExpression.coerce(rvc, rhs, new ExpressionComparabilityWrapper() {
+
+                        @Override
+                        public Expression wrap(final Expression lhs, final Expression rhs) throws SQLException {
+                            final KeyPart childPart = keySlotsIterator.next().iterator().next().getKeyPart();
+                            // TODO: DelegateExpression
+                            return new BaseTerminalExpression() {
+                                @Override
+                                public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+                                    if (childPart == null) {
+                                        return rhs.evaluate(tuple, ptr);
+                                    }
+                                    if (!rhs.evaluate(tuple, ptr)) {
+                                        return false;
+                                    }
+                                    if (ptr.getLength() == 0) {
+                                        ptr.set(ByteUtil.EMPTY_BYTE_ARRAY);
+                                        return true;
+                                    }
+                                    KeyRange range = childPart.getKeyRange(rvcElementOp, rhs);
+                                    // This can happen when an EQUAL operator is used and the expression cannot possibly match.
+                                    if (range == KeyRange.EMPTY_RANGE) {
+                                        return false;
+                                    }
+                                    // We have to take the range and condense it down to a single key. We use which ever
+                                    // part of the range is inclusive (which implies being bound as well). This works in all
+                                    // cases, including this substring one, which produces a lower inclusive range and an
+                                    // upper non inclusive range.
+                                    // (a, substr(b,1,1)) IN (('a','b'), ('c','d'))
+                                    byte[] key = range.isLowerInclusive() ? range.getLowerRange() : range.getUpperRange();
+                                    // FIXME: this is kind of a hack. The above call will fill a fixed width key, but
+                                    // we don't want to fill the key yet because it can throw off our the logic we
+                                    // use to compute the next key when we evaluate the RHS row value constructor
+                                    // below.  We could create a new childPart with a delegate column that returns
+                                    // null for getByteSize().
+                                    if (lhs.getByteSize() != null && key.length != lhs.getByteSize()) {
+                                        key = Arrays.copyOf(key, lhs.getByteSize());
+                                    }
+                                    ptr.set(key);
+                                    return true;
+                                }
+
+                                @Override
+                                public PDataType getDataType() {
+                                    return childPart.getColumn().getDataType();
+                                }
+                                
+                                @Override
+                                public boolean isNullable() {
+                                    return childPart.getColumn().isNullable();
+                                }
+
+                                @Override
+                                public Integer getByteSize() {
+                                    return lhs.getByteSize();
+                               }
+
+                                @Override
+                                public Integer getMaxLength() {
+                                    return lhs.getMaxLength();
+                                }
+
+                                @Override
+                                public Integer getScale() {
+                                    return childPart.getColumn().getScale();
+                                }
+
+                                @Override
+                                public ColumnModifier getColumnModifier() {
+                                    return childPart.getColumn().getColumnModifier();
+                                }
+                            };
+                        }
+                        
+                    });
+                } catch (SQLException e) {
+                    return null; // Shouldn't happen
+                }
+                ImmutableBytesWritable ptr = context.getTempPtr();
+                if (!rhs.evaluate(null, ptr) || ptr.getLength()==0) {
+                    return null; 
+                }
+                byte[] key = ByteUtil.copyKeyBytesIfNecessary(ptr);
+                return ByteUtil.getKeyRange(key, op, PDataType.VARBINARY);
+            }
+
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseRegionScanner.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseRegionScanner.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseRegionScanner.java
new file mode 100644
index 0000000..44f4303
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseRegionScanner.java
@@ -0,0 +1,69 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.coprocessor;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.regionserver.RegionScanner;
+
+public abstract class BaseRegionScanner implements RegionScanner {
+
+    @Override
+    public boolean isFilterDone() {
+        return false; 
+    }
+
+    @Override
+    public boolean next(List<KeyValue> results, String metric) throws IOException {
+        return next(results);
+    }
+
+    @Override
+    public boolean next(List<KeyValue> result, int limit, String metric) throws IOException {
+        return next(result);
+    }
+    @Override
+    public boolean next(List<KeyValue> result, int limit) throws IOException {
+        return next(result);
+    }
+    
+    @Override
+    public boolean reseek(byte[] row) throws IOException {
+        throw new DoNotRetryIOException("Unsupported");
+    }
+
+    @Override
+    public long getMvccReadPoint() {
+        return Long.MAX_VALUE;
+    }
+
+    @Override
+    public boolean nextRaw(List<KeyValue> result, String metric) throws IOException {
+        return next(result, metric);
+    }
+
+    @Override
+    public boolean nextRaw(List<KeyValue> result, int limit, String metric) throws IOException {
+        return next(result, limit, metric);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..6c8306e
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java
@@ -0,0 +1,57 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.coprocessor;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.coprocessor.*;
+import org.apache.hadoop.hbase.regionserver.RegionScanner;
+
+import org.apache.phoenix.util.ServerUtil;
+
+
+abstract public class BaseScannerRegionObserver extends BaseRegionObserver {
+    
+    /**
+     * Used by logger to identify coprocessor
+     */
+    @Override
+    public String toString() {
+        return this.getClass().getName();
+    }
+    
+    abstract protected RegionScanner doPostScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c, final Scan scan, final RegionScanner s) throws Throwable;
+    
+    /**
+     * Wrapper for {@link #postScannerOpen(ObserverContext, Scan, RegionScanner)} that ensures no non IOException is thrown,
+     * to prevent the coprocessor from becoming blacklisted.
+     * 
+     */
+    @Override
+    public final RegionScanner postScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c, final Scan scan, final RegionScanner s) throws IOException {
+        try {
+            return doPostScannerOpen(c, scan, s);
+        } catch (Throwable t) {
+            ServerUtil.throwIOException(c.getEnvironment().getRegion().getRegionNameAsString(), t);
+            return null; // impossible
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupByCache.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupByCache.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupByCache.java
new file mode 100644
index 0000000..1fdf272
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupByCache.java
@@ -0,0 +1,41 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.coprocessor;
+
+import java.io.Closeable;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.regionserver.RegionScanner;
+
+import org.apache.phoenix.expression.aggregator.Aggregator;
+
+/**
+ * 
+ * Interface to abstract the way in which distinct group by
+ * elements are cached
+ *
+ * @author jtaylor
+ * @since 3.0.0
+ */
+public interface GroupByCache extends Closeable {
+    int size();
+    Aggregator[] cache(ImmutableBytesWritable key);
+    RegionScanner getScanner(RegionScanner s);
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..abdd7bb
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupedAggregateRegionObserver.java
@@ -0,0 +1,507 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.coprocessor;
+
+import static org.apache.phoenix.query.QueryConstants.AGG_TIMESTAMP;
+import static org.apache.phoenix.query.QueryConstants.SINGLE_COLUMN;
+import static org.apache.phoenix.query.QueryConstants.SINGLE_COLUMN_FAMILY;
+import static org.apache.phoenix.query.QueryServices.GROUPBY_SPILLABLE_ATTRIB;
+import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_GROUPBY_SPILLABLE;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.EOFException;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.MultiVersionConsistencyControl;
+import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.WritableUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Maps;
+import com.google.common.io.Closeables;
+import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.cache.GlobalCache;
+import org.apache.phoenix.cache.TenantCache;
+import org.apache.phoenix.cache.aggcache.SpillableGroupByCache;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.ExpressionType;
+import org.apache.phoenix.expression.aggregator.Aggregator;
+import org.apache.phoenix.expression.aggregator.ServerAggregators;
+import org.apache.phoenix.join.HashJoinInfo;
+import org.apache.phoenix.join.ScanProjector;
+import org.apache.phoenix.memory.MemoryManager.MemoryChunk;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.tuple.MultiKeyValueTuple;
+import org.apache.phoenix.util.KeyValueUtil;
+import org.apache.phoenix.util.ScanUtil;
+import org.apache.phoenix.util.SizedUtil;
+import org.apache.phoenix.util.TupleUtil;
+
+/**
+ * Region observer that aggregates grouped rows (i.e. SQL query with GROUP BY clause)
+ * @author jtaylor
+ * @since 0.1
+ */
+public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
+    private static final Logger logger = LoggerFactory
+            .getLogger(GroupedAggregateRegionObserver.class);
+
+    public static final String AGGREGATORS = "Aggs";
+    public static final String UNORDERED_GROUP_BY_EXPRESSIONS = "UnorderedGroupByExpressions";
+    public static final String KEY_ORDERED_GROUP_BY_EXPRESSIONS = "OrderedGroupByExpressions";
+
+    public static final String ESTIMATED_DISTINCT_VALUES = "EstDistinctValues";
+    public static final int DEFAULT_ESTIMATED_DISTINCT_VALUES = 10000;
+    public static final int MIN_DISTINCT_VALUES = 100;
+
+    /**
+     * Replaces the RegionScanner s with a RegionScanner that groups by the key formed by the list
+     * of expressions from the scan and returns the aggregated rows of each group. For example,
+     * given the following original rows in the RegionScanner: KEY COL1 row1 a row2 b row3 a row4 a
+     * the following rows will be returned for COUNT(*): KEY COUNT a 3 b 1 The client is required to
+     * do a sort and a final aggregation, since multiple rows with the same key may be returned from
+     * different regions.
+     */
+    @Override
+    protected RegionScanner doPostScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
+            Scan scan, RegionScanner s) throws IOException {
+        boolean keyOrdered = false;
+        byte[] expressionBytes = scan.getAttribute(UNORDERED_GROUP_BY_EXPRESSIONS);
+
+        if (expressionBytes == null) {
+            expressionBytes = scan.getAttribute(KEY_ORDERED_GROUP_BY_EXPRESSIONS);
+            if (expressionBytes == null) {
+                return s;
+            }
+            keyOrdered = true;
+        }
+        List<Expression> expressions = deserializeGroupByExpressions(expressionBytes);
+
+        ServerAggregators aggregators =
+                ServerAggregators.deserialize(scan
+                        .getAttribute(GroupedAggregateRegionObserver.AGGREGATORS), c
+                        .getEnvironment().getConfiguration());
+
+        final ScanProjector p = ScanProjector.deserializeProjectorFromScan(scan);
+        final HashJoinInfo j = HashJoinInfo.deserializeHashJoinFromScan(scan);
+        RegionScanner innerScanner = s;
+        if (p != null || j != null) {
+            innerScanner =
+                    new HashJoinRegionScanner(s, p, j, ScanUtil.getTenantId(scan),
+                            c.getEnvironment());
+        }
+
+        if (keyOrdered) { // Optimize by taking advantage that the rows are
+                          // already in the required group by key order
+            return scanOrdered(c, scan, innerScanner, expressions, aggregators);
+        } else { // Otherwse, collect them all up in an in memory map
+            return scanUnordered(c, scan, innerScanner, expressions, aggregators);
+        }
+    }
+
+    public static int sizeOfUnorderedGroupByMap(int nRows, int valueSize) {
+        return SizedUtil.sizeOfMap(nRows, SizedUtil.IMMUTABLE_BYTES_WRITABLE_SIZE, valueSize);
+    }
+
+    public static void serializeIntoScan(Scan scan, String attribName,
+            List<Expression> groupByExpressions) {
+        ByteArrayOutputStream stream =
+                new ByteArrayOutputStream(Math.max(1, groupByExpressions.size() * 10));
+        try {
+            if (groupByExpressions.isEmpty()) { // FIXME ?
+                stream.write(QueryConstants.TRUE);
+            } else {
+                DataOutputStream output = new DataOutputStream(stream);
+                for (Expression expression : groupByExpressions) {
+                    WritableUtils.writeVInt(output, ExpressionType.valueOf(expression).ordinal());
+                    expression.write(output);
+                }
+            }
+        } catch (IOException e) {
+            throw new RuntimeException(e); // Impossible
+        } finally {
+            try {
+                stream.close();
+            } catch (IOException e) {
+                throw new RuntimeException(e);
+            }
+        }
+        scan.setAttribute(attribName, stream.toByteArray());
+
+    }
+
+    private List<Expression> deserializeGroupByExpressions(byte[] expressionBytes)
+            throws IOException {
+        List<Expression> expressions = new ArrayList<Expression>(3);
+        ByteArrayInputStream stream = new ByteArrayInputStream(expressionBytes);
+        try {
+            DataInputStream input = new DataInputStream(stream);
+            while (true) {
+                try {
+                    int expressionOrdinal = WritableUtils.readVInt(input);
+                    Expression expression =
+                            ExpressionType.values()[expressionOrdinal].newInstance();
+                    expression.readFields(input);
+                    expressions.add(expression);
+                } catch (EOFException e) {
+                    break;
+                }
+            }
+        } finally {
+            stream.close();
+        }
+        return expressions;
+    }
+
+    /**
+     * 
+     * Cache for distinct values and their aggregations which is completely
+     * in-memory (as opposed to spilling to disk). Used when GROUPBY_SPILLABLE_ATTRIB
+     * is set to false. The memory usage is tracked at a coursed grain and will
+     * throw and abort if too much is used.
+     *
+     * @author jtaylor
+     * @since 3.0.0
+     */
+    private static final class InMemoryGroupByCache implements GroupByCache {
+        private final MemoryChunk chunk;
+        private final Map<ImmutableBytesPtr, Aggregator[]> aggregateMap;
+        private final ServerAggregators aggregators;
+        private final RegionCoprocessorEnvironment env;
+        
+        private int estDistVals;
+        
+        InMemoryGroupByCache(RegionCoprocessorEnvironment env, ImmutableBytesWritable tenantId, ServerAggregators aggregators, int estDistVals) {
+            int estValueSize = aggregators.getEstimatedByteSize();
+            int estSize = sizeOfUnorderedGroupByMap(estDistVals, estValueSize);
+            TenantCache tenantCache = GlobalCache.getTenantCache(env, tenantId);
+            this.env = env;
+            this.estDistVals = estDistVals;
+            this.aggregators = aggregators;
+            this.aggregateMap = Maps.newHashMapWithExpectedSize(estDistVals);
+            this.chunk = tenantCache.getMemoryManager().allocate(estSize);
+        }
+        
+        @Override
+        public void close() throws IOException {
+            this.chunk.close();
+        }
+
+        @Override
+        public Aggregator[] cache(ImmutableBytesWritable cacheKey) {
+            ImmutableBytesPtr key = new ImmutableBytesPtr(cacheKey);
+            Aggregator[] rowAggregators = aggregateMap.get(key);
+            if (rowAggregators == null) {
+                // If Aggregators not found for this distinct
+                // value, clone our original one (we need one
+                // per distinct value)
+                if (logger.isDebugEnabled()) {
+                    logger.debug("Adding new aggregate bucket for row key "
+                            + Bytes.toStringBinary(key.get(), key.getOffset(),
+                                key.getLength()));
+                }
+                rowAggregators =
+                        aggregators.newAggregators(env.getConfiguration());
+                aggregateMap.put(key, rowAggregators);
+
+                if (aggregateMap.size() > estDistVals) { // increase allocation
+                    estDistVals *= 1.5f;
+                    int estSize = sizeOfUnorderedGroupByMap(estDistVals, aggregators.getEstimatedByteSize());
+                    chunk.resize(estSize);
+                }
+            }
+            return rowAggregators;
+        }
+
+        @Override
+        public RegionScanner getScanner(final RegionScanner s) {
+            // Compute final allocation
+            int estSize = sizeOfUnorderedGroupByMap(aggregateMap.size(), aggregators.getEstimatedByteSize());
+            chunk.resize(estSize);
+
+            final List<KeyValue> aggResults = new ArrayList<KeyValue>(aggregateMap.size());
+            
+            final Iterator<Map.Entry<ImmutableBytesPtr, Aggregator[]>> cacheIter =
+                    aggregateMap.entrySet().iterator();
+            while (cacheIter.hasNext()) {
+                Map.Entry<ImmutableBytesPtr, Aggregator[]> entry = cacheIter.next();
+                ImmutableBytesPtr key = entry.getKey();
+                Aggregator[] rowAggregators = entry.getValue();
+                // Generate byte array of Aggregators and set as value of row
+                byte[] value = aggregators.toBytes(rowAggregators);
+
+                if (logger.isDebugEnabled()) {
+                    logger.debug("Adding new distinct group: "
+                            + Bytes.toStringBinary(key.get(), key.getOffset(), key.getLength())
+                            + " with aggregators " + Arrays.asList(rowAggregators).toString()
+                            + " value = " + Bytes.toStringBinary(value));
+                }
+                KeyValue keyValue =
+                        KeyValueUtil.newKeyValue(key.get(), key.getOffset(), key.getLength(),
+                            SINGLE_COLUMN_FAMILY, SINGLE_COLUMN, AGG_TIMESTAMP, value, 0,
+                            value.length);
+                aggResults.add(keyValue);
+            }
+            // scanner using the non spillable, memory-only implementation
+            return new BaseRegionScanner() {
+                private int index = 0;
+
+                @Override
+                public HRegionInfo getRegionInfo() {
+                    return s.getRegionInfo();
+                }
+
+                @Override
+                public void close() throws IOException {
+                    try {
+                        s.close();
+                    } finally {
+                        InMemoryGroupByCache.this.close();
+                    }
+                }
+
+                @Override
+                public boolean next(List<KeyValue> results) throws IOException {
+                    if (index >= aggResults.size()) return false;
+                    results.add(aggResults.get(index));
+                    index++;
+                    return index < aggResults.size();
+                }
+            };
+        }
+
+        @Override
+        public int size() {
+            return aggregateMap.size();
+        }
+        
+    }
+    private static final class GroupByCacheFactory {
+        public static final GroupByCacheFactory INSTANCE = new GroupByCacheFactory();
+        
+        private GroupByCacheFactory() {
+        }
+        
+        GroupByCache newCache(RegionCoprocessorEnvironment env, ImmutableBytesWritable tenantId, ServerAggregators aggregators, int estDistVals) {
+            Configuration conf = env.getConfiguration();
+            boolean spillableEnabled =
+                    conf.getBoolean(GROUPBY_SPILLABLE_ATTRIB, DEFAULT_GROUPBY_SPILLABLE);
+            if (spillableEnabled) {
+                return new SpillableGroupByCache(env, tenantId, aggregators, estDistVals);
+            } 
+            
+            return new InMemoryGroupByCache(env, tenantId, aggregators, estDistVals);
+        }
+    }
+    /**
+     * Used for an aggregate query in which the key order does not necessarily match the group by
+     * key order. In this case, we must collect all distinct groups within a region into a map,
+     * aggregating as we go.
+     */
+    private RegionScanner scanUnordered(ObserverContext<RegionCoprocessorEnvironment> c, Scan scan,
+            final RegionScanner s, final List<Expression> expressions,
+            final ServerAggregators aggregators) throws IOException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("Grouped aggregation over unordered rows with scan " + scan
+                    + ", group by " + expressions + ", aggregators " + aggregators);
+        }
+        int estDistVals = DEFAULT_ESTIMATED_DISTINCT_VALUES;
+        byte[] estDistValsBytes = scan.getAttribute(ESTIMATED_DISTINCT_VALUES);
+        if (estDistValsBytes != null) {
+            // Allocate 1.5x estimation
+            estDistVals = Math.min(MIN_DISTINCT_VALUES, 
+                            (int) (Bytes.toInt(estDistValsBytes) * 1.5f));
+        }
+
+        RegionCoprocessorEnvironment env = c.getEnvironment();
+        Configuration conf = env.getConfiguration();
+        final boolean spillableEnabled =
+                conf.getBoolean(GROUPBY_SPILLABLE_ATTRIB, DEFAULT_GROUPBY_SPILLABLE);
+
+        GroupByCache groupByCache = 
+                GroupByCacheFactory.INSTANCE.newCache(
+                        env, ScanUtil.getTenantId(scan), 
+                        aggregators, estDistVals);
+
+        boolean success = false;
+        try {
+            boolean hasMore;
+
+            MultiKeyValueTuple result = new MultiKeyValueTuple();
+            if (logger.isDebugEnabled()) {
+                logger.debug("Spillable groupby enabled: " + spillableEnabled);
+            }
+
+            HRegion region = c.getEnvironment().getRegion();
+            MultiVersionConsistencyControl.setThreadReadPoint(s.getMvccReadPoint());
+            region.startRegionOperation();
+            try {
+                do {
+                    List<KeyValue> results = new ArrayList<KeyValue>();
+                    // 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
+                    hasMore = s.nextRaw(results, null);
+                    if (!results.isEmpty()) {
+                        result.setKeyValues(results);
+                        ImmutableBytesWritable key =
+                                TupleUtil.getConcatenatedValue(result, expressions);
+                        Aggregator[] rowAggregators = groupByCache.cache(key);
+                        // Aggregate values here
+                        aggregators.aggregate(rowAggregators, result);
+                    }
+                } while (hasMore);
+            } finally {
+                region.closeRegionOperation();
+            }
+
+            RegionScanner regionScanner = groupByCache.getScanner(s);
+
+            // Do not sort here, but sort back on the client instead
+            // The reason is that if the scan ever extends beyond a region
+            // (which can happen if we're basing our parallelization split
+            // points on old metadata), we'll get incorrect query results.
+            success = true;
+            return regionScanner;
+        } finally {
+            if (!success) {
+                Closeables.closeQuietly(groupByCache);
+            }
+        }
+    }
+
+    /**
+     * 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.
+     */
+    private RegionScanner scanOrdered(final ObserverContext<RegionCoprocessorEnvironment> c,
+            Scan scan, final RegionScanner s, final List<Expression> expressions,
+            final ServerAggregators aggregators) {
+
+        if (logger.isDebugEnabled()) {
+            logger.debug("Grouped aggregation over ordered rows with scan " + scan + ", group by "
+                    + expressions + ", aggregators " + aggregators);
+        }
+        return new BaseRegionScanner() {
+            private ImmutableBytesWritable currentKey = null;
+
+            @Override
+            public HRegionInfo getRegionInfo() {
+                return s.getRegionInfo();
+            }
+
+            @Override
+            public void close() throws IOException {
+                s.close();
+            }
+
+            @Override
+            public boolean next(List<KeyValue> results) throws IOException {
+                boolean hasMore;
+                boolean aggBoundary = false;
+                MultiKeyValueTuple result = new MultiKeyValueTuple();
+                ImmutableBytesWritable key = null;
+                Aggregator[] rowAggregators = aggregators.getAggregators();
+                HRegion region = c.getEnvironment().getRegion();
+                MultiVersionConsistencyControl.setThreadReadPoint(s.getMvccReadPoint());
+                region.startRegionOperation();
+                try {
+                    do {
+                        List<KeyValue> kvs = new ArrayList<KeyValue>();
+                        // 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
+                        hasMore = s.nextRaw(kvs, null);
+                        if (!kvs.isEmpty()) {
+                            result.setKeyValues(kvs);
+                            key = TupleUtil.getConcatenatedValue(result, expressions);
+                            aggBoundary = currentKey != null && currentKey.compareTo(key) != 0;
+                            if (!aggBoundary) {
+                                aggregators.aggregate(rowAggregators, result);
+                                if (logger.isDebugEnabled()) {
+                                    logger.debug("Row passed filters: " + kvs
+                                            + ", aggregated values: "
+                                            + Arrays.asList(rowAggregators));
+                                }
+                                currentKey = key;
+                            }
+                        }
+                    } while (hasMore && !aggBoundary);
+                } finally {
+                    region.closeRegionOperation();
+                }
+
+                if (currentKey != null) {
+                    byte[] value = aggregators.toBytes(rowAggregators);
+                    KeyValue keyValue =
+                            KeyValueUtil.newKeyValue(currentKey.get(), currentKey.getOffset(),
+                                currentKey.getLength(), SINGLE_COLUMN_FAMILY, SINGLE_COLUMN,
+                                AGG_TIMESTAMP, value, 0, value.length);
+                    results.add(keyValue);
+                    if (logger.isDebugEnabled()) {
+                        logger.debug("Adding new aggregate row: "
+                                + keyValue
+                                + ",for current key "
+                                + Bytes.toStringBinary(currentKey.get(), currentKey.getOffset(),
+                                    currentKey.getLength()) + ", aggregated values: "
+                                + Arrays.asList(rowAggregators));
+                    }
+                    // If we're at an aggregation boundary, reset the
+                    // aggregators and
+                    // aggregate with the current result (which is not a part of
+                    // the returned result).
+                    if (aggBoundary) {
+                        aggregators.reset(rowAggregators);
+                        aggregators.aggregate(rowAggregators, result);
+                        currentKey = key;
+                    }
+                }
+                // Continue if there are more
+                if (hasMore || aggBoundary) {
+                    return true;
+                }
+                currentKey = null;
+                return false;
+            }
+        };
+    }
+}


[10/51] [partial] Initial commit of master branch from github

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/schema/PDataType.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PDataType.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PDataType.java
new file mode 100644
index 0000000..8d14fb0
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PDataType.java
@@ -0,0 +1,6926 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 java.math.BigDecimal;
+import java.math.BigInteger;
+import java.math.MathContext;
+import java.math.RoundingMode;
+import java.sql.Date;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.sql.Types;
+import java.text.Format;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Base64;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.math.LongMath;
+import com.google.common.primitives.Booleans;
+import com.google.common.primitives.Doubles;
+import com.google.common.primitives.Longs;
+import org.apache.phoenix.query.KeyRange;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.DateUtil;
+import org.apache.phoenix.util.NumberUtil;
+import org.apache.phoenix.util.StringUtil;
+
+
+/**
+ * The data types of PColumns
+ *
+ * @author wmacklem
+ * @author jtaylor
+ * @since 0.1
+ *
+ * TODO: cleanup implementation to reduce copy/paste duplication
+ */
+@SuppressWarnings("rawtypes")
+public enum PDataType {
+	
+    VARCHAR("VARCHAR", Types.VARCHAR, String.class, null) {
+        @Override
+        public byte[] toBytes(Object object) {
+            // TODO: consider using avro UTF8 object instead of String
+            // so that we get get the size easily
+            if (object == null) {
+                return ByteUtil.EMPTY_BYTE_ARRAY;
+            }
+            return Bytes.toBytes((String)object);
+        }
+
+        @Override
+        public int toBytes(Object object, byte[] bytes, int offset) {
+            if (object == null) {
+                return 0;
+            }
+            byte[] b = toBytes(object); // TODO: no byte[] allocation: use CharsetEncoder
+            System.arraycopy(b, 0, bytes, offset, b.length);
+            return b.length;
+        }
+
+        @Override
+        public Object toObject(byte[] bytes, int offset, int length, PDataType actualType) {
+            if (!actualType.isCoercibleTo(this)) {
+                throw new ConstraintViolationException(actualType + " cannot be coerced to " + this);
+            }
+            return length == 0 ? null : Bytes.toString(bytes, offset, length);
+        }
+
+        @Override
+        public Object toObject(Object object, PDataType actualType) {
+            switch (actualType) {
+            case VARCHAR:
+            case CHAR:
+                return object;
+            default:
+                return super.toObject(object, actualType);
+            }
+        }
+
+        @Override
+        public boolean isCoercibleTo(PDataType targetType) {
+            // TODO: should CHAR not be here?
+            return this == targetType || targetType == CHAR || targetType == VARBINARY || targetType == BINARY;
+        }
+
+        @Override
+        public boolean isCoercibleTo(PDataType targetType, Object value) {
+            if (isCoercibleTo(targetType)) {
+                if (targetType == PDataType.CHAR) {
+                    return value != null;
+                }
+                return true;
+            }
+            return false;
+        }
+
+        @Override
+        public boolean isSizeCompatible(PDataType srcType, Object value, byte[] b,
+                Integer maxLength, Integer desiredMaxLength, Integer scale, Integer desiredScale) {
+            if (srcType == PDataType.CHAR && maxLength != null && desiredMaxLength != null) {
+                return maxLength <= desiredMaxLength;
+            }
+            return true;
+        }
+
+        @Override
+        public boolean isFixedWidth() {
+            return false;
+        }
+
+        @Override
+        public int estimateByteSize(Object o) {
+            String value = (String) o;
+            return value == null ? 1 : value.length();
+        }
+
+        @Override
+        public Integer getByteSize() {
+            return null;
+        }
+
+        @Override
+        public int compareTo(Object lhs, Object rhs, PDataType rhsType) {
+            return ((String)lhs).compareTo((String)rhs);
+        }
+
+        @Override
+        public Object toObject(String value) {
+            return value;
+        }
+
+        @Override
+        public boolean isBytesComparableWith(PDataType otherType) {
+            return super.isBytesComparableWith(otherType) || this == CHAR;
+        }
+
+        @Override
+        public String toStringLiteral(byte[] b, int offset, int length, Format formatter) {
+            while (b[length-1] == 0) {
+                length--;
+            }
+            if (formatter != null) {
+                Object o = toObject(b,offset,length);
+                return "'" + formatter.format(o) + "'";
+            }
+            return "'" + Bytes.toStringBinary(b, offset, length) + "'";
+        }
+    },
+    /**
+     * Fixed length single byte characters
+     */
+    CHAR("CHAR", Types.CHAR, String.class, null) { // Delegate to VARCHAR
+        @Override
+        public byte[] toBytes(Object object) {
+            if (object == null) {
+                throw new ConstraintViolationException(this + " may not be null");
+            }
+            byte[] b = VARCHAR.toBytes(object);
+            if (b.length != ((String) object).length()) {
+                throw new IllegalDataException("CHAR types may only contain single byte characters (" + object + ")");
+            }
+            return b;
+        }
+
+        @Override
+        public int toBytes(Object object, byte[] bytes, int offset) {
+            if (object == null) {
+                throw new ConstraintViolationException(this + " may not be null");
+            }
+            int len = VARCHAR.toBytes(object, bytes, offset);
+            if (len != ((String) object).length()) {
+                throw new IllegalDataException("CHAR types may only contain single byte characters (" + object + ")");
+            }
+            return len;
+        }
+
+        @Override
+        public Object toObject(byte[] bytes, int offset, int length, PDataType actualType) {
+            if (!actualType.isCoercibleTo(this)) { // TODO: have isCoercibleTo that takes bytes, offset?
+                throw new ConstraintViolationException(actualType + " cannot be coerced to " + this);
+            }
+            if (length == 0) {
+                return null;
+            }
+            length = StringUtil.getUnpaddedCharLength(bytes, offset, length, null);
+            String s = Bytes.toString(bytes, offset, length);
+            if (length != s.length()) {
+               throw new IllegalDataException("CHAR types may only contain single byte characters (" + s + ")");
+            }
+            return s;
+        }
+
+        @Override
+        public Object toObject(Object object, PDataType actualType) {
+            switch (actualType) {
+            case VARCHAR:
+            case CHAR:
+                return object;
+            default:
+                return super.toObject(object, actualType);
+            }
+        }
+
+        @Override
+        public boolean isCoercibleTo(PDataType targetType) {
+            return this == targetType || targetType == VARCHAR || targetType == BINARY || targetType == VARBINARY;
+        }
+
+        @Override
+        public boolean isSizeCompatible(PDataType srcType, Object value, byte[] b,
+                Integer maxLength, Integer desiredMaxLength, Integer scale, Integer desiredScale) {
+            if ((srcType == PDataType.VARCHAR && ((String)value).length() != b.length) ||
+                    (maxLength != null && desiredMaxLength != null && maxLength > desiredMaxLength)){
+                return false;
+            }
+            return true;
+        }
+
+        @Override
+        public boolean isFixedWidth() {
+            return true;
+        }
+
+        @Override
+        public Integer getByteSize() {
+            return null;
+        }
+
+        @Override
+        public Integer getMaxLength(Object o) {
+            if (o == null) {
+                return null;
+            }
+            String value = (String) o;
+            return value.length();
+        }
+
+        @Override
+        public int estimateByteSize(Object o) {
+            String value = (String) o;
+            return value.length();
+        }
+
+        @Override
+        public int compareTo(Object lhs, Object rhs, PDataType rhsType) {
+            return VARCHAR.compareTo(lhs, rhs, rhsType);
+        }
+
+        @Override
+        public Object toObject(String value) {
+            if (value == null || value.length() == 0) {
+                throw new ConstraintViolationException(this + " may not be null");
+            }
+            if (StringUtil.hasMultiByteChars(value)) {
+                throw new IllegalDataException("CHAR types may only contain single byte characters (" + value + ")");
+            }
+            return value;
+        }
+
+        @Override
+        public Integer estimateByteSizeFromLength(Integer length) {
+            return length;
+        }
+
+        @Override
+        public boolean isBytesComparableWith(PDataType otherType) {
+            return super.isBytesComparableWith(otherType) || this == VARCHAR;
+        }
+        
+        @Override
+        public String toStringLiteral(byte[] b, int offset, int length, Format formatter) {
+            return VARCHAR.toStringLiteral(b, offset, length, formatter);
+        }
+    },
+    LONG("BIGINT", Types.BIGINT, Long.class, new LongCodec()) {
+
+        @Override
+        public byte[] toBytes(Object object) {
+            byte[] b = new byte[Bytes.SIZEOF_LONG];
+            toBytes(object, b, 0);
+            return b;
+        }
+
+        @Override
+        public int toBytes(Object object, byte[] b, int o) {
+            if (object == null) {
+                throw new ConstraintViolationException(this + " may not be null");
+            }
+            return this.getCodec().encodeLong(((Number)object).longValue(), b, o);
+        }
+
+        @Override
+        public Object toObject(Object object, PDataType actualType) {
+            if (object == null) {
+                return null;
+            }
+            switch (actualType) {
+            case LONG:
+            case UNSIGNED_LONG:
+                return object;
+            case UNSIGNED_INT:
+            case INTEGER:
+                long s = (Integer)object;
+                return s;
+            case TINYINT:
+            case UNSIGNED_TINYINT:
+                s = (Byte)object;
+                return s;
+            case SMALLINT:
+            case UNSIGNED_SMALLINT:
+                s = (Short)object;
+                return s;
+            case FLOAT:
+            case UNSIGNED_FLOAT:
+                Float f = (Float)object;
+                if (f > Long.MAX_VALUE || f < Long.MIN_VALUE) {
+                    throw new IllegalDataException(actualType + " value " + f + " cannot be cast to Long without changing its value");
+                }
+                s = f.longValue();
+                return s;
+            case DOUBLE:
+            case UNSIGNED_DOUBLE:
+                Double de = (Double) object;
+                if (de > Long.MAX_VALUE || de < Long.MIN_VALUE) {
+                    throw new IllegalDataException(actualType + " value " + de + " cannot be cast to Long without changing its value");
+                }
+                s = de.longValue();
+                return s;
+            case DECIMAL:
+                BigDecimal d = (BigDecimal)object;
+                return d.longValueExact();
+            case VARBINARY:
+            case BINARY:
+                byte[] o = (byte[]) object;
+                if (o.length == Bytes.SIZEOF_LONG) {
+                    return Bytes.toLong(o);
+                } else if (o.length == Bytes.SIZEOF_INT) {
+                    int iv = Bytes.toInt(o);
+                    return (long) iv;
+                } else {
+                    throw new IllegalDataException("Bytes passed doesn't represent an integer.");
+                }
+            default:
+                return super.toObject(object, actualType);
+            }
+        }
+
+        @Override
+        public Object toObject(byte[] b, int o, int l, PDataType actualType) {
+            if (l == 0) {
+                return null;
+            }
+            switch (actualType) {
+            case LONG:
+            case UNSIGNED_LONG:
+            case INTEGER:
+            case UNSIGNED_INT:
+            case SMALLINT:
+            case UNSIGNED_SMALLINT:
+            case UNSIGNED_TINYINT:
+            case TINYINT:
+            case UNSIGNED_FLOAT:
+            case FLOAT:
+            case UNSIGNED_DOUBLE:
+            case DOUBLE:
+                return actualType.getCodec().decodeLong(b, o, null);
+            default:
+                return super.toObject(b,o,l,actualType);
+            }
+        }
+
+        @Override
+        public boolean isCoercibleTo(PDataType targetType) {
+            // In general, don't allow conversion of LONG to INTEGER. There are times when
+            // we check isComparableTo for a more relaxed check and then throw a runtime
+            // exception if we overflow
+            return this == targetType || targetType == DECIMAL
+                    || targetType == VARBINARY || targetType == BINARY
+                    || targetType == FLOAT || targetType == DOUBLE;
+        }
+
+        @Override
+        public boolean isComparableTo(PDataType targetType) {
+            return DECIMAL.isComparableTo(targetType);
+        }
+
+        @Override
+        public boolean isCoercibleTo(PDataType targetType, Object value) {
+            if (value != null) {
+                long l;
+                switch (targetType) {
+                    case UNSIGNED_DOUBLE:
+                    case UNSIGNED_FLOAT:
+                    case UNSIGNED_LONG:
+                        l = (Long) value;
+                        return l >= 0;
+                    case UNSIGNED_INT:
+                        l = (Long) value;
+                        return (l >= 0 && l <= Integer.MAX_VALUE);
+                    case INTEGER:
+                        l = (Long) value;
+                        return (l >= Integer.MIN_VALUE && l <= Integer.MAX_VALUE);
+                    case UNSIGNED_SMALLINT:
+                        l = (Long) value;
+                        return (l >= 0 && l <= Short.MAX_VALUE);
+                    case SMALLINT:
+                        l = (Long) value;
+                        return (l >=Short.MIN_VALUE && l<=Short.MAX_VALUE);
+                    case TINYINT:
+                        l = (Long)value;
+                        return (l >=Byte.MIN_VALUE && l<Byte.MAX_VALUE);
+                    case UNSIGNED_TINYINT:
+                        l = (Long)value;
+                        return (l >=0 && l<Byte.MAX_VALUE);
+                    default:
+                        break;
+                }
+            }
+            return super.isCoercibleTo(targetType, value);
+        }
+
+        @Override
+        public boolean isFixedWidth() {
+            return true;
+        }
+
+        @Override
+        public Integer getByteSize() {
+            return Bytes.SIZEOF_LONG;
+        }
+
+        @Override
+        public Integer getMaxLength(Object o) {
+            return LONG_PRECISION;
+        }
+
+        @Override
+        public Integer getScale(Object o) {
+            return ZERO;
+        }
+
+        @Override
+        public int compareTo(Object lhs, Object rhs, PDataType rhsType) {
+            if (rhsType == DECIMAL) {
+                return -((BigDecimal)rhs).compareTo(BigDecimal.valueOf(((Number)lhs).longValue()));
+            } else if (rhsType == DOUBLE || rhsType == FLOAT || rhsType == UNSIGNED_DOUBLE || rhsType == UNSIGNED_FLOAT) {
+                return Doubles.compare(((Number)lhs).doubleValue(), ((Number)rhs).doubleValue());
+            }
+            return Longs.compare(((Number)lhs).longValue(), ((Number)rhs).longValue());
+        }
+
+        @Override
+        public Object toObject(String value) {
+            if (value == null || value.length() == 0) {
+                return null;
+            }
+            try {
+                return Long.parseLong(value);
+            } catch (NumberFormatException e) {
+                throw new IllegalDataException(e);
+            }
+        }
+    },
+    INTEGER("INTEGER", Types.INTEGER, Integer.class, new IntCodec()) {
+    	
+        @Override
+        public byte[] toBytes(Object object) {
+            byte[] b = new byte[Bytes.SIZEOF_INT];
+            toBytes(object, b, 0);
+            return b;
+        }
+
+        @Override
+        public int toBytes(Object object, byte[] b, int o) {
+            if (object == null) {
+                throw new ConstraintViolationException(this + " may not be null");
+            }
+            return this.getCodec().encodeInt(((Number)object).intValue(), b, o);
+        }
+
+        @Override
+        public Object toObject(Object object, PDataType actualType) {
+            Object o = LONG.toObject(object, actualType);
+            if (!(o instanceof Long) || o == null) {
+                return o;
+            }
+            long l = (Long)o;
+            if (l < Integer.MIN_VALUE || l > Integer.MAX_VALUE) {
+                throw new IllegalDataException(actualType + " value " + l + " cannot be cast to Integer without changing its value");
+            }
+            int v = (int)l;
+            return v;
+        }
+
+        @Override
+        public Object toObject(byte[] b, int o, int l, PDataType actualType) {
+            if (l == 0) {
+                return null;
+            }
+            switch (actualType) {
+            case LONG:
+            case UNSIGNED_LONG:
+            case INTEGER:
+            case UNSIGNED_INT:
+            case SMALLINT:
+            case UNSIGNED_SMALLINT:
+            case TINYINT:
+            case UNSIGNED_TINYINT:
+            case FLOAT:
+            case UNSIGNED_FLOAT:
+            case DOUBLE:
+            case UNSIGNED_DOUBLE:
+                return actualType.getCodec().decodeInt(b, o, null);
+            default:
+                return super.toObject(b,o,l,actualType);
+            }
+        }
+
+        @Override
+        public boolean isCoercibleTo(PDataType targetType, Object value) {
+            if (value != null) {
+                switch (targetType) {
+                    case UNSIGNED_DOUBLE:
+                    case UNSIGNED_FLOAT:
+                    case UNSIGNED_LONG:
+                    case UNSIGNED_INT:
+                        int i = (Integer) value;
+                        return i >= 0;
+                    case UNSIGNED_SMALLINT:
+                        i = (Integer) value;
+                        return (i >= 0 && i <= Short.MAX_VALUE);
+                    case SMALLINT:
+                        i = (Integer) value;
+                        return (i >=Short.MIN_VALUE && i<=Short.MAX_VALUE);
+                    case TINYINT:
+                        i = (Integer)value;
+                        return (i >=Byte.MIN_VALUE && i<=Byte.MAX_VALUE);
+                    case UNSIGNED_TINYINT:
+                        i = (Integer)value;
+                        return (i >=0 && i<Byte.MAX_VALUE);
+                    default:
+                        break;
+                }
+            }
+            return super.isCoercibleTo(targetType, value);
+        }
+
+        @Override
+        public boolean isCoercibleTo(PDataType targetType) {
+            return this == targetType || LONG.isCoercibleTo(targetType);
+        }
+
+        @Override
+        public boolean isFixedWidth() {
+            return true;
+        }
+
+        @Override
+        public Integer getByteSize() {
+            return Bytes.SIZEOF_INT;
+        }
+
+        @Override
+        public Integer getMaxLength(Object o) {
+            return INT_PRECISION;
+        }
+
+        @Override
+        public Integer getScale(Object o) {
+            return ZERO;
+        }
+
+        @Override
+        public int compareTo(Object lhs, Object rhs, PDataType rhsType) {
+            return LONG.compareTo(lhs,rhs,rhsType);
+        }
+
+        @Override
+        public boolean isComparableTo(PDataType targetType) {
+            return DECIMAL.isComparableTo(targetType);
+        }
+
+        @Override
+        public Object toObject(String value) {
+            if (value == null || value.length() == 0) {
+                return null;
+            }
+            try {
+                return Integer.parseInt(value);
+            } catch (NumberFormatException e) {
+                throw new IllegalDataException(e);
+            }
+        }
+    },
+    SMALLINT("SMALLINT", Types.SMALLINT, Short.class, new ShortCodec()){
+
+      @Override
+      public int compareTo(Object lhs, Object rhs, PDataType rhsType) {
+        return LONG.compareTo(lhs, rhs, rhsType);
+      }
+      
+      @Override
+      public boolean isComparableTo(PDataType targetType) {
+          return DECIMAL.isComparableTo(targetType);
+      }
+
+      @Override
+      public boolean isFixedWidth() {
+        return true;
+      }
+
+      @Override
+      public Integer getByteSize() {
+        return Bytes.SIZEOF_SHORT;
+      }
+
+      @Override
+      public Integer getScale(Object o) {
+          return ZERO;
+      }
+      
+      @Override
+      public Integer getMaxLength(Object o) {
+          return SHORT_PRECISION;
+      }
+
+      @Override
+      public byte[] toBytes(Object object) {
+        byte[] b = new byte[Bytes.SIZEOF_SHORT];
+        toBytes(object, b, 0);
+        return b;
+      }
+
+      @Override
+      public int toBytes(Object object, byte[] bytes, int offset) {
+        if (object == null) {
+          throw new ConstraintViolationException(this + " may not be null");
+        }
+        return this.getCodec().encodeShort(((Number)object).shortValue(), bytes, offset);
+      }
+      
+      @Override
+      public Object toObject(Object object, PDataType actualType) {
+          Object o = LONG.toObject(object, actualType);
+          if (!(o instanceof Long) || o == null) {
+              return o;
+          }
+          long l = (Long)o;
+          if (l < Short.MIN_VALUE || l > Short.MAX_VALUE) {
+              throw new IllegalDataException(actualType + " value " + l + " cannot be cast to Short without changing its value");
+          }
+          short s = (short)l;
+          return s;
+      }
+
+      @Override
+      public Object toObject(byte[] b, int o, int l, PDataType actualType) {
+          if (l == 0) {
+              return null;
+          }
+          switch (actualType) {
+          case SMALLINT:
+          case UNSIGNED_SMALLINT:
+          case TINYINT:
+          case UNSIGNED_TINYINT:
+          case LONG:
+          case UNSIGNED_LONG:
+          case INTEGER:
+          case UNSIGNED_INT:
+          case FLOAT:
+          case UNSIGNED_FLOAT:
+          case DOUBLE:
+          case UNSIGNED_DOUBLE:
+              return actualType.getCodec().decodeShort(b, o, null);
+          default:
+              return super.toObject(b,o,l,actualType);
+          }
+      }
+
+      @Override
+      public Object toObject(String value) {
+        if (value == null || value.length() == 0) {
+          return null;
+        }
+        try {
+            return Short.parseShort(value);
+        } catch (NumberFormatException e) {
+            throw new IllegalDataException(e);
+        }
+      }
+      
+      @Override
+      public boolean isCoercibleTo(PDataType targetType, Object value) {
+          if (value != null) {
+              switch (targetType) {
+                  case UNSIGNED_DOUBLE:
+                  case UNSIGNED_FLOAT:
+                  case UNSIGNED_LONG:
+                  case UNSIGNED_INT:
+                  case UNSIGNED_SMALLINT:
+                      short i = (Short) value;
+                      return i >= 0;
+                  case UNSIGNED_TINYINT:
+                      i = (Short) value;
+                      return (i>=0 && i<= Byte.MAX_VALUE);
+                  case TINYINT:
+                      i = (Short)value;
+                      return (i>=Byte.MIN_VALUE && i<= Byte.MAX_VALUE);
+                  default:
+                      break;
+              }
+          }
+          return super.isCoercibleTo(targetType, value);
+      }
+
+      @Override
+      public boolean isCoercibleTo(PDataType targetType) {
+          return this == targetType || INTEGER.isCoercibleTo(targetType);
+      }
+      
+    },
+    TINYINT("TINYINT", Types.TINYINT, Byte.class, new ByteCodec()) {
+
+      @Override
+      public int compareTo(Object lhs, Object rhs, PDataType rhsType) {
+        return LONG.compareTo(lhs, rhs, rhsType);
+      }
+      
+      @Override
+      public boolean isComparableTo(PDataType targetType) {
+          return DECIMAL.isComparableTo(targetType);
+      }
+
+      @Override
+      public boolean isFixedWidth() {
+        return true;
+      }
+
+      @Override
+      public Integer getByteSize() {
+        return Bytes.SIZEOF_BYTE;
+      }
+      
+      @Override
+      public Integer getScale(Object o) {
+          return ZERO;
+      }
+      
+      @Override
+      public Integer getMaxLength(Object o) {
+          return BYTE_PRECISION;
+      }
+
+      @Override
+      public byte[] toBytes(Object object) {
+        byte[] b = new byte[Bytes.SIZEOF_BYTE];
+        toBytes(object, b, 0);
+        return b;
+      }
+
+      @Override
+      public int toBytes(Object object, byte[] bytes, int offset) {
+        if (object == null) {
+          throw new ConstraintViolationException(this + " may not be null");
+        }
+        return this.getCodec().encodeByte(((Number)object).byteValue(), bytes, offset);
+      }
+
+      @Override
+      public Object toObject(String value) {
+        if (value == null || value.length() == 0) {
+          return null;
+        }
+        try {
+          Byte b = Byte.parseByte(value);
+          return b;
+        } catch (NumberFormatException e) {
+          throw new IllegalDataException(e);
+        }
+      }
+      
+      @Override
+      public Object toObject(Object object, PDataType actualType) {
+          Object o = LONG.toObject(object, actualType);
+          if(!(o instanceof Long) || o == null) {
+              return o;
+          }
+          long l = (Long)o;
+          if (l < Byte.MIN_VALUE || l > Byte.MAX_VALUE) {
+              throw new IllegalDataException(actualType + " value " + l + " cannot be cast to Byte without changing its value");
+          }
+          return (byte)l;
+      }
+      
+      @Override
+      public Object toObject(byte[] b, int o, int l, PDataType actualType) {
+          if (l == 0) {
+              return null;
+          }
+          switch (actualType) {
+          case UNSIGNED_DOUBLE:
+          case DOUBLE:
+          case UNSIGNED_FLOAT:
+          case FLOAT:
+          case UNSIGNED_LONG:
+          case LONG:
+          case UNSIGNED_INT:
+          case INTEGER:
+          case UNSIGNED_SMALLINT:
+          case SMALLINT:
+          case UNSIGNED_TINYINT:
+          case TINYINT:
+              return actualType.getCodec().decodeByte(b, o, null);
+          default:
+              return super.toObject(b,o,l,actualType);
+          }
+      }
+      
+      @Override
+      public boolean isCoercibleTo(PDataType targetType, Object value) {
+          if (value != null) {
+              switch (targetType) {
+                  case UNSIGNED_DOUBLE:
+                  case UNSIGNED_FLOAT:
+                  case UNSIGNED_LONG:
+                  case UNSIGNED_INT:
+                  case UNSIGNED_SMALLINT:
+                  case UNSIGNED_TINYINT:
+                      byte i = (Byte) value;
+                      return i >= 0;
+                  default:
+                      break;
+              }
+          }
+          return super.isCoercibleTo(targetType, value);
+      }
+      
+      @Override
+      public boolean isCoercibleTo(PDataType targetType) {
+          return this == targetType || SMALLINT.isCoercibleTo(targetType);
+      }
+      
+    },
+    FLOAT("FLOAT", Types.FLOAT, Float.class, new FloatCodec()) {
+
+        @Override
+        public int compareTo(Object lhs, Object rhs, PDataType rhsType) {
+            return DOUBLE.compareTo(lhs, rhs, rhsType);
+        }
+
+        @Override
+        public boolean isFixedWidth() {
+            return true;
+        }
+
+        @Override
+        public Integer getByteSize() {
+            return Bytes.SIZEOF_INT;
+        }
+        
+        @Override
+        public Integer getScale(Object o) {
+            if (o == null) {
+                return null;
+            }
+            Float v = (Float) o;
+            BigDecimal bd = BigDecimal.valueOf(v);
+            return bd.scale();
+        }
+        
+        @Override
+        public Integer getMaxLength(Object o) {
+            if (o == null) {
+                return null;
+            }
+            Float v = (Float) o;
+            BigDecimal bd = BigDecimal.valueOf(v);
+            return bd.precision();
+        }
+
+        @Override
+        public byte[] toBytes(Object object) {
+            byte[] b = new byte[Bytes.SIZEOF_INT];
+            toBytes(object, b, 0);
+            return b;
+        }
+
+        @Override
+        public int toBytes(Object object, byte[] bytes, int offset) {
+            if (object == null) {
+                throw new ConstraintViolationException(this + " may not be null");
+            }
+            return this.getCodec().encodeFloat(((Number) object).floatValue(),
+                    bytes, offset);
+        }
+
+        @Override
+        public Object toObject(String value) {
+            if (value == null || value.length() == 0) {
+                return null;
+            }
+            try {
+                return Float.parseFloat(value);
+            } catch (NumberFormatException e) {
+                throw new IllegalDataException(e);
+            }
+        }
+        
+        @Override
+        public Object toObject(Object object, PDataType actualType) {
+            if (object == null) {
+                return null;
+            }
+            switch (actualType) {
+            case FLOAT:
+            case UNSIGNED_FLOAT:
+                return object;
+            case DOUBLE:
+            case UNSIGNED_DOUBLE:
+                double d = (Double)object;
+                if (Double.isNaN(d)
+                        || d == Double.POSITIVE_INFINITY
+                        || d == Double.NEGATIVE_INFINITY
+                        || (d >= -Float.MAX_VALUE && d <= Float.MAX_VALUE)) {
+                    return (float) d;
+                } else {
+                    throw new IllegalDataException(actualType + " value " + d + " cannot be cast to Float without changing its value");
+                }
+            case LONG:
+            case UNSIGNED_LONG:
+                float f = (Long)object;
+                return f;
+            case UNSIGNED_INT:
+            case INTEGER:
+                f = (Integer)object;
+                return f;
+            case TINYINT:
+            case UNSIGNED_TINYINT:
+                f = (Byte)object;
+                return f;
+            case SMALLINT:
+            case UNSIGNED_SMALLINT:
+                f = (Short)object;
+                return f;
+            case DECIMAL:
+                BigDecimal dl = (BigDecimal)object;
+                return dl.floatValue();
+            default:
+                return super.toObject(object, actualType);
+            }
+        }
+        
+        @Override
+        public Object toObject(byte[] b, int o, int l, PDataType actualType) {
+            if (l <= 0) {
+                return null;
+            }
+            switch (actualType) {
+            case FLOAT:
+            case UNSIGNED_FLOAT:
+            case DOUBLE:
+            case UNSIGNED_DOUBLE:
+            case UNSIGNED_LONG:
+            case LONG:
+            case UNSIGNED_INT:
+            case INTEGER:
+            case UNSIGNED_SMALLINT:
+            case SMALLINT:
+            case UNSIGNED_TINYINT:
+            case TINYINT:
+                return actualType.getCodec().decodeFloat(b, o, null);
+            default:
+                return super.toObject(b,o,l,actualType);
+            }
+        }
+        
+        @Override
+        public boolean isCoercibleTo(PDataType targetType, Object value) {
+            if (value != null) {
+                float f = (Float)value;
+                switch (targetType) {
+                case UNSIGNED_FLOAT:
+                    return f >= 0;
+                case UNSIGNED_LONG:
+                    return (f >= 0 && f <= Long.MAX_VALUE);
+                case LONG:
+                    return (f >= Long.MIN_VALUE && f <= Long.MAX_VALUE);
+                case UNSIGNED_INT:
+                    return (f >= 0 && f <= Integer.MAX_VALUE);
+                case INTEGER:
+                    return (f >= Integer.MIN_VALUE && f <= Integer.MAX_VALUE);
+                case UNSIGNED_SMALLINT:
+                    return (f >= 0 && f <= Short.MAX_VALUE);
+                case SMALLINT:
+                    return (f >=Short.MIN_VALUE && f<=Short.MAX_VALUE);
+                case TINYINT:
+                    return (f >=Byte.MIN_VALUE && f<Byte.MAX_VALUE);
+                case UNSIGNED_TINYINT:
+                    return (f >=0 && f<Byte.MAX_VALUE);
+                default:
+                    break;
+                }
+            }
+            return super.isCoercibleTo(targetType, value);
+        }
+        
+        @Override
+        public boolean isCoercibleTo(PDataType targetType) {
+            return this == targetType || DOUBLE.isCoercibleTo(targetType);
+        }
+        
+    },
+    DOUBLE("DOUBLE", Types.DOUBLE, Double.class, new DoubleCodec()) {
+
+        @Override
+        public int compareTo(Object lhs, Object rhs, PDataType rhsType) {
+            if (rhsType == DECIMAL) {
+                return -((BigDecimal)rhs).compareTo(BigDecimal.valueOf(((Number)lhs).doubleValue()));
+            }
+            return Doubles.compare(((Number)lhs).doubleValue(), ((Number)rhs).doubleValue());
+        }
+
+        @Override
+        public boolean isFixedWidth() {
+            return true;
+        }
+
+        @Override
+        public Integer getByteSize() {
+            return Bytes.SIZEOF_LONG;
+        }
+        
+        @Override
+        public Integer getScale(Object o) {
+            if (o == null) {
+                return null;
+            }
+            Double v = (Double) o;
+            BigDecimal bd = BigDecimal.valueOf(v);
+            return bd.scale();
+        }
+        
+        @Override
+        public Integer getMaxLength(Object o) {
+            if (o == null) {
+                return null;
+            }
+            Double v = (Double) o;
+            BigDecimal db = BigDecimal.valueOf(v);
+            return db.precision();
+        }
+
+        @Override
+        public byte[] toBytes(Object object) {
+            byte[] b = new byte[Bytes.SIZEOF_LONG];
+            toBytes(object, b, 0);
+            return b;
+        }
+
+        @Override
+        public int toBytes(Object object, byte[] bytes, int offset) {
+            if (object == null) {
+                throw new ConstraintViolationException(this + " may not be null");
+            } 
+            return this.getCodec().encodeDouble(((Number) object).doubleValue(),
+                    bytes, offset); 
+        }
+
+        @Override
+        public Object toObject(String value) {
+            if (value == null || value.length() == 0) {
+                return null;
+            }
+            try {
+                return Double.parseDouble(value);
+            } catch (NumberFormatException e) {
+                throw new IllegalDataException(e);
+            }
+        }
+        
+        @Override
+        public Object toObject(Object object, PDataType actualType) {
+            if (object == null) {
+                return null;
+            }
+            double de;
+            switch (actualType) {
+            case DOUBLE:
+            case UNSIGNED_DOUBLE:
+                return object; 
+            case FLOAT:
+            case UNSIGNED_FLOAT:
+                de = (Float)object;
+                return de;
+            case LONG:
+            case UNSIGNED_LONG:
+                de = (Long)object;
+                return de;
+            case UNSIGNED_INT:
+            case INTEGER:
+                de = (Integer)object;
+                return de;
+            case TINYINT:
+            case UNSIGNED_TINYINT:
+                de = (Byte)object;
+                return de;
+            case SMALLINT:
+            case UNSIGNED_SMALLINT:
+                de = (Short)object;
+                return de;
+            case DECIMAL:
+                BigDecimal d = (BigDecimal)object;
+                return d.doubleValue();
+            default:
+                return super.toObject(object, actualType);
+            }
+        }
+        
+        @Override
+        public Object toObject(byte[] b, int o, int l, PDataType actualType) {
+            if (l <= 0) {
+                return null;
+            }
+            switch (actualType) {
+            case DOUBLE:
+            case UNSIGNED_DOUBLE:
+            case FLOAT:
+            case UNSIGNED_FLOAT:
+            case UNSIGNED_LONG:
+            case LONG:
+            case UNSIGNED_INT:
+            case INTEGER:
+            case UNSIGNED_SMALLINT:
+            case SMALLINT:
+            case UNSIGNED_TINYINT:
+            case TINYINT:
+                return actualType.getCodec().decodeDouble(b, o, null);
+            default:
+                return super.toObject(b,o,l,actualType);
+            }
+        }
+        
+        @Override
+        public boolean isCoercibleTo(PDataType targetType, Object value) {
+            if (value != null) {
+                double d = (Double)value;
+                switch (targetType) {
+                case UNSIGNED_DOUBLE:
+                    return d >= 0;
+                case FLOAT:
+                    return Double.isNaN(d)
+                            || d == Double.POSITIVE_INFINITY
+                            || d == Double.NEGATIVE_INFINITY
+                            || (d >= -Float.MAX_VALUE && d <= Float.MAX_VALUE);
+                case UNSIGNED_FLOAT:
+                    return Double.isNaN(d) || d == Double.POSITIVE_INFINITY
+                            || (d >= 0 && d <= Float.MAX_VALUE);
+                case UNSIGNED_LONG:
+                    return (d >= 0 && d <= Long.MAX_VALUE);
+                case LONG:
+                    return (d >= Long.MIN_VALUE && d <= Long.MAX_VALUE);
+                case UNSIGNED_INT:
+                    return (d >= 0 && d <= Integer.MAX_VALUE);
+                case INTEGER:
+                    return (d >= Integer.MIN_VALUE && d <= Integer.MAX_VALUE);
+                case UNSIGNED_SMALLINT:
+                    return (d >= 0 && d <= Short.MAX_VALUE);
+                case SMALLINT:
+                    return (d >=Short.MIN_VALUE && d<=Short.MAX_VALUE);
+                case TINYINT:
+                    return (d >=Byte.MIN_VALUE && d<Byte.MAX_VALUE);
+                case UNSIGNED_TINYINT:
+                    return (d >=0 && d<Byte.MAX_VALUE);
+                default:
+                    break;
+                }
+            }
+            return super.isCoercibleTo(targetType, value);
+        }
+        
+        @Override
+        public boolean isCoercibleTo(PDataType targetType) {
+            return this == targetType || targetType == DECIMAL
+                    || targetType == VARBINARY || targetType == BINARY;
+        }
+        
+    },
+    DECIMAL("DECIMAL", Types.DECIMAL, BigDecimal.class, null) {
+        @Override
+        public byte[] toBytes(Object object) {
+            if (object == null) {
+                return ByteUtil.EMPTY_BYTE_ARRAY;
+            }
+            BigDecimal v = (BigDecimal) object;
+            v = NumberUtil.normalize(v);
+            int len = getLength(v);
+            byte[] result = new byte[Math.min(len, MAX_BIG_DECIMAL_BYTES)];
+            PDataType.toBytes(v, result, 0, len);
+            return result;
+        }
+
+        @Override
+        public int toBytes(Object object, byte[] bytes, int offset) {
+            if (object == null) {
+                return 0;
+            }
+            BigDecimal v = (BigDecimal) object;
+            v = NumberUtil.normalize(v);
+            int len = getLength(v);
+            return PDataType.toBytes(v, bytes, offset, len);
+        }
+
+        private int getLength(BigDecimal v) {
+            int signum = v.signum();
+            if (signum == 0) { // Special case for zero
+                return 1;
+            }
+            /*
+             * Size of DECIMAL includes:
+             * 1) one byte for exponent
+             * 2) one byte for terminal byte if negative
+             * 3) one byte for every two digits with the following caveats:
+             *    a) add one to round up in the case when there is an odd number of digits
+             *    b) add one in the case that the scale is odd to account for 10x of lowest significant digit
+             *       (basically done to increase the range of exponents that can be represented)
+             */
+            return (signum < 0 ? 2 : 1) + (v.precision() +  1 + (v.scale() % 2 == 0 ? 0 : 1)) / 2;
+        }
+
+        @Override
+        public int estimateByteSize(Object o) {
+            if (o == null) {
+                return 1;
+            }
+            BigDecimal v = (BigDecimal) o;
+            // TODO: should we strip zeros and round here too?
+            return Math.min(getLength(v),MAX_BIG_DECIMAL_BYTES);
+        }
+
+        @Override
+        public Integer getMaxLength(Object o) {
+            if (o == null) {
+                return null;
+            }
+            BigDecimal v = (BigDecimal) o;
+            return v.precision();
+        }
+
+        @Override
+        public Integer getScale(Object o) {
+            if (o == null) {
+                return null;
+            }
+            BigDecimal v = (BigDecimal) o;
+            return v.scale();
+        }
+
+        @Override
+        public Object toObject(byte[] b, int o, int l, PDataType actualType, ColumnModifier columnModifier) {
+            if (l == 0) {
+                return null;
+            }
+            if (columnModifier != null) {
+                b = columnModifier.apply(b, o, new byte[l], 0, l);
+                o = 0;
+            }
+            switch (actualType) {
+            case DECIMAL:
+                return toBigDecimal(b, o, l);
+            case DATE:
+            case TIME:
+            case LONG:
+            case INTEGER:
+            case SMALLINT:
+            case TINYINT:
+            case UNSIGNED_LONG:
+            case UNSIGNED_INT:
+            case UNSIGNED_SMALLINT:
+            case UNSIGNED_TINYINT:
+                return BigDecimal.valueOf(actualType.getCodec().decodeLong(b, o, null));
+            case FLOAT:
+            case UNSIGNED_FLOAT:
+                return BigDecimal.valueOf(actualType.getCodec().decodeFloat(b, o, null));
+            case DOUBLE:
+            case UNSIGNED_DOUBLE:
+                return BigDecimal.valueOf(actualType.getCodec().decodeDouble(b, o, null));
+            case TIMESTAMP:
+            case UNSIGNED_TIMESTAMP:
+                Timestamp ts = (Timestamp) actualType.toObject(b, o, l) ;
+                long millisPart = ts.getTime();
+                BigDecimal nanosPart = BigDecimal.valueOf((ts.getNanos() % QueryConstants.MILLIS_TO_NANOS_CONVERTOR)/QueryConstants.MILLIS_TO_NANOS_CONVERTOR);
+                BigDecimal value = BigDecimal.valueOf(millisPart).add(nanosPart);
+                return value;
+            default:
+                return super.toObject(b,o,l,actualType);
+            }
+        }
+
+        @Override
+        public Object toObject(Object object, PDataType actualType) {
+            if (object == null) {
+                return null;
+            }
+            switch (actualType) {
+            case INTEGER:
+            case UNSIGNED_INT:
+                return BigDecimal.valueOf((Integer)object);
+            case LONG:
+            case UNSIGNED_LONG:
+                return BigDecimal.valueOf((Long)object);
+            case SMALLINT:
+            case UNSIGNED_SMALLINT:
+                return BigDecimal.valueOf((Short)object);
+            case TINYINT:
+            case UNSIGNED_TINYINT:
+                return BigDecimal.valueOf((Byte)object);
+            case FLOAT:
+            case UNSIGNED_FLOAT:
+                return BigDecimal.valueOf((Float)object);
+            case DOUBLE:
+            case UNSIGNED_DOUBLE:
+                return BigDecimal.valueOf((Double)object);
+            case DECIMAL:
+                return object;
+            case TIMESTAMP:
+            case UNSIGNED_TIMESTAMP:
+                Timestamp ts = (Timestamp)object;
+                long millisPart = ts.getTime();
+                BigDecimal nanosPart = BigDecimal.valueOf((ts.getNanos() % QueryConstants.MILLIS_TO_NANOS_CONVERTOR)/QueryConstants.MILLIS_TO_NANOS_CONVERTOR);
+                BigDecimal value = BigDecimal.valueOf(millisPart).add(nanosPart);
+                return value;
+            default:
+                return super.toObject(object, actualType);
+            }
+        }
+
+        @Override
+        public boolean isFixedWidth() {
+            return false;
+        }
+
+        @Override
+        public Integer getByteSize() {
+            return MAX_BIG_DECIMAL_BYTES;
+        }
+
+        @Override
+        public int compareTo(Object lhs, Object rhs, PDataType rhsType) {
+            if (rhsType == DECIMAL) {
+                return ((BigDecimal)lhs).compareTo((BigDecimal)rhs);
+            }
+            return -rhsType.compareTo(rhs, lhs, this);
+        }
+
+        @Override
+        public boolean isCoercibleTo(PDataType targetType, Object value) {
+            if (value != null) {
+                BigDecimal bd;
+                switch (targetType) {
+                    case UNSIGNED_LONG:
+                    case UNSIGNED_INT:
+                    case UNSIGNED_SMALLINT:
+                    case UNSIGNED_TINYINT:
+                        bd = (BigDecimal) value;
+                        if (bd.signum() == -1) {
+                            return false;
+                        }
+                    case LONG:
+                        bd = (BigDecimal) value;
+                        try {
+                            bd.longValueExact();
+                            return true;
+                        } catch (ArithmeticException e) {
+                            return false;
+                        }
+                    case INTEGER:
+                        bd = (BigDecimal) value;
+                        try {
+                            bd.intValueExact();
+                            return true;
+                        } catch (ArithmeticException e) {
+                            return false;
+                        }
+                    case SMALLINT:
+                        bd = (BigDecimal) value;
+                        try {
+                            bd.shortValueExact();
+                            return true;
+                        } catch (ArithmeticException e) {
+                            return false;
+                        }
+                    case TINYINT:
+                        bd = (BigDecimal) value;
+                        try {
+                            bd.byteValueExact();
+                            return true;
+                        } catch (ArithmeticException e) {
+                            return false;
+                        }
+                    case UNSIGNED_FLOAT:
+                        bd = (BigDecimal) value;
+                        try {
+                            BigDecimal maxFloat = MAX_FLOAT_AS_BIG_DECIMAL;
+                            boolean isNegtive = (bd.signum() == -1);
+                            return bd.compareTo(maxFloat)<=0 && !isNegtive;
+                        } catch(Exception e) {
+                            return false;
+                        }
+                    case FLOAT:
+                        bd = (BigDecimal) value;
+                        try {
+                            BigDecimal maxFloat = MAX_FLOAT_AS_BIG_DECIMAL;
+                            // Float.MIN_VALUE should not be used here, as this is the
+                            // smallest in terms of closest to zero.
+                            BigDecimal minFloat = MIN_FLOAT_AS_BIG_DECIMAL;
+                            return bd.compareTo(maxFloat)<=0 && bd.compareTo(minFloat)>=0;
+                        } catch(Exception e) {
+                            return false;
+                        }
+                    case UNSIGNED_DOUBLE:
+                        bd = (BigDecimal) value;
+                        try {
+                            BigDecimal maxDouble = MAX_DOUBLE_AS_BIG_DECIMAL;
+                            boolean isNegtive = (bd.signum() == -1);
+                            return bd.compareTo(maxDouble)<=0 && !isNegtive;
+                        } catch(Exception e) {
+                            return false;
+                        }
+                    case DOUBLE:
+                        bd = (BigDecimal) value;
+                        try {
+                            BigDecimal maxDouble = MAX_DOUBLE_AS_BIG_DECIMAL;
+                            BigDecimal minDouble = MIN_DOUBLE_AS_BIG_DECIMAL;
+                            return bd.compareTo(maxDouble)<=0 && bd.compareTo(minDouble)>=0;
+                        } catch(Exception e) {
+                            return false;
+                        }
+                    default:
+                        break;
+                }
+            }
+            return super.isCoercibleTo(targetType, value);
+        }
+
+        @Override
+        public boolean isSizeCompatible(PDataType srcType, Object value, byte[] b, Integer maxLength,
+        		Integer desiredMaxLength, Integer scale, Integer desiredScale) {
+            // Get precision and scale if it is not already passed in and either the object or byte values
+            // is meaningful.
+            if (maxLength == null && scale == null) {
+                if (value != null) {
+                    BigDecimal v = (BigDecimal) value;
+                    maxLength = v.precision();
+                    scale = v.scale();
+                } else if (b != null && b.length > 0) {
+                    int[] v = getDecimalPrecisionAndScale(b, 0, b.length);
+                    maxLength = v[0];
+                    scale = v[1];
+                } else {
+                    // the value does not contains maxLength nor scale. Just return true.
+                    return true;
+                }
+            }
+            if (desiredMaxLength != null && desiredScale != null && maxLength != null && scale != null &&
+            		((desiredScale == null && desiredMaxLength < maxLength) || 
+            				(desiredMaxLength - desiredScale) < (maxLength - scale))) {
+                return false;
+            }
+            return true;
+        }
+
+        @Override
+        public byte[] coerceBytes(byte[] b, Object object, PDataType actualType, Integer maxLength, Integer scale,
+                Integer desiredMaxLength, Integer desiredScale) {
+            if (desiredScale == null) {
+                // deiredScale not available, or we do not have scale requirement, delegate to parents.
+                return super.coerceBytes(b, object, actualType);
+            }
+            if (scale == null) {
+                if (object != null) {
+                    BigDecimal v = (BigDecimal) object;
+                    scale = v.scale();
+                } else if (b != null && b.length > 0) {
+                    int[] v = getDecimalPrecisionAndScale(b, 0, b.length);
+                    scale = v[1];
+                } else {
+                    // Neither the object value nor byte value is meaningful, delegate to super.
+                    return super.coerceBytes(b, object, actualType);
+                }
+            }
+            if (this == actualType && scale <= desiredScale) {
+                // No coerce and rescale necessary
+                return b;
+            } else {
+                BigDecimal decimal;
+                // Rescale is necessary.
+                if (object != null) { // value object is passed in.
+                    decimal = (BigDecimal) toObject(object, actualType);
+                } else { // only value bytes is passed in, need to convert to object first.
+                    decimal = (BigDecimal) toObject(b);
+                }
+                decimal = decimal.setScale(desiredScale, BigDecimal.ROUND_DOWN);
+                return toBytes(decimal);
+            }
+        }
+
+        @Override
+        public Object toObject(String value) {
+            if (value == null || value.length() == 0) {
+                return null;
+            }
+            try {
+                return new BigDecimal(value);
+            } catch (NumberFormatException e) {
+                throw new IllegalDataException(e);
+            }
+        }
+
+        @Override
+        public Integer estimateByteSizeFromLength(Integer length) {
+            // No association of runtime byte size from decimal precision.
+            return null;
+        }
+    },
+    TIMESTAMP("TIMESTAMP", Types.TIMESTAMP, Timestamp.class, new DateCodec()) {
+
+        @Override
+        public byte[] toBytes(Object object) {
+            if (object == null) {
+                throw new ConstraintViolationException(this + " may not be null");
+            }
+            byte[] bytes = new byte[getByteSize()];
+            toBytes(object, bytes, 0);
+            return bytes;
+        }
+
+        @Override
+        public int toBytes(Object object, byte[] bytes, int offset) {
+            if (object == null) {
+                throw new ConstraintViolationException(this + " may not be null");
+            }
+            Timestamp value = (Timestamp)object;
+            DATE.getCodec().encodeLong(value.getTime(), bytes, offset);
+            
+            /*
+             * By not getting the stuff that got spilled over from the millis part,
+             * it leaves the timestamp's byte representation saner - 8 bytes of millis | 4 bytes of nanos.
+             * Also, it enables timestamp bytes to be directly compared with date/time bytes.   
+             */
+            Bytes.putInt(bytes, offset + Bytes.SIZEOF_LONG, value.getNanos() % 1000000);  
+            return getByteSize();
+        }
+
+        @Override
+        public Object toObject(Object object, PDataType actualType) {
+            if (object == null) {
+                return null;
+            }
+            switch (actualType) {
+            case DATE:
+            case TIME:
+            case UNSIGNED_DATE:
+            case UNSIGNED_TIME:
+                return new Timestamp(((java.util.Date)object).getTime());
+            case TIMESTAMP:
+            case UNSIGNED_TIMESTAMP:
+                return object;
+            default:
+                return super.toObject(object, actualType);
+            }
+        }
+
+        @Override
+        public Object toObject(byte[] b, int o, int l, PDataType actualType, ColumnModifier columnModifier) {
+            if (actualType == null || l == 0) {
+                return null;
+            }
+            if (columnModifier != null) {
+                b = columnModifier.apply(b, o, new byte[l], 0, l);
+                o = 0;
+            }
+            switch (actualType) {
+            case TIMESTAMP:
+            case UNSIGNED_TIMESTAMP:
+                long millisDeserialized = (actualType == TIMESTAMP ? DATE : UNSIGNED_DATE).getCodec().decodeLong(b, o, null);
+                Timestamp v = new Timestamp(millisDeserialized);
+                int nanosDeserialized = Bytes.toInt(b, o + Bytes.SIZEOF_LONG, Bytes.SIZEOF_INT);
+                /*
+                 * There was a bug in serialization of timestamps which was causing the sub-second millis part
+                 * of time stamp to be present both in the LONG and INT bytes. Having the <100000 check
+                 * makes this serialization fix backward compatible.
+                 */
+                v.setNanos(nanosDeserialized < 1000000 ? v.getNanos() + nanosDeserialized : nanosDeserialized);
+                return v;
+            case DATE:
+            case TIME:
+            case UNSIGNED_DATE:
+            case UNSIGNED_TIME:
+                return new Timestamp(actualType.getCodec().decodeLong(b, o, null));
+            case DECIMAL:
+                BigDecimal bd = (BigDecimal) actualType.toObject(b, o, l);
+                long ms = bd.longValue();
+                int nanos = (bd.remainder(BigDecimal.ONE).multiply(QueryConstants.BD_MILLIS_NANOS_CONVERSION)).intValue();
+                v = DateUtil.getTimestamp(ms, nanos);
+                return v;
+            default:
+                throw new ConstraintViolationException(actualType + " cannot be coerced to " + this);
+            }
+        }
+        
+        @Override
+        public boolean isCastableTo(PDataType targetType) {
+            return DATE.isCastableTo(targetType);
+        }
+
+       @Override
+        public boolean isCoercibleTo(PDataType targetType) {
+            return this == targetType || targetType == VARBINARY || targetType == BINARY;
+        }
+
+        @Override
+        public boolean isCoercibleTo(PDataType targetType, Object value) {
+            return DATE.isCoercibleTo(targetType, value) && (targetType == UNSIGNED_TIMESTAMP || targetType == this || ((Timestamp)value).getNanos() == 0);
+        }
+        
+        @Override
+        public boolean isFixedWidth() {
+            return true;
+        }
+
+        @Override
+        public Integer getByteSize() {
+            return MAX_TIMESTAMP_BYTES;
+        }
+
+        @Override
+        public int compareTo(Object lhs, Object rhs, PDataType rhsType) {
+            if (rhsType == TIMESTAMP || rhsType == UNSIGNED_TIMESTAMP) {
+                return ((Timestamp)lhs).compareTo((Timestamp)rhs);
+            }
+            int c = ((Date)rhs).compareTo((Date)lhs);
+            if (c != 0) return c;
+            return ((Timestamp)lhs).getNanos();
+        }
+
+        @Override
+        public Object toObject(String value) {
+            if (value == null || value.length() == 0) {
+                return null;
+            }
+            return DateUtil.parseTimestamp(value);
+        }
+        
+        @Override
+        public String toStringLiteral(byte[] b, int offset, int length, Format formatter) {
+            Timestamp value = (Timestamp)toObject(b,offset,length);
+            if (formatter == null || formatter == DateUtil.DEFAULT_DATE_FORMATTER) {
+                // If default formatter has not been overridden,
+                // use one that displays milliseconds.
+                formatter = DateUtil.DEFAULT_MS_DATE_FORMATTER;
+            }
+            return "'" + super.toStringLiteral(b, offset, length, formatter) + "." + value.getNanos() + "'";
+        }
+        
+        @Override
+        public int getNanos(ImmutableBytesWritable ptr, ColumnModifier cm) {
+            int nanos = PDataType.UNSIGNED_INT.getCodec().decodeInt(ptr.get(), ptr.getOffset() + PDataType.LONG.getByteSize(), cm);
+            return nanos;
+        }
+        
+        @Override
+        public long getMillis(ImmutableBytesWritable ptr, ColumnModifier cm) {
+            long millis = PDataType.LONG.getCodec().decodeLong(ptr.get(),ptr.getOffset(), cm);
+            return millis;
+        }
+
+    },
+    TIME("TIME", Types.TIME, Time.class, new DateCodec()) {
+
+        @Override
+        public byte[] toBytes(Object object) {
+            return DATE.toBytes(object);
+        }
+
+        @Override
+        public int toBytes(Object object, byte[] bytes, int offset) {
+            return DATE.toBytes(object, bytes, offset);
+        }
+
+        @Override
+        public Object toObject(byte[] b, int o, int l, PDataType actualType) {
+            if (l == 0) {
+                return null;
+            }
+            switch (actualType) {
+            case TIMESTAMP:
+            case UNSIGNED_TIMESTAMP:
+            case DATE:
+            case TIME:
+            case UNSIGNED_DATE:
+            case UNSIGNED_TIME:
+                return new Time(actualType.getCodec().decodeLong(b, o, null));
+            default:
+                throw new ConstraintViolationException(actualType + " cannot be coerced to " + this);
+            }
+        }
+
+        @Override
+        public Object toObject(Object object, PDataType actualType) {
+            if (object == null) {
+                return null;
+            }
+            switch (actualType) {
+            case DATE:
+            case UNSIGNED_DATE:
+                return new Time(((Date)object).getTime());
+            case TIMESTAMP:
+            case UNSIGNED_TIMESTAMP:
+                return new Time(((Timestamp)object).getTime());
+            case TIME:
+            case UNSIGNED_TIME:
+                return object;
+            default:
+                return super.toObject(object, actualType);
+            }
+        }
+
+        @Override
+        public boolean isCastableTo(PDataType targetType) {
+            return DATE.isCastableTo(targetType);
+        }
+
+        @Override
+        public boolean isCoercibleTo(PDataType targetType) {
+            return this == targetType || targetType == DATE || targetType == TIMESTAMP
+                    || targetType == VARBINARY || targetType == BINARY;
+        }
+
+        @Override
+        public boolean isCoercibleTo(PDataType targetType, Object value) {
+            return DATE.isCoercibleTo(targetType, value);
+        }
+        
+        @Override
+        public boolean isFixedWidth() {
+            return true;
+        }
+
+        @Override
+        public Integer getByteSize() {
+            return Bytes.SIZEOF_LONG;
+        }
+
+        @Override
+        public int compareTo(Object lhs, Object rhs, PDataType rhsType) {
+            return DATE.compareTo(lhs, rhs, rhsType);
+        }
+
+        @Override
+        public Object toObject(String value) {
+            if (value == null || value.length() == 0) {
+                return null;
+            }
+            return DateUtil.parseTime(value);
+        }
+
+        @Override
+        public boolean isBytesComparableWith(PDataType otherType) {
+            return super.isBytesComparableWith(otherType) ||  this == DATE;
+        }
+        
+        @Override
+        public String toStringLiteral(byte[] b, int offset, int length, Format formatter) {
+            // TODO: different default formatter for TIME?
+            return DATE.toStringLiteral(b, offset, length, formatter);
+        }
+    },
+    DATE("DATE", Types.DATE, Date.class, new DateCodec()) { // After TIMESTAMP and DATE to ensure toLiteral finds those first
+
+        @Override
+        public byte[] toBytes(Object object) {
+            if (object == null) {
+                throw new ConstraintViolationException(this + " may not be null");
+            }
+            byte[] bytes = new byte[getByteSize()];
+            toBytes(object, bytes, 0);
+            return bytes;
+        }
+
+        @Override
+        public int toBytes(Object object, byte[] bytes, int offset) {
+            if (object == null) {
+                throw new ConstraintViolationException(this + " may not be null");
+            }
+            getCodec().encodeLong(((java.util.Date)object).getTime(), bytes, offset);
+            return this.getByteSize();
+        }
+
+        @Override
+        public Object toObject(Object object, PDataType actualType) {
+            if (object == null) {
+                return null;
+            }
+            switch (actualType) {
+            case TIME:
+            case UNSIGNED_TIME:
+                return new Date(((Time)object).getTime());
+            case TIMESTAMP:
+            case UNSIGNED_TIMESTAMP:
+                return new Date(((Timestamp)object).getTime());
+            case DATE:
+            case UNSIGNED_DATE:
+                return object;
+            default:
+                return super.toObject(object, actualType);
+            }
+        }
+
+        @Override
+        public Object toObject(byte[] b, int o, int l, PDataType actualType) {
+            if (l == 0) {
+                return null;
+            }
+            switch (actualType) {
+            case TIMESTAMP:
+            case UNSIGNED_TIMESTAMP:
+            case DATE:
+            case TIME:
+            case UNSIGNED_DATE:
+            case UNSIGNED_TIME:
+                return new Date(actualType.getCodec().decodeLong(b, o, null));
+            default:
+                throw new ConstraintViolationException(actualType + " cannot be coerced to " + this);
+            }
+        }
+
+        @Override
+        public boolean isCastableTo(PDataType targetType) {
+            return super.isCastableTo(targetType) || DECIMAL.isCastableTo(targetType);
+        }
+
+        @Override
+        public boolean isCoercibleTo(PDataType targetType) {
+            return this == targetType || targetType == TIME || targetType == TIMESTAMP
+                    || targetType == VARBINARY || targetType == BINARY;
+        }
+
+        @Override
+        public boolean isCoercibleTo(PDataType targetType, Object value) {
+            if (value != null) {
+                switch (targetType) {
+                    case UNSIGNED_DATE:
+                    case UNSIGNED_TIME:
+                    case UNSIGNED_TIMESTAMP:
+                        return ((java.util.Date)value).getTime() >= 0;
+                    default:
+                        break;
+                }
+            }
+            return super.isCoercibleTo(targetType, value);
+        }
+
+        @Override
+        public boolean isFixedWidth() {
+            return true;
+        }
+
+        @Override
+        public Integer getByteSize() {
+            return Bytes.SIZEOF_LONG;
+        }
+
+        @Override
+        public int compareTo(Object lhs, Object rhs, PDataType rhsType) {
+            if (rhsType == TIMESTAMP || rhsType == UNSIGNED_TIMESTAMP) {
+                return -rhsType.compareTo(rhs, lhs, TIME);
+            }
+            return ((java.util.Date)rhs).compareTo((java.util.Date)lhs);
+        }
+
+        @Override
+        public Object toObject(String value) {
+            if (value == null || value.length() == 0) {
+                return null;
+            }
+            return DateUtil.parseDate(value);
+        }
+
+        @Override
+        public boolean isBytesComparableWith(PDataType otherType) {
+            return super.isBytesComparableWith(otherType) || this == TIME;
+        }
+        
+        @Override
+        public String toStringLiteral(byte[] b, int offset, int length, Format formatter) {
+            if (formatter == null || formatter == DateUtil.DEFAULT_DATE_FORMATTER) {
+                // If default formatter has not been overridden,
+                // use one that displays milliseconds.
+                formatter = DateUtil.DEFAULT_MS_DATE_FORMATTER;
+            }
+            return "'" + super.toStringLiteral(b, offset, length, formatter) + "'";
+        }
+        
+        @Override
+        public void coerceBytes(ImmutableBytesWritable ptr, PDataType actualType, ColumnModifier actualModifier, ColumnModifier expectedModifier) {
+            if (ptr.getLength() > 0 && actualType  == PDataType.TIMESTAMP && actualModifier == expectedModifier) {
+                ptr.set(ptr.get(), ptr.getOffset(), getByteSize());
+                return;
+            }
+            super.coerceBytes(ptr, actualType, actualModifier, expectedModifier);
+        }
+    },
+    UNSIGNED_TIMESTAMP("UNSIGNED_TIMESTAMP", 19, Timestamp.class, null) {
+
+        @Override
+        public byte[] toBytes(Object object) {
+            if (object == null) {
+                throw new ConstraintViolationException(this + " may not be null");
+            }
+            byte[] bytes = new byte[getByteSize()];
+            toBytes(object, bytes, 0);
+            return bytes;
+        }
+
+        @Override
+        public int toBytes(Object object, byte[] bytes, int offset) {
+            if (object == null) {
+                throw new ConstraintViolationException(this + " may not be null");
+            }
+            Timestamp value = (Timestamp)object;
+            UNSIGNED_DATE.getCodec().encodeLong(value.getTime(), bytes, offset);
+            
+            /*
+             * By not getting the stuff that got spilled over from the millis part,
+             * it leaves the timestamp's byte representation saner - 8 bytes of millis | 4 bytes of nanos.
+             * Also, it enables timestamp bytes to be directly compared with date/time bytes.   
+             */
+            Bytes.putInt(bytes, offset + Bytes.SIZEOF_LONG, value.getNanos() % 1000000);  
+            return getByteSize();
+        }
+
+        @Override
+        public Object toObject(Object object, PDataType actualType) {
+            return TIMESTAMP.toObject(object, actualType);
+        }
+
+        @Override
+        public Object toObject(byte[] b, int o, int l, PDataType actualType) {
+            return TIMESTAMP.toObject(b, o, l, actualType);
+        }
+        
+        @Override
+        public boolean isCastableTo(PDataType targetType) {
+            return TIMESTAMP.isCastableTo(targetType);
+        }
+
+        @Override
+        public boolean isCoercibleTo(PDataType targetType) {
+            return this == targetType || targetType == TIMESTAMP || targetType == VARBINARY || targetType == BINARY;
+        }
+
+        @Override
+        public boolean isFixedWidth() {
+            return true;
+        }
+
+        @Override
+        public Integer getByteSize() {
+            return TIMESTAMP.getByteSize();
+        }
+
+        @Override
+        public int compareTo(Object lhs, Object rhs, PDataType rhsType) {
+            return TIMESTAMP.compareTo(lhs, rhs, rhsType);
+        }
+
+        @Override
+        public Object toObject(String value) {
+            return TIMESTAMP.toObject(value);
+        }
+        
+        @Override
+        public String toStringLiteral(byte[] b, int offset, int length, Format formatter) {
+            Timestamp value = (Timestamp)toObject(b,offset,length);
+            if (formatter == null || formatter == DateUtil.DEFAULT_DATE_FORMATTER) {
+                // If default formatter has not been overridden,
+                // use one that displays milliseconds.
+                formatter = DateUtil.DEFAULT_MS_DATE_FORMATTER;
+            }
+            return "'" + super.toStringLiteral(b, offset, length, formatter) + "." + value.getNanos() + "'";
+        }
+        
+        @Override
+        public int getNanos(ImmutableBytesWritable ptr, ColumnModifier cm) {
+            int nanos = PDataType.UNSIGNED_INT.getCodec().decodeInt(ptr.get(), ptr.getOffset() + PDataType.LONG.getByteSize(), cm);
+            return nanos;
+        }
+        
+        @Override
+        public long getMillis(ImmutableBytesWritable ptr, ColumnModifier cm) {
+            long millis = PDataType.UNSIGNED_LONG.getCodec().decodeLong(ptr.get(),ptr.getOffset(), cm);
+            return millis;
+        }
+    },
+    UNSIGNED_TIME("UNSIGNED_TIME", 18, Time.class, new UnsignedDateCodec()) {
+
+        @Override
+        public byte[] toBytes(Object object) {
+            return UNSIGNED_DATE.toBytes(object);
+        }
+
+        @Override
+        public int toBytes(Object object, byte[] bytes, int offset) {
+            return UNSIGNED_DATE.toBytes(object, bytes, offset);
+        }
+
+        @Override
+        public Object toObject(byte[] b, int o, int l, PDataType actualType) {
+            return TIME.toObject(b, o, l, actualType);
+        }
+
+        @Override
+        public Object toObject(Object object, PDataType actualType) {
+            return TIME.toObject(object, actualType);
+        }
+
+        @Override
+        public boolean isCastableTo(PDataType targetType) {
+            return TIME.isCastableTo(targetType);
+        }
+
+        @Override
+        public boolean isCoercibleTo(PDataType targetType) {
+            return this == targetType || targetType == TIME || targetType == UNSIGNED_DATE || targetType == DATE || targetType == UNSIGNED_TIMESTAMP || targetType == TIMESTAMP
+                    || targetType == VARBINARY || targetType == BINARY;
+        }
+
+        @Override
+        public boolean isFixedWidth() {
+            return true;
+        }
+
+        @Override
+        public Integer getByteSize() {
+            return Bytes.SIZEOF_LONG;
+        }
+
+        @Override
+        public int compareTo(Object lhs, Object rhs, PDataType rhsType) {
+            return TIME.compareTo(lhs, rhs, rhsType);
+        }
+
+        @Override
+        public Object toObject(String value) {
+            return TIME.toObject(value);
+        }
+
+        @Override
+        public boolean isBytesComparableWith(PDataType otherType) {
+            return super.isBytesComparableWith(otherType) ||  this == UNSIGNED_DATE;
+        }
+        
+        @Override
+        public String toStringLiteral(byte[] b, int offset, int length, Format formatter) {
+            return UNSIGNED_DATE.toStringLiteral(b, offset, length, formatter);
+        }
+    },
+    UNSIGNED_DATE("UNSIGNED_DATE", 19, Date.class, new UnsignedDateCodec()) { // After TIMESTAMP and DATE to ensure toLiteral finds those first
+
+        @Override
+        public byte[] toBytes(Object object) {
+            if (object == null) {
+                throw new ConstraintViolationException(this + " may not be null");
+            }
+            byte[] bytes = new byte[getByteSize()];
+            toBytes(object, bytes, 0);
+            return bytes;
+        }
+
+        @Override
+        public int toBytes(Object object, byte[] bytes, int offset) {
+            if (object == null) {
+                throw new ConstraintViolationException(this + " may not be null");
+            }
+            getCodec().encodeLong(((java.util.Date)object).getTime(), bytes, offset);
+            return this.getByteSize();
+        }
+
+        @Override
+        public Object toObject(Object object, PDataType actualType) {
+            return DATE.toObject(object, actualType);
+        }
+
+        @Override
+        public Object toObject(byte[] b, int o, int l, PDataType actualType) {
+            return DATE.toObject(b,o,l,actualType);
+        }
+
+        @Override
+        public boolean isCastableTo(PDataType targetType) {
+            return DATE.isCastableTo(targetType);
+        }
+
+        @Override
+        public boolean isCoercibleTo(PDataType targetType) {
+            return this == targetType || targetType == DATE || targetType == UNSIGNED_TIME || targetType == TIME || targetType == UNSIGNED_TIMESTAMP || targetType == TIMESTAMP
+                    || targetType == VARBINARY || targetType == BINARY;
+        }
+
+        @Override
+        public boolean isFixedWidth() {
+            return true;
+        }
+
+        @Override
+        public Integer getByteSize() {
+            return DATE.getByteSize();
+        }
+
+        @Override
+        public int compareTo(Object lhs, Object rhs, PDataType rhsType) {
+            return DATE.compareTo(lhs, rhs, rhsType);
+        }
+
+        @Override
+        public Object toObject(String value) {
+            return DATE.toObject(value);
+        }
+
+        @Override
+        public boolean isBytesComparableWith(PDataType otherType) {
+            return super.isBytesComparableWith(otherType) || this == UNSIGNED_TIME;
+        }
+        
+        @Override
+        public String toStringLiteral(byte[] b, int offset, int length, Format formatter) {
+            // Can't delegate, as the super.toStringLiteral calls this.toBytes
+            if (formatter == null || formatter == DateUtil.DEFAULT_DATE_FORMATTER) {
+                // If default formatter has not been overridden,
+                // use one that displays milliseconds.
+                formatter = DateUtil.DEFAULT_MS_DATE_FORMATTER;
+            }
+            return "'" + super.toStringLiteral(b, offset, length, formatter) + "'";
+        }
+        
+        @Override
+        public void coerceBytes(ImmutableBytesWritable ptr, PDataType actualType, ColumnModifier actualModifier, ColumnModifier expectedModifier) {
+            if (ptr.getLength() > 0 && actualType  == PDataType.UNSIGNED_TIMESTAMP && actualModifier == expectedModifier) {
+                ptr.set(ptr.get(), ptr.getOffset(), getByteSize());
+                return;
+            }
+            super.coerceBytes(ptr, actualType, actualModifier, expectedModifier);
+        }
+    },
+    /**
+     * Unsigned long type that restricts values to be from 0 to {@link java.lang.Long#MAX_VALUE} inclusive. May be used to map to existing HTable values created through {@link org.apache.hadoop.hbase.util.Bytes#toBytes(long)}
+     * as long as all values are non negative (the leading sign bit of negative numbers would cause them to sort ahead of positive numbers when
+     * they're used as part of the row key when using the HBase utility methods).
+     */
+    UNSIGNED_LONG("UNSIGNED_LONG", 10 /* no constant available in Types */, Long.class, new UnsignedLongCodec()) {
+
+        @Override
+        public byte[] toBytes(Object object) {
+            byte[] b = new byte[Bytes.SIZEOF_LONG];
+            toBytes(object, b, 0);
+            return b;
+        }
+
+        @Override
+        public int toBytes(Object object, byte[] b, int o) {
+            if (object == null) {
+                throw new ConstraintViolationException(this + " may not be null");
+            }
+            return this.getCodec().encodeLong(((Number)object).longValue(), b, o);
+        }
+
+        @Override
+        public Object toObject(Object object, PDataType actualType) {
+            if (object == null) {
+                return null;
+            }
+            switch (actualType) {
+            case LONG:
+            case UNSIGNED_LONG:
+                long v = (Long) object;
+                if (v < 0) {
+                    throw new IllegalDataException("Value may not be negative(" + v + ")");
+                }
+                return v;
+            case UNSIGNED_INT:
+            case INTEGER:
+                v = (Integer) object;
+                if (v < 0) {
+                    throw new IllegalDataException("Value may not be negative(" + v + ")");
+                }
+                return v;
+            case SMALLINT:
+            case UNSIGNED_SMALLINT:
+                v = (Short) object;
+                if (v < 0) {
+                    throw new IllegalDataException("Value may not be negative(" + v + ")");
+                }
+                return v;
+            case UNSIGNED_TINYINT:
+            case TINYINT:
+                v = (Byte) object;
+                if (v < 0) {
+                    throw new IllegalDataException("Value may not be negative(" + v + ")");
+                }
+                return v;
+            case UNSIGNED_FLOAT:
+            case FLOAT:
+                Float f = (Float) object;
+                v = f.longValue();
+                if (v < 0) {
+                    throw new IllegalDataException("Value may not be negative(" + v + ")");
+                }
+                return v;
+            case UNSIGNED_DOUBLE:
+            case DOUBLE:
+                Double de = (Double) object;
+                v = de.longValue();
+                if (v < 0) {
+                    throw new IllegalDataException("Value may not be negative(" + v + ")");
+                }
+                return v;
+            case DECIMAL:
+                BigDecimal d = (BigDecimal) object;
+                if (d.signum() == -1) {
+                    throw new IllegalDataException("Value may not be negative(" + d + ")");
+                }
+                return d.longValueExact();
+            default:
+                return super.toObject(object, actualType);
+            }
+        }
+
+        @Override
+        public Object toObject(byte[] b, int o, int l, PDataType actualType) {
+            if (l == 0) {
+                return null;
+            }
+            switch (actualType) {
+            case INTEGER:
+            case LONG:
+            case UNSIGNED_LONG:
+            case UNSIGNED_INT:
+            case SMALLINT:
+            case UNSIGNED_SMALLINT:
+            case TINYINT:
+            case UNSIGNED_TINYINT:
+            case FLOAT:
+            case UNSIGNED_FLOAT:
+            case DOUBLE:
+            case UNSIGNED_DOUBLE:
+                return actualType.getCodec().decodeLong(b, o, null);
+            default:
+                return super.toObject(b,o,l,actualType);
+            }
+        }
+
+        @Override
+        public boolean isCoercibleTo(PDataType targetType) {
+            return this == targetType || targetType == LONG || targetType == DECIMAL
+                    || targetType == VARBINARY || targetType == BINARY 
+                    || targetType == FLOAT || targetType == DOUBLE || targetType == UNSIGNED_FLOAT
+                    || targetType == UNSIGNED_DOUBLE;
+        }
+
+        @Override
+        public boolean isCoercibleTo(PDataType targetType, Object value) {
+            if (value != null) {
+                switch (targetType) {
+                    case UNSIGNED_INT:
+                    case INTEGER:
+                        long l = (Long) value;
+                        return (l >= Integer.MIN_VALUE && l <= Integer.MAX_VALUE);
+                    case UNSIGNED_SMALLINT:
+                    case SMALLINT:
+                        long s = (Long)value;
+                        return (s>=Short.MIN_VALUE && s<=Short.MAX_VALUE);
+                    case TINYINT:
+                        long t = (Long)value;
+                        return (t>=Byte.MIN_VALUE && t<=Byte.MAX_VALUE);
+                    case UNSIGNED_TINYINT:
+                        t = (Long)value;
+                        return (t>=0 && t<=Byte.MAX_VALUE);
+                    default:
+                        break;
+                }
+            }
+            return super.isCoercibleTo(targetType, value);
+        }
+
+        @Override
+        public boolean isFixedWidth() {
+            return true;
+        }
+
+        @Override
+        public Integer getByteSize() {
+            return Bytes.SIZEOF_LONG;
+        }
+
+        @Override
+        public int compareTo(Object lhs, Object rhs, PDataType rhsType) {
+            if (rhsType == DECIMAL) {
+                return -((BigDecimal)rhs).compareTo(BigDecimal.valueOf(((Number)lhs).longValue()));
+            } else if (rhsType == DOUBLE || rhsType == FLOAT || rhsType == UNSIGNED_DOUBLE || rhsType == UNSIGNED_FLOAT) {
+                return Doubles.compare(((Number)lhs).doubleValue(), ((Number)rhs).doubleValue());
+            }
+            return Longs.compare(((Number)lhs).longValue(), ((Number)rhs).longValue());
+        }
+
+        @Override
+        public boolean isComparableTo(PDataType targetType) {
+            return DECIMAL.isComparableTo(targetType);
+        }
+
+        @Override
+        public Object toObject(String value) {
+            if (value == null || value.length() == 0) {
+                return null;
+            }
+            try {
+                Long l = Long.parseLong(value);
+                if (l.longValue() < 0) {
+                    throw new IllegalDataException("Value may not be negative(" + l + ")");
+                }
+                return l;
+            } catch (NumberFormatException e) {
+                throw new IllegalDataException(e);
+            }
+        }
+        
+        @Override
+        public int getResultSetSqlType() {
+            return LONG.getResultSetSqlType();
+        }
+    },
+    /**
+     * Unsigned integer type that restricts values to be from 0 to {@link java.lang.Integer#MAX_VALUE} inclusive. May be used to map to existing HTable values created through {@link org.apache.hadoop.hbase.util.Bytes#toBytes(int)}
+     * as long as all values are non negative (the leading sign bit of negative numbers would cause them to sort ahead of positive numbers when
+     * they're used as part of the row key when using the HBase utility methods).
+     */
+    UNSIGNED_INT("UNSIGNED_INT", 9 /* no constant available in Types */, Integer.class, new UnsignedIntCodec()) {
+
+        @Override
+        public byte[] toBytes(Object object) {
+            byte[] b = new byte[Bytes.SIZEOF_INT];
+            toBytes(object, b, 0);
+            return b;
+        }
+
+        @Override
+        public int toBytes(Object object, byte[] b, int o) {
+            if (object == null) {
+                throw new ConstraintViolationException(this + " may not be null");
+            }
+            return this.getCodec().encodeInt(((Number)object).intValue(), b, o);
+        }
+
+        @Override
+        public Object toObject(Object object, PDataType actualType) {
+            Object o = UNSIGNED_LONG.toObject(object, actualType);
+            if(!(o instanceof Long) || o == null) {
+                return o;
+            }
+            long l = (Long)o;
+            if (l > Integer.MAX_VALUE) {
+                throw new IllegalDataException(actualType + " value " + l + " cannot be cast to Unsigned Integer without changing its value");
+            }
+            return (int)l;
+        }
+
+        @Override
+        public Object toObject(byte[] b, int o, int l, PDataType actualType) {
+            if (l == 0) {
+                return null;
+            }
+            switch (actualType) {
+            case UNSIGNED_LONG:
+            case LONG:
+            case UNSIGNED_INT:
+            case INTEGER:
+            case SMALLINT:
+            case UNSIGNED_SMALLINT:
+            case TINYINT:
+            case UNSIGNED_TINYINT:
+            case FLOAT:
+            case UNSIGNED_FLOAT:
+            case DOUBLE:
+            case UNSIGNED_DOUBLE:
+                return actualType.getCodec().decodeInt(b, o, null);
+            default:
+                return super.toObject(b,o,l,actualType);
+            }
+        }
+
+        @Override
+        public boolean isCoercibleTo(PDataType targetType) {
+            return this == targetType || targetType == INTEGER || UNSIGNED_LONG.isCoercibleTo(targetType);
+        }
+        
+        @Override
+        public boolean isCoercibleTo(PDataType targetType, Object value) {
+            if (value != null) {
+                switch (targetType) {
+                    case UNSIGNED_SMALLINT:
+           

<TRUNCATED>

[45/51] [partial] Initial commit of master branch from github

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/recovery/TrackingParallelWriterIndexCommitter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/recovery/TrackingParallelWriterIndexCommitter.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/recovery/TrackingParallelWriterIndexCommitter.java
new file mode 100644
index 0000000..46a771b
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/recovery/TrackingParallelWriterIndexCommitter.java
@@ -0,0 +1,228 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.index.write.recovery;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+
+import com.google.common.collect.Multimap;
+import org.apache.hadoop.hbase.index.CapturingAbortable;
+import org.apache.hadoop.hbase.index.exception.MultiIndexWriteFailureException;
+import org.apache.hadoop.hbase.index.exception.SingleIndexWriteFailureException;
+import org.apache.hadoop.hbase.index.parallel.EarlyExitFailure;
+import org.apache.hadoop.hbase.index.parallel.Task;
+import org.apache.hadoop.hbase.index.parallel.TaskBatch;
+import org.apache.hadoop.hbase.index.parallel.TaskRunner;
+import org.apache.hadoop.hbase.index.parallel.ThreadPoolBuilder;
+import org.apache.hadoop.hbase.index.parallel.ThreadPoolManager;
+import org.apache.hadoop.hbase.index.parallel.WaitForCompletionTaskRunner;
+import org.apache.hadoop.hbase.index.table.CachingHTableFactory;
+import org.apache.hadoop.hbase.index.table.HTableFactory;
+import org.apache.hadoop.hbase.index.table.HTableInterfaceReference;
+import org.apache.hadoop.hbase.index.write.IndexCommitter;
+import org.apache.hadoop.hbase.index.write.IndexWriter;
+import org.apache.hadoop.hbase.index.write.IndexWriterUtils;
+import org.apache.hadoop.hbase.index.write.ParallelWriterIndexCommitter;
+
+/**
+ * Like the {@link ParallelWriterIndexCommitter}, but blocks until all writes have attempted to
+ * allow the caller to retrieve the failed and succeeded index updates. Therefore, this class will
+ * be a lot slower, in the face of failures, when compared to the
+ * {@link ParallelWriterIndexCommitter} (though as fast for writes), so it should be used only when
+ * you need to at least attempt all writes and know their result; for instance, this is fine for
+ * doing WAL recovery - it's not a performance intensive situation and we want to limit the the
+ * edits we need to retry.
+ * <p>
+ * On failure to {@link #write(Multimap)}, we return a {@link MultiIndexWriteFailureException} that
+ * contains the list of {@link HTableInterfaceReference} that didn't complete successfully.
+ * <p>
+ * Failures to write to the index can happen several different ways:
+ * <ol>
+ * <li><tt>this</tt> is {@link #stop(String) stopped} or aborted (via the passed {@link Abortable}.
+ * This causing any pending tasks to fail whatever they are doing as fast as possible. Any writes
+ * that have not begun are not even attempted and marked as failures.</li>
+ * <li>A batch write fails. This is the generic HBase write failure - it may occur because the index
+ * table is not available, .META. or -ROOT- is unavailable, or any other (of many) possible HBase
+ * exceptions.</li>
+ * </ol>
+ * Regardless of how the write fails, we still wait for all writes to complete before passing the
+ * failure back to the client.
+ */
+public class TrackingParallelWriterIndexCommitter implements IndexCommitter {
+  private static final Log LOG = LogFactory.getLog(TrackingParallelWriterIndexCommitter.class);
+
+  public static final String NUM_CONCURRENT_INDEX_WRITER_THREADS_CONF_KEY = "index.trackingwriter.threads.max";
+  private static final int DEFAULT_CONCURRENT_INDEX_WRITER_THREADS = 10;
+  private static final String INDEX_WRITER_KEEP_ALIVE_TIME_CONF_KEY =
+      "index.trackingwriter.threads.keepalivetime";
+  
+  private TaskRunner pool;
+  private HTableFactory factory;
+  private CapturingAbortable abortable;
+  private Stoppable stopped;
+
+  @Override
+  public void setup(IndexWriter parent, RegionCoprocessorEnvironment env, String name) {
+    Configuration conf = env.getConfiguration();
+    setup(IndexWriterUtils.getDefaultDelegateHTableFactory(env),
+      ThreadPoolManager.getExecutor(
+        new ThreadPoolBuilder(name, conf).
+          setMaxThread(NUM_CONCURRENT_INDEX_WRITER_THREADS_CONF_KEY,
+            DEFAULT_CONCURRENT_INDEX_WRITER_THREADS).
+          setCoreTimeout(INDEX_WRITER_KEEP_ALIVE_TIME_CONF_KEY), env),
+      env.getRegionServerServices(), parent, CachingHTableFactory.getCacheSize(conf));
+  }
+
+  /**
+   * Setup <tt>this</tt>.
+   * <p>
+   * Exposed for TESTING
+   */
+  void setup(HTableFactory factory, ExecutorService pool, Abortable abortable, Stoppable stop,
+      int cacheSize) {
+    this.pool = new WaitForCompletionTaskRunner(pool);
+    this.factory = new CachingHTableFactory(factory, cacheSize);
+    this.abortable = new CapturingAbortable(abortable);
+    this.stopped = stop;
+  }
+
+  @Override
+  public void write(Multimap<HTableInterfaceReference, Mutation> toWrite)
+      throws MultiIndexWriteFailureException {
+    Set<Entry<HTableInterfaceReference, Collection<Mutation>>> entries = toWrite.asMap().entrySet();
+    TaskBatch<Boolean> tasks = new TaskBatch<Boolean>(entries.size());
+    List<HTableInterfaceReference> tables = new ArrayList<HTableInterfaceReference>(entries.size());
+    for (Entry<HTableInterfaceReference, Collection<Mutation>> entry : entries) {
+      // get the mutations for each table. We leak the implementation here a little bit to save
+      // doing a complete copy over of all the index update for each table.
+      final List<Mutation> mutations = (List<Mutation>) entry.getValue();
+      // track each reference so we can get at it easily later, when determing failures
+      final HTableInterfaceReference tableReference = entry.getKey();
+      tables.add(tableReference);
+
+      /*
+       * Write a batch of index updates to an index table. This operation stops (is cancelable) via
+       * two mechanisms: (1) setting aborted or stopped on the IndexWriter or, (2) interrupting the
+       * running thread. The former will only work if we are not in the midst of writing the current
+       * batch to the table, though we do check these status variables before starting and before
+       * writing the batch. The latter usage, interrupting the thread, will work in the previous
+       * situations as was at some points while writing the batch, depending on the underlying
+       * writer implementation (HTableInterface#batch is blocking, but doesn't elaborate when is
+       * supports an interrupt).
+       */
+      tasks.add(new Task<Boolean>() {
+
+        /**
+         * Do the actual write to the primary table. We don't need to worry about closing the table
+         * because that is handled the {@link CachingHTableFactory}.
+         */
+        @Override
+        public Boolean call() throws Exception {
+          try {
+            // this may have been queued, but there was an abort/stop so we try to early exit
+            throwFailureIfDone();
+
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("Writing index update:" + mutations + " to table: " + tableReference);
+            }
+            HTableInterface table = factory.getTable(tableReference.get());
+            throwFailureIfDone();
+            table.batch(mutations);
+          } catch (InterruptedException e) {
+            // reset the interrupt status on the thread
+            Thread.currentThread().interrupt();
+            throw e;
+          } catch (Exception e) {
+            throw e;
+          }
+          return Boolean.TRUE;
+        }
+
+        private void throwFailureIfDone() throws SingleIndexWriteFailureException {
+          if (stopped.isStopped() || abortable.isAborted()
+              || Thread.currentThread().isInterrupted()) {
+            throw new SingleIndexWriteFailureException(
+                "Pool closed, not attempting to write to the index!", null);
+          }
+
+        }
+      });
+    }
+
+    List<Boolean> results = null;
+    try {
+      LOG.debug("Waiting on index update tasks to complete...");
+      results = this.pool.submitUninterruptible(tasks);
+    } catch (ExecutionException e) {
+      throw new RuntimeException(
+          "Should not fail on the results while using a WaitForCompletionTaskRunner", e);
+    } catch (EarlyExitFailure e) {
+      throw new RuntimeException("Stopped while waiting for batch, quiting!", e);
+    }
+    
+    // track the failures. We only ever access this on return from our calls, so no extra
+    // synchronization is needed. We could update all the failures as we find them, but that add a
+    // lot of locking overhead, and just doing the copy later is about as efficient.
+    List<HTableInterfaceReference> failures = new ArrayList<HTableInterfaceReference>();
+    int index = 0;
+    for (Boolean result : results) {
+      // there was a failure
+      if (result == null) {
+        // we know which table failed by the index of the result
+        failures.add(tables.get(index));
+      }
+      index++;
+    }
+
+    // if any of the tasks failed, then we need to propagate the failure
+    if (failures.size() > 0) {
+      // make the list unmodifiable to avoid any more synchronization concerns
+      throw new MultiIndexWriteFailureException(Collections.unmodifiableList(failures));
+    }
+    return;
+  }
+
+  @Override
+  public void stop(String why) {
+    LOG.info("Shutting down " + this.getClass().getSimpleName());
+    this.pool.stop(why);
+    this.factory.shutdown();
+  }
+
+  @Override
+  public boolean isStopped() {
+    return this.stopped.isStopped();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexKeyValueSkipListSet.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexKeyValueSkipListSet.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexKeyValueSkipListSet.java
new file mode 100644
index 0000000..8314fef
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexKeyValueSkipListSet.java
@@ -0,0 +1,78 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.regionserver;
+
+import java.util.Comparator;
+import java.util.concurrent.ConcurrentSkipListMap;
+
+import org.apache.hadoop.hbase.KeyValue;
+
+/**
+ * Like a {@link KeyValueSkipListSet}, but also exposes useful, atomic methods (e.g.
+ * {@link #putIfAbsent(KeyValue)}).
+ */
+public class IndexKeyValueSkipListSet extends KeyValueSkipListSet {
+
+  // this is annoying that we need to keep this extra pointer around here, but its pretty minimal
+  // and means we don't need to change the HBase code.
+  private ConcurrentSkipListMap<KeyValue, KeyValue> delegate;
+
+  /**
+   * Create a new {@link IndexKeyValueSkipListSet} based on the passed comparator.
+   * @param comparator to use when comparing keyvalues. It is used both to determine sort order as
+   *          well as object equality in the map.
+   * @return a map that uses the passed comparator
+   */
+  public static IndexKeyValueSkipListSet create(Comparator<KeyValue> comparator) {
+    ConcurrentSkipListMap<KeyValue, KeyValue> delegate =
+        new ConcurrentSkipListMap<KeyValue, KeyValue>(comparator);
+    IndexKeyValueSkipListSet ret = new IndexKeyValueSkipListSet(delegate);
+    return ret;
+  }
+
+  /**
+   * @param delegate map to which to delegate all calls
+   */
+  public IndexKeyValueSkipListSet(ConcurrentSkipListMap<KeyValue, KeyValue> delegate) {
+    super(delegate);
+    this.delegate = delegate;
+  }
+
+  /**
+   * Add the passed {@link KeyValue} to the set, only if one is not already set. This is equivalent
+   * to
+   * <pre>
+   * if (!set.containsKey(key))
+   *   return set.put(key);
+   * else
+   *  return map.set(key);
+   * </pre>
+   * except that the action is performed atomically.
+   * @param kv {@link KeyValue} to add
+   * @return the previous value associated with the specified key, or <tt>null</tt> if there was no
+   *         previously stored key
+   * @throws ClassCastException if the specified key cannot be compared with the keys currently in
+   *           the map
+   * @throws NullPointerException if the specified key is null
+   */
+  public KeyValue putIfAbsent(KeyValue kv) {
+    return this.delegate.putIfAbsent(kv, kv);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/wal/IndexedHLogReader.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/wal/IndexedHLogReader.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/wal/IndexedHLogReader.java
new file mode 100644
index 0000000..bad82c4
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/wal/IndexedHLogReader.java
@@ -0,0 +1,152 @@
+package org.apache.hadoop.hbase.regionserver.wal;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.regionserver.wal.HLog.Entry;
+import org.apache.hadoop.hbase.regionserver.wal.HLog.Reader;
+import org.apache.hadoop.io.Writable;
+
+
+
+/**
+ * A WALReader that can also deserialize custom {@link WALEdit}s that contain index information.
+ * <p>
+ * This is basically a wrapper around a {@link SequenceFileLogReader} that has a custom
+ * {@link SequenceFileLogReader.WALReader#next(Object)} method that only replaces the creation of the WALEdit with our own custom
+ * type
+ * <p>
+ * This is a little bit of a painful way of going about this, but saves the effort of hacking the
+ * HBase source (and deal with getting it reviewed and backported, etc.) and still works.
+ */
+/*
+ * TODO: Support splitting index updates into their own WAL entries on recovery (basically, just
+ * queue them up in next), if we know that the region was on the server when it crashed. However,
+ * this is kind of difficult as we need to know a lot of things the state of the system - basically,
+ * we need to track which of the regions were on the server when it crashed only only split those
+ * edits out into their respective regions.
+ */
+public class IndexedHLogReader implements Reader {
+  private static final Log LOG = LogFactory.getLog(IndexedHLogReader.class);
+
+  private SequenceFileLogReader delegate;
+
+
+  private static class IndexedWALReader extends SequenceFileLogReader.WALReader {
+
+    /**
+     * @param fs
+     * @param p
+     * @param c
+     * @throws IOException
+     */
+    IndexedWALReader(FileSystem fs, Path p, Configuration c) throws IOException {
+      super(fs, p, c);
+    }
+
+    /**
+     * we basically have to reproduce what the SequenceFile.Reader is doing in next(), but without
+     * the check on the value class, since we have a special value class that doesn't directly match
+     * what was specified in the file header
+     */
+    @Override
+    public synchronized boolean next(Writable key, Writable val) throws IOException {
+      boolean more = next(key);
+
+      if (more) {
+        getCurrentValue(val);
+      }
+
+      return more;
+    }
+
+  }
+
+  public IndexedHLogReader() {
+    this.delegate = new SequenceFileLogReader();
+  }
+
+  @Override
+  public void init(final FileSystem fs, final Path path, Configuration conf) throws IOException {
+    this.delegate.init(fs, path, conf);
+    // close the old reader and replace with our own, custom one
+    this.delegate.reader.close();
+    this.delegate.reader = new IndexedWALReader(fs, path, conf);
+    Exception e = new Exception();
+    LOG.info("Instantiated indexed log reader." + Arrays.toString(e.getStackTrace()));
+    LOG.info("Got conf: " + conf);
+  }
+
+  @Override
+  public void close() throws IOException {
+    this.delegate.close();
+  }
+
+  @Override
+  public Entry next() throws IOException {
+    return next(null);
+  }
+
+  @Override
+  public Entry next(Entry reuse) throws IOException {
+    delegate.entryStart = delegate.reader.getPosition();
+    HLog.Entry e = reuse;
+    if (e == null) {
+      HLogKey key;
+      if (delegate.keyClass == null) {
+        key = HLog.newKey(delegate.conf);
+      } else {
+        try {
+          key = delegate.keyClass.newInstance();
+        } catch (InstantiationException ie) {
+          throw new IOException(ie);
+        } catch (IllegalAccessException iae) {
+          throw new IOException(iae);
+        }
+      }
+      WALEdit val = new WALEdit();
+      e = new HLog.Entry(key, val);
+    }
+
+    // now read in the HLog.Entry from the WAL
+    boolean nextPairValid = false;
+    try {
+      if (delegate.compressionContext != null) {
+        throw new UnsupportedOperationException(
+            "Reading compression isn't supported with the IndexedHLogReader! Compresed WALEdits "
+                + "are only support for HBase 0.94.9+ and with the IndexedWALEditCodec!");
+      }
+      // this is the special bit - we use our custom entry to read in the key-values that have index
+      // information, but otherwise it looks just like a regular WALEdit
+      IndexedWALEdit edit = new IndexedWALEdit(e.getEdit());
+      nextPairValid = delegate.reader.next(e.getKey(), edit);
+    } catch (IOException ioe) {
+      throw delegate.addFileInfoToException(ioe);
+    }
+    delegate.edit++;
+    if (delegate.compressionContext != null && delegate.emptyCompressionContext) {
+      delegate.emptyCompressionContext = false;
+    }
+    return nextPairValid ? e : null;
+  }
+
+  @Override
+  public void seek(long pos) throws IOException {
+    this.delegate.seek(pos);
+  }
+
+  @Override
+  public long getPosition() throws IOException {
+    return this.delegate.getPosition();
+  }
+
+  @Override
+  public void reset() throws IOException {
+    this.delegate.reset();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/wal/IndexedWALEdit.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/wal/IndexedWALEdit.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/wal/IndexedWALEdit.java
new file mode 100644
index 0000000..6749cc9
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/wal/IndexedWALEdit.java
@@ -0,0 +1,91 @@
+package org.apache.hadoop.hbase.regionserver.wal;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.List;
+import java.util.NavigableMap;
+import java.util.TreeMap;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import org.apache.hadoop.hbase.index.wal.KeyValueCodec;
+
+/**
+ * Read in data for a delegate {@link WALEdit}. This should only be used in concert with an IndexedHLogReader
+ * <p>
+ * This class should only be used with HBase &lt; 0.94.9. Newer installations of HBase should
+ * instead use the IndexedWALEditCodec along with the correct configuration options.
+ */
+public class IndexedWALEdit extends WALEdit {
+  //reproduced here so we don't need to modify the HBase source.
+  private static final int VERSION_2 = -1;
+  private WALEdit delegate;
+
+  /**
+   * Copy-constructor. Only does a surface copy of the delegates fields - no actual data is copied, only referenced.
+   * @param delegate to copy
+   */
+  @SuppressWarnings("deprecation")
+  public IndexedWALEdit(WALEdit delegate) {
+    this.delegate = delegate;
+    // reset the delegate's fields
+    this.delegate.getKeyValues().clear();
+    if (this.delegate.getScopes() != null) {
+      this.delegate.getScopes().clear();
+    }
+  }
+
+  public IndexedWALEdit() {
+
+  }
+
+  @Override
+public void setCompressionContext(CompressionContext context) {
+    throw new UnsupportedOperationException(
+        "Compression not supported for IndexedWALEdit! If you are using HBase 0.94.9+, use IndexedWALEditCodec instead.");
+  }
+
+  @SuppressWarnings("deprecation")
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    delegate.getKeyValues().clear();
+    if (delegate.getScopes() != null) {
+      delegate.getScopes().clear();
+    }
+    // ----------------------------------------------------------------------------------------
+    // no compression, so we do pretty much what the usual WALEdit does, plus a little magic to
+    // capture the index updates
+    // -----------------------------------------------------------------------------------------
+    int versionOrLength = in.readInt();
+    if (versionOrLength != VERSION_2) {
+      throw new IOException("You must update your cluster to the lastest version of HBase and"
+          + " clean out all logs (cleanly start and then shutdown) before enabling indexing!");
+    }
+    // this is new style HLog entry containing multiple KeyValues.
+    List<KeyValue> kvs = KeyValueCodec.readKeyValues(in);
+    delegate.getKeyValues().addAll(kvs);
+
+    // then read in the rest of the WALEdit
+    int numFamilies = in.readInt();
+    NavigableMap<byte[], Integer> scopes = delegate.getScopes();
+    if (numFamilies > 0) {
+      if (scopes == null) {
+        scopes = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
+      }
+      for (int i = 0; i < numFamilies; i++) {
+        byte[] fam = Bytes.readByteArray(in);
+        int scope = in.readInt();
+        scopes.put(fam, scope);
+      }
+      delegate.setScopes(scopes);
+    }
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    throw new IOException(
+        "Indexed WALEdits aren't written directly out - use IndexedKeyValues instead");
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/wal/IndexedWALEditCodec.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/wal/IndexedWALEditCodec.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/wal/IndexedWALEditCodec.java
new file mode 100644
index 0000000..01d7390
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/wal/IndexedWALEditCodec.java
@@ -0,0 +1,195 @@
+package org.apache.hadoop.hbase.regionserver.wal;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.codec.BaseDecoder;
+import org.apache.hadoop.hbase.codec.BaseEncoder;
+import org.apache.hadoop.hbase.codec.Decoder;
+import org.apache.hadoop.hbase.codec.Encoder;
+
+import org.apache.hadoop.hbase.index.wal.IndexedKeyValue;
+import org.apache.hadoop.hbase.index.wal.KeyValueCodec;
+
+
+/**
+ * Support custom indexing {@link KeyValue}s when written to the WAL.
+ * <p>
+ * Currently, we don't support reading older WAL files - only new WAL files. Therefore, this should
+ * not be installed on a running cluster, but rather one that has been cleanly shutdown and requires
+ * no WAL replay on startup.
+ */
+public class IndexedWALEditCodec extends WALEditCodec {
+
+  // can't have negative values because reading off a stream returns a negative if its the end of
+  // the stream
+  private static final int REGULAR_KEY_VALUE_MARKER = 0;
+  private CompressionContext compression;
+
+  /** Required nullary constructor */
+  public IndexedWALEditCodec() {
+  }
+
+  /**
+   * Override the parent implementation so we can get access to the current context too
+   * @param compression compression to support for the encoder/decoder
+   */
+  @Override
+  public void setCompression(CompressionContext compression) {
+    super.setCompression(compression);
+    this.compression = compression;
+  }
+
+  @Override
+  public Decoder getDecoder(InputStream is) {
+    // compression isn't enabled
+    if (this.compression == null) {
+      return new IndexKeyValueDecoder(is);
+    }
+
+    // there is compression, so we get the standard decoder to handle reading those kvs
+    Decoder decoder = super.getDecoder(is);
+    // compression is on, reqturn our custom decoder
+    return new CompressedIndexKeyValueDecoder(is, decoder);
+  }
+
+  @Override
+  public Encoder getEncoder(OutputStream os) {
+    // compression isn't on, do the default thing
+    if (this.compression == null) {
+      return new IndexKeyValueEncoder(os);
+    }
+
+    // compression is on, return our one that will handle putting in the correct markers
+    Encoder encoder = super.getEncoder(os);
+    return new CompressedIndexKeyValueEncoder(os, encoder);
+  }
+
+  /**
+   * Custom {@link Decoder} that can handle a stream of regular and indexed {@link KeyValue}s.
+   */
+  public class IndexKeyValueDecoder extends BaseDecoder {
+
+    /**
+     * Create a {@link Decoder} on the given input stream with the given {@link Decoder} to parse
+     * generic {@link KeyValue}s.
+     * @param is stream to read from
+     */
+    public IndexKeyValueDecoder(InputStream is){
+      super(is);
+    }
+
+    @Override
+    protected KeyValue parseCell() throws IOException{
+      return KeyValueCodec.readKeyValue((DataInput) this.in);
+    }
+  }
+
+  public class CompressedIndexKeyValueDecoder extends BaseDecoder {
+
+    private Decoder decoder;
+
+    /**
+     * Create a {@link Decoder} on the given input stream with the given {@link Decoder} to parse
+     * generic {@link KeyValue}s.
+     * @param is stream to read from
+     * @param compressedDecoder decoder for generic {@link KeyValue}s. Should support the expected
+     *          compression.
+     */
+    public CompressedIndexKeyValueDecoder(InputStream is, Decoder compressedDecoder) {
+      super(is);
+      this.decoder = compressedDecoder;
+    }
+
+    @Override
+    protected KeyValue parseCell() throws IOException {
+      // reader the marker
+      int marker = this.in.read();
+      if (marker < 0) {
+        throw new EOFException(
+            "Unexepcted end of stream found while reading next (Indexed) KeyValue");
+      }
+
+      // do the normal thing, if its a regular kv
+      if (marker == REGULAR_KEY_VALUE_MARKER) {
+        if (!this.decoder.advance()) {
+          throw new IOException("Could not read next key-value from generic KeyValue Decoder!");
+        }
+        return this.decoder.current();
+      }
+
+      // its an indexedKeyValue, so parse it out specially
+      return KeyValueCodec.readKeyValue((DataInput) this.in);
+    }
+  }
+
+  /**
+   * Encode {@link IndexedKeyValue}s via the {@link KeyValueCodec}. Does <b>not</b> support
+   * compression.
+   */
+  private static class IndexKeyValueEncoder extends BaseEncoder {
+    public IndexKeyValueEncoder(OutputStream os) {
+      super(os);
+    }
+
+    @Override
+    public void flush() throws IOException {
+      super.flush();
+    }
+
+    @Override
+    public void write(KeyValue cell) throws IOException {
+      // make sure we are open
+      checkFlushed();
+
+      // use the standard encoding mechanism
+      KeyValueCodec.write((DataOutput) this.out, cell);
+    }
+  }
+
+  /**
+   * Write {@link IndexedKeyValue}s along side compressed {@link KeyValue}s. This Encoder is
+   * <b>not</b> compatible with the {@link IndexKeyValueDecoder} - one cannot intermingle compressed
+   * and uncompressed WALs that contain index entries.
+   */
+  private static class CompressedIndexKeyValueEncoder extends BaseEncoder {
+    private Encoder compressedKvEncoder;
+
+    public CompressedIndexKeyValueEncoder(OutputStream os, Encoder compressedKvEncoder) {
+      super(os);
+      this.compressedKvEncoder = compressedKvEncoder;
+    }
+
+    @Override
+    public void flush() throws IOException {
+      this.compressedKvEncoder.flush();
+      super.flush();
+    }
+
+    @Override
+    public void write(KeyValue cell) throws IOException {
+      //make sure we are open
+      checkFlushed();
+      
+      //write the special marker so we can figure out which kind of kv is it
+      int marker = IndexedWALEditCodec.REGULAR_KEY_VALUE_MARKER;
+      if (cell instanceof IndexedKeyValue) {
+        marker = KeyValueCodec.INDEX_TYPE_LENGTH_MARKER;
+      }
+      out.write(marker);
+      
+      //then serialize based on the marker
+      if (marker == IndexedWALEditCodec.REGULAR_KEY_VALUE_MARKER) {
+        this.compressedKvEncoder.write(cell);
+      }
+      else{
+        KeyValueCodec.write((DataOutput) out, cell);
+      }
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/cache/GlobalCache.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/cache/GlobalCache.java b/phoenix-core/src/main/java/org/apache/phoenix/cache/GlobalCache.java
new file mode 100644
index 0000000..77684c5
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/cache/GlobalCache.java
@@ -0,0 +1,114 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.cache;
+
+import static org.apache.phoenix.query.QueryServices.MAX_MEMORY_PERC_ATTRIB;
+import static org.apache.phoenix.query.QueryServices.MAX_MEMORY_WAIT_MS_ATTRIB;
+import static org.apache.phoenix.query.QueryServices.MAX_TENANT_MEMORY_PERC_ATTRIB;
+
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.memory.ChildMemoryManager;
+import org.apache.phoenix.memory.GlobalMemoryManager;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.PTable;
+
+
+/**
+ * 
+ * Global root cache for the server. Each tenant is managed as a child tenant cache of this one. Queries
+ * not associated with a particular tenant use this as their tenant cache.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class GlobalCache extends TenantCacheImpl {
+    private static GlobalCache INSTANCE; 
+    
+    private final Configuration config;
+    // TODO: Use Guava cache with auto removal after lack of access 
+    private final ConcurrentMap<ImmutableBytesWritable,TenantCache> perTenantCacheMap = new ConcurrentHashMap<ImmutableBytesWritable,TenantCache>();
+    // Cache for lastest PTable for a given Phoenix table
+    private final ConcurrentHashMap<ImmutableBytesPtr,PTable> metaDataCacheMap = new ConcurrentHashMap<ImmutableBytesPtr,PTable>();
+    
+    public static synchronized GlobalCache getInstance(RegionCoprocessorEnvironment env) {
+        // See http://www.cs.umd.edu/~pugh/java/memoryModel/DoubleCheckedLocking.html
+        // for explanation of why double locking doesn't work. 
+        if (INSTANCE == null) {
+            INSTANCE = new GlobalCache(env.getConfiguration());
+        }
+        return INSTANCE;
+    }
+    
+    public ConcurrentHashMap<ImmutableBytesPtr,PTable> getMetaDataCache() {
+        return metaDataCacheMap;
+    }
+    
+    /**
+     * Get the tenant cache associated with the tenantId. If tenantId is not applicable, null may be
+     * used in which case a global tenant cache is returned.
+     * @param env the HBase configuration
+     * @param tenantId the tenant ID or null if not applicable.
+     * @return TenantCache
+     */
+    public static TenantCache getTenantCache(RegionCoprocessorEnvironment env, ImmutableBytesWritable tenantId) {
+        GlobalCache globalCache = GlobalCache.getInstance(env);
+        TenantCache tenantCache = tenantId == null ? globalCache : globalCache.getChildTenantCache(tenantId);      
+        return tenantCache;
+    }
+    
+    private GlobalCache(Configuration config) {
+        super(new GlobalMemoryManager(Runtime.getRuntime().totalMemory() * 
+                                          config.getInt(MAX_MEMORY_PERC_ATTRIB, QueryServicesOptions.DEFAULT_MAX_MEMORY_PERC) / 100,
+                                      config.getInt(MAX_MEMORY_WAIT_MS_ATTRIB, QueryServicesOptions.DEFAULT_MAX_MEMORY_WAIT_MS)),
+              config.getInt(QueryServices.MAX_SERVER_CACHE_TIME_TO_LIVE_MS, QueryServicesOptions.DEFAULT_MAX_SERVER_CACHE_TIME_TO_LIVE_MS));
+        this.config = config;
+    }
+    
+    public Configuration getConfig() {
+        return config;
+    }
+    
+    /**
+     * Retrieve the tenant cache given an tenantId.
+     * @param tenantId the ID that identifies the tenant
+     * @return the existing or newly created TenantCache
+     */
+    public TenantCache getChildTenantCache(ImmutableBytesWritable tenantId) {
+        TenantCache tenantCache = perTenantCacheMap.get(tenantId);
+        if (tenantCache == null) {
+            int maxTenantMemoryPerc = config.getInt(MAX_TENANT_MEMORY_PERC_ATTRIB, QueryServicesOptions.DEFAULT_MAX_TENANT_MEMORY_PERC);
+            int maxServerCacheTimeToLive = config.getInt(QueryServices.MAX_SERVER_CACHE_TIME_TO_LIVE_MS, QueryServicesOptions.DEFAULT_MAX_SERVER_CACHE_TIME_TO_LIVE_MS);
+            TenantCacheImpl newTenantCache = new TenantCacheImpl(new ChildMemoryManager(getMemoryManager(), maxTenantMemoryPerc), maxServerCacheTimeToLive);
+            tenantCache = perTenantCacheMap.putIfAbsent(tenantId, newTenantCache);
+            if (tenantCache == null) {
+                tenantCache = newTenantCache;
+            }
+        }
+        return tenantCache;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/cache/HashCache.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/cache/HashCache.java b/phoenix-core/src/main/java/org/apache/phoenix/cache/HashCache.java
new file mode 100644
index 0000000..4260a50
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/cache/HashCache.java
@@ -0,0 +1,40 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.cache;
+
+import java.io.Closeable;
+import java.util.List;
+
+import org.apache.http.annotation.Immutable;
+
+import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+
+/**
+ * Encapsulate deserialized hash cache from bytes into Map.
+ * The Map uses the row key as the key and the row as the value.
+ * @author jtaylor
+ * @since 0.1
+ */
+@Immutable
+public interface HashCache extends Closeable {
+    public List<Tuple> get(ImmutableBytesPtr hashKey);
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/cache/IndexMetaDataCache.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/cache/IndexMetaDataCache.java b/phoenix-core/src/main/java/org/apache/phoenix/cache/IndexMetaDataCache.java
new file mode 100644
index 0000000..fac78de
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/cache/IndexMetaDataCache.java
@@ -0,0 +1,10 @@
+package org.apache.phoenix.cache;
+
+import java.io.Closeable;
+import java.util.List;
+
+import org.apache.phoenix.index.IndexMaintainer;
+
+public interface IndexMetaDataCache extends Closeable {
+    public List<IndexMaintainer> getIndexMaintainers();
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..8703c7c
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/cache/ServerCacheClient.java
@@ -0,0 +1,278 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.cache;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import com.google.common.collect.ImmutableSet;
+import org.apache.phoenix.compile.ScanRanges;
+import org.apache.phoenix.coprocessor.ServerCachingProtocol;
+import org.apache.phoenix.coprocessor.ServerCachingProtocol.ServerCacheFactory;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.job.JobManager.JobCallable;
+import org.apache.phoenix.memory.MemoryManager.MemoryChunk;
+import org.apache.phoenix.query.ConnectionQueryServices;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.util.Closeables;
+import org.apache.phoenix.util.SQLCloseable;
+import org.apache.phoenix.util.SQLCloseables;
+
+/**
+ * 
+ * Client for sending cache to each region server
+ * 
+ * @author jtaylor
+ * @since 0.1
+ */
+public class ServerCacheClient {
+    public static final int UUID_LENGTH = Bytes.SIZEOF_LONG;
+    private static final Log LOG = LogFactory.getLog(ServerCacheClient.class);
+    private static final Random RANDOM = new Random();
+    private final PhoenixConnection connection;
+    private final Map<Integer, TableRef> cacheUsingTableRefMap = new HashMap<Integer, TableRef>();
+
+    /**
+     * Construct client used to create a serialized cached snapshot of a table and send it to each region server
+     * for caching during hash join processing.
+     * @param connection the client connection
+     * 
+     * TODO: instead of minMaxKeyRange, have an interface for iterating through ranges as we may be sending to
+     * servers when we don't have to if the min is in first region and max is in last region, especially for point queries.
+     */
+    public ServerCacheClient(PhoenixConnection connection) {
+        this.connection = connection;
+    }
+
+    public PhoenixConnection getConnection() {
+        return connection;
+    }
+    
+    /**
+     * Client-side representation of a server cache.  Call {@link #close()} when usage
+     * is complete to free cache up on region server
+     *
+     * @author jtaylor
+     * @since 0.1
+     */
+    public class ServerCache implements SQLCloseable {
+        private final int size;
+        private final byte[] id;
+        private final ImmutableSet<HRegionLocation> servers;
+        
+        public ServerCache(byte[] id, Set<HRegionLocation> servers, int size) {
+            this.id = id;
+            this.servers = ImmutableSet.copyOf(servers);
+            this.size = size;
+        }
+
+        /**
+         * Gets the size in bytes of hash cache
+         */
+        public int getSize() {
+            return size;
+        }
+
+        /**
+         * Gets the unique identifier for this hash cache
+         */
+        public byte[] getId() {
+            return id;
+        }
+
+        /**
+         * Call to free up cache on region servers when no longer needed
+         */
+        @Override
+        public void close() throws SQLException {
+            removeServerCache(id, servers);
+        }
+
+    }
+    
+    public ServerCache addServerCache(ScanRanges keyRanges, final ImmutableBytesWritable cachePtr, final ServerCacheFactory cacheFactory, final TableRef cacheUsingTableRef) throws SQLException {
+        ConnectionQueryServices services = connection.getQueryServices();
+        MemoryChunk chunk = services.getMemoryManager().allocate(cachePtr.getLength());
+        List<Closeable> closeables = new ArrayList<Closeable>();
+        closeables.add(chunk);
+        ServerCache hashCacheSpec = null;
+        SQLException firstException = null;
+        final byte[] cacheId = generateId();
+        /**
+         * Execute EndPoint in parallel on each server to send compressed hash cache 
+         */
+        // TODO: generalize and package as a per region server EndPoint caller
+        // (ideally this would be functionality provided by the coprocessor framework)
+        boolean success = false;
+        ExecutorService executor = services.getExecutor();
+        List<Future<Boolean>> futures = Collections.emptyList();
+        try {
+            List<HRegionLocation> locations = services.getAllTableRegions(cacheUsingTableRef.getTable().getPhysicalName().getBytes());
+            int nRegions = locations.size();
+            // Size these based on worst case
+            futures = new ArrayList<Future<Boolean>>(nRegions);
+            Set<HRegionLocation> servers = new HashSet<HRegionLocation>(nRegions);
+            for (HRegionLocation entry : locations) {
+                // Keep track of servers we've sent to and only send once
+                if ( ! servers.contains(entry) && 
+                        keyRanges.intersect(entry.getRegionInfo().getStartKey(), entry.getRegionInfo().getEndKey())) {  // Call RPC once per server
+                    servers.add(entry);
+                    if (LOG.isDebugEnabled()) {LOG.debug("Adding cache entry to be sent for " + entry);}
+                    final byte[] key = entry.getRegionInfo().getStartKey();
+                    final HTableInterface htable = services.getTable(cacheUsingTableRef.getTable().getPhysicalName().getBytes());
+                    closeables.add(htable);
+                    futures.add(executor.submit(new JobCallable<Boolean>() {
+                        
+                        @Override
+                        public Boolean call() throws Exception {
+                            ServerCachingProtocol protocol = htable.coprocessorProxy(ServerCachingProtocol.class, key);
+                            return protocol.addServerCache(connection.getTenantId() == null ? null : connection.getTenantId().getBytes(), cacheId, cachePtr, cacheFactory);
+                        }
+
+                        /**
+                         * Defines the grouping for round robin behavior.  All threads spawned to process
+                         * this scan will be grouped together and time sliced with other simultaneously
+                         * executing parallel scans.
+                         */
+                        @Override
+                        public Object getJobId() {
+                            return ServerCacheClient.this;
+                        }
+                    }));
+                } else {
+                    if (LOG.isDebugEnabled()) {LOG.debug("NOT adding cache entry to be sent for " + entry + " since one already exists for that entry");}
+                }
+            }
+            
+            hashCacheSpec = new ServerCache(cacheId,servers,cachePtr.getLength());
+            // Execute in parallel
+            int timeoutMs = services.getProps().getInt(QueryServices.THREAD_TIMEOUT_MS_ATTRIB, QueryServicesOptions.DEFAULT_THREAD_TIMEOUT_MS);
+            for (Future<Boolean> future : futures) {
+                future.get(timeoutMs, TimeUnit.MILLISECONDS);
+            }
+            
+            cacheUsingTableRefMap.put(Bytes.mapKey(cacheId), cacheUsingTableRef);
+            success = true;
+        } catch (SQLException e) {
+            firstException = e;
+        } catch (Exception e) {
+            firstException = new SQLException(e);
+        } finally {
+            try {
+                if (!success) {
+                    SQLCloseables.closeAllQuietly(Collections.singletonList(hashCacheSpec));
+                    for (Future<Boolean> future : futures) {
+                        future.cancel(true);
+                    }
+                }
+            } finally {
+                try {
+                    Closeables.closeAll(closeables);
+                } catch (IOException e) {
+                    if (firstException == null) {
+                        firstException = new SQLException(e);
+                    }
+                } finally {
+                    if (firstException != null) {
+                        throw firstException;
+                    }
+                }
+            }
+        }
+        return hashCacheSpec;
+    }
+    
+    /**
+     * Remove the cached table from all region servers
+     * @param cacheId unique identifier for the hash join (returned from {@link #addHashCache(HTable, Scan, Set)})
+     * @param servers list of servers upon which table was cached (filled in by {@link #addHashCache(HTable, Scan, Set)})
+     * @throws SQLException
+     * @throws IllegalStateException if hashed table cannot be removed on any region server on which it was added
+     */
+    private void removeServerCache(byte[] cacheId, Set<HRegionLocation> servers) throws SQLException {
+        ConnectionQueryServices services = connection.getQueryServices();
+        Throwable lastThrowable = null;
+        TableRef cacheUsingTableRef = cacheUsingTableRefMap.get(Bytes.mapKey(cacheId));
+        byte[] tableName = cacheUsingTableRef.getTable().getPhysicalName().getBytes();
+        HTableInterface iterateOverTable = services.getTable(tableName);
+        List<HRegionLocation> locations = services.getAllTableRegions(tableName);
+        Set<HRegionLocation> remainingOnServers = new HashSet<HRegionLocation>(servers);
+        /**
+         * Allow for the possibility that the region we based where to send our cache has split and been
+         * relocated to another region server *after* we sent it, but before we removed it. To accommodate
+         * this, we iterate through the current metadata boundaries and remove the cache once for each
+         * server that we originally sent to.
+         */
+        for (HRegionLocation entry : locations) {
+            if (remainingOnServers.contains(entry)) {  // Call once per server
+                try {
+                    byte[] key = entry.getRegionInfo().getStartKey();
+                    ServerCachingProtocol protocol = iterateOverTable.coprocessorProxy(ServerCachingProtocol.class, key);
+                    protocol.removeServerCache(connection.getTenantId() == null ? null : connection.getTenantId().getBytes(), cacheId);
+                    remainingOnServers.remove(entry);
+                } catch (Throwable t) {
+                    lastThrowable = t;
+                    LOG.error("Error trying to remove hash cache for " + entry, t);
+                }
+            }
+        }
+        if (!remainingOnServers.isEmpty()) {
+            LOG.warn("Unable to remove hash cache for " + remainingOnServers, lastThrowable);
+        }
+    }
+
+    /**
+     * Create an ID to keep the cached information across other operations independent.
+     * Using simple long random number, since the length of time we need this to be unique
+     * is very limited. 
+     */
+    public static byte[] generateId() {
+        long rand = RANDOM.nextLong();
+        return Bytes.toBytes(rand);
+    }
+    
+    public static String idToString(byte[] uuid) {
+        assert(uuid.length == Bytes.SIZEOF_LONG);
+        return Long.toString(Bytes.toLong(uuid));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..ad40d8c
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/cache/TenantCache.java
@@ -0,0 +1,44 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.cache;
+
+import java.io.Closeable;
+import java.sql.SQLException;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.coprocessor.ServerCachingProtocol.ServerCacheFactory;
+import org.apache.phoenix.memory.MemoryManager;
+
+
+/**
+ * 
+ * Inteface to set and set cached values for a tenant
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public interface TenantCache {
+    MemoryManager getMemoryManager();
+    Closeable getServerCache(ImmutableBytesPtr cacheId);
+    Closeable addServerCache(ImmutableBytesPtr cacheId, ImmutableBytesWritable cachePtr, ServerCacheFactory cacheFactory) throws SQLException;
+    void removeServerCache(ImmutableBytesPtr cacheId) throws SQLException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/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
new file mode 100644
index 0000000..eabeb11
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/cache/TenantCacheImpl.java
@@ -0,0 +1,96 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.cache;
+
+import java.io.Closeable;
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import com.google.common.cache.*;
+import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.coprocessor.ServerCachingProtocol.ServerCacheFactory;
+import org.apache.phoenix.memory.MemoryManager;
+import org.apache.phoenix.memory.MemoryManager.MemoryChunk;
+import org.apache.phoenix.util.Closeables;
+
+/**
+ * 
+ * Cache per tenant on server side.  Tracks memory usage for each
+ * tenat as well and rolling up usage to global memory manager.
+ * 
+ * @author jtaylor
+ * @since 0.1
+ */
+public class TenantCacheImpl implements TenantCache {
+    private final int maxTimeToLiveMs;
+    private final MemoryManager memoryManager;
+    private volatile Cache<ImmutableBytesPtr, Closeable> serverCaches;
+
+    public TenantCacheImpl(MemoryManager memoryManager, int maxTimeToLiveMs) {
+        this.memoryManager = memoryManager;
+        this.maxTimeToLiveMs = maxTimeToLiveMs;
+    }
+    
+    @Override
+    public MemoryManager getMemoryManager() {
+        return memoryManager;
+    }
+
+    private Cache<ImmutableBytesPtr,Closeable> getServerCaches() {
+        /* Delay creation of this map until it's needed */
+        if (serverCaches == null) {
+            synchronized(this) {
+                if (serverCaches == null) {
+                    serverCaches = CacheBuilder.newBuilder()
+                        .expireAfterAccess(maxTimeToLiveMs, TimeUnit.MILLISECONDS)
+                        .removalListener(new RemovalListener<ImmutableBytesPtr, Closeable>(){
+                            @Override
+                            public void onRemoval(RemovalNotification<ImmutableBytesPtr, Closeable> notification) {
+                                Closeables.closeAllQuietly(Collections.singletonList(notification.getValue()));
+                            }
+                        })
+                        .build();
+                }
+            }
+        }
+        return serverCaches;
+    }
+    
+    @Override
+    public Closeable getServerCache(ImmutableBytesPtr cacheId) {
+        return getServerCaches().getIfPresent(cacheId);
+    }
+    
+    @Override
+    public Closeable addServerCache(ImmutableBytesPtr cacheId, ImmutableBytesWritable cachePtr, ServerCacheFactory cacheFactory) throws SQLException {
+        MemoryChunk chunk = this.getMemoryManager().allocate(cachePtr.getLength());
+        Closeable element = cacheFactory.newCache(cachePtr, chunk);
+        getServerCaches().put(cacheId, element);
+        return element;
+    }
+    
+    @Override
+    public void removeServerCache(ImmutableBytesPtr cacheId) throws SQLException {
+        getServerCaches().invalidate(cacheId);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/cache/aggcache/SpillFile.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/cache/aggcache/SpillFile.java b/phoenix-core/src/main/java/org/apache/phoenix/cache/aggcache/SpillFile.java
new file mode 100644
index 0000000..f2ad908
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/cache/aggcache/SpillFile.java
@@ -0,0 +1,142 @@
+package org.apache.phoenix.cache.aggcache;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.MappedByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.channels.FileChannel.MapMode;
+import java.util.Map;
+import java.util.UUID;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Maps;
+import com.google.common.io.Closeables;
+
+/**
+ * This class abstracts a SpillFile It is a accessible on a per page basis
+ * For every SpillFile object a single spill file is always created. 
+ * Additional overflow files are dynamically created in case the page index requested is not covered by
+ * the spillFiles allocated so far
+ */
+public class SpillFile implements Closeable {
+
+    private static final Logger logger = LoggerFactory.getLogger(SpillFile.class);
+    // Default size for a single spillFile 2GB
+    private static final int SPILL_FILE_SIZE = Integer.MAX_VALUE;
+    // Page size for a spill file 4K
+    static final int DEFAULT_PAGE_SIZE = 4096;
+    // Map of initial SpillFile at index 0, and overflow spillFiles
+    private Map<Integer, TempFile> tempFiles;
+    
+    // Wrapper class for a TempFile: File + RandomAccessFile
+    private static class TempFile implements Closeable{
+    	private RandomAccessFile rndFile;
+    	private File file;
+    	
+    	public TempFile(File file, RandomAccessFile rndFile) {
+    		this.file = file;
+    		this.rndFile = rndFile;
+    	}    	
+    	    	
+    	public FileChannel getChannel() {
+    		return rndFile.getChannel();
+    	}
+
+		@Override
+		public void close() throws IOException {
+			Closeables.closeQuietly(rndFile.getChannel());
+			Closeables.closeQuietly(rndFile);
+			
+			if (file != null) {
+                if (logger.isDebugEnabled()) {
+                    logger.debug("Deleting tempFile: " + file.getAbsolutePath());
+                }
+                try {
+                    file.delete();
+                } catch (SecurityException e) {
+                    logger.warn("IOException thrown while closing Closeable." + e);
+            	}
+            }
+		}
+    }
+    
+    /**
+     * Create a new SpillFile using the Java TempFile creation function. SpillFile is access in
+     * pages.
+     */
+    public static SpillFile createSpillFile() {
+    	try {    		
+    		return new SpillFile(createTempFile());    		
+    	} catch (IOException ioe) {
+        	throw new RuntimeException("Could not create Spillfile " + ioe);
+        }
+    }
+    
+    
+    private static TempFile createTempFile() throws IOException {
+        File tempFile = File.createTempFile(UUID.randomUUID().toString(), null);
+        if (logger.isDebugEnabled()) {
+            logger.debug("Creating new SpillFile: " + tempFile.getAbsolutePath());
+        }
+        RandomAccessFile file = new RandomAccessFile(tempFile, "rw");
+        file.setLength(SPILL_FILE_SIZE);
+        
+        return new TempFile(tempFile, file);
+    }
+
+    
+    private SpillFile(TempFile spFile) throws IOException {
+        this.tempFiles = Maps.newHashMap();
+        // Init the first pre-allocated spillFile
+        tempFiles.put(0, spFile);
+    }
+
+    /**
+     * Random access to a page of the current spill file
+     * @param index
+     */
+    public MappedByteBuffer getPage(int index) {
+        try {
+        	TempFile tempFile = null;
+        	int fileIndex = 0;
+        	
+            long offset = (long) index * (long) DEFAULT_PAGE_SIZE;            
+            if(offset >= SPILL_FILE_SIZE) {
+            	// Offset exceeds the first SpillFile size
+            	// Get the index of the file that should contain the pageID
+            	fileIndex = (int)(offset / SPILL_FILE_SIZE);
+            	if(!tempFiles.containsKey(fileIndex)) {
+            		// Dynamically add new spillFiles if directory grows beyond 
+            		// max page ID.
+            		tempFile = createTempFile();
+            		tempFiles.put(fileIndex, tempFile);
+            	}
+            }
+        	tempFile = tempFiles.get(fileIndex);
+        	// Channel gets buffered in file object
+        	FileChannel fc = tempFile.getChannel();
+
+        	return fc.map(MapMode.READ_WRITE, offset, DEFAULT_PAGE_SIZE);
+        } catch (IOException ioe) {
+            // Close resource
+            close();
+            throw new RuntimeException("Could not get page at index: " + index);
+        } catch (IllegalArgumentException iae) {
+            // Close resource
+            close();
+            throw iae;
+        }
+    }
+
+    @Override
+    public void close() {
+    	for(TempFile file : tempFiles.values()) {
+            // Swallow IOExceptions
+            Closeables.closeQuietly(file);
+    	}
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/cache/aggcache/SpillManager.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/cache/aggcache/SpillManager.java b/phoenix-core/src/main/java/org/apache/phoenix/cache/aggcache/SpillManager.java
new file mode 100644
index 0000000..448ffa5
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/cache/aggcache/SpillManager.java
@@ -0,0 +1,323 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.cache.aggcache;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.Closeable;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.io.WritableUtils;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.io.Closeables;
+import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.expression.aggregator.Aggregator;
+import org.apache.phoenix.expression.aggregator.ServerAggregators;
+import org.apache.phoenix.expression.function.SingleAggregateFunction;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.KeyValueSchema;
+import org.apache.phoenix.schema.ValueBitSet;
+import org.apache.phoenix.schema.tuple.SingleKeyValueTuple;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.KeyValueUtil;
+import org.apache.phoenix.util.TupleUtil;
+
+/**
+ * Class servers as an adapter between the in-memory LRU cache and the Spill data structures. It
+ * takes care of serializing / deserializing the key/value groupby tuples, and spilling them to the
+ * correct spill partition
+ */
+public class SpillManager implements Closeable {
+
+    // Wrapper class for DESERIALIZED groupby key/value tuples
+    public static class CacheEntry<T extends ImmutableBytesWritable> implements
+            Map.Entry<T, Aggregator[]> {
+
+        protected T key;
+        protected Aggregator[] aggs;
+
+        public CacheEntry(T key, Aggregator[] aggs) {
+            this.key = key;
+            this.aggs = aggs;
+        }
+
+        public Aggregator[] getValue(Configuration conf) {
+            return aggs;
+        }
+
+        public int getKeyLength() {
+            return key.getLength();
+        }
+
+        @Override
+        public Aggregator[] getValue() {
+            return aggs;
+        }
+
+        @Override
+        public Aggregator[] setValue(Aggregator[] arg0) {
+            this.aggs = arg0;
+            return aggs;
+        }
+
+        @Override
+        public T getKey() {
+            return key;
+        }
+
+    }
+
+    private final ArrayList<SpillMap> spillMaps;
+    private final int numSpillFiles;
+
+    private final ServerAggregators aggregators;
+    private final Configuration conf;
+
+    /**
+     * SpillManager takes care of spilling and loading tuples from spilled data structs
+     * @param numSpillFiles
+     * @param serverAggregators
+     */
+    public SpillManager(int numSpillFiles, ServerAggregators serverAggregators,
+            Configuration conf, SpillableGroupByCache.QueryCache cache) {
+        try {
+            int estValueSize = serverAggregators.getEstimatedByteSize();
+            spillMaps = Lists.newArrayList();
+            this.numSpillFiles = numSpillFiles;
+            this.aggregators = serverAggregators;
+            this.conf = conf;
+            
+            // Ensure that a single element fits onto a page!!!
+            Preconditions.checkArgument(SpillFile.DEFAULT_PAGE_SIZE > estValueSize);
+
+            // Create a list of spillFiles
+            // Each Spillfile only handles up to 2GB data
+            for (int i = 0; i < numSpillFiles; i++) {
+                SpillFile file = SpillFile.createSpillFile();
+                spillMaps.add(new SpillMap(file, SpillFile.DEFAULT_PAGE_SIZE, estValueSize, cache));
+            }
+        } catch (IOException ioe) {
+            throw new RuntimeException("Could not init the SpillManager");
+        }
+    }
+
+    // serialize a key/value tuple into a byte array
+    // WARNING: expensive
+    private byte[] serialize(ImmutableBytesPtr key, Aggregator[] aggs,
+            ServerAggregators serverAggs) throws IOException {
+
+        DataOutputStream output = null;
+        ByteArrayOutputStream bai = null;
+        try {
+            bai = new ByteArrayOutputStream();
+            output = new DataOutputStream(bai);
+            // key length
+            WritableUtils.writeVInt(output, key.getLength());
+            // key
+            output.write(key.get(), key.getOffset(), key.getLength());
+            byte[] aggsByte = serverAggs.toBytes(aggs);
+            // aggs length
+            WritableUtils.writeVInt(output, aggsByte.length);
+            // aggs
+            output.write(aggsByte);
+            return bai.toByteArray();
+        } finally {
+
+            if (bai != null) {
+                bai.close();
+                bai = null;
+            }
+            if (output != null) {
+                output.close();
+                output = null;
+            }
+        }
+    }
+
+    /**
+     * Helper method to deserialize the key part from a serialized byte array
+     * @param data
+     * @return
+     * @throws IOException
+     */
+    static ImmutableBytesPtr getKey(byte[] data) throws IOException {
+        DataInputStream input = null;
+        try {
+            input = new DataInputStream(new ByteArrayInputStream(data));
+            // key length
+            int keyLength = WritableUtils.readVInt(input);
+            int offset = WritableUtils.getVIntSize(keyLength);
+            // key
+            return new ImmutableBytesPtr(data, offset, keyLength);
+        } finally {
+            if (input != null) {
+                input.close();
+                input = null;
+            }
+        }
+    }
+
+    
+    // Instantiate Aggregators form a serialized byte array
+    private Aggregator[] getAggregators(byte[] data) throws IOException {
+        DataInputStream input = null;
+        try {
+            input = new DataInputStream(new ByteArrayInputStream(data));
+            // key length
+            int keyLength = WritableUtils.readVInt(input);
+            int vIntKeyLength = WritableUtils.getVIntSize(keyLength);
+            ImmutableBytesPtr ptr = new ImmutableBytesPtr(data, vIntKeyLength, keyLength);
+
+            // value length
+            input.skip(keyLength);
+            int valueLength = WritableUtils.readVInt(input);
+            int vIntValLength = WritableUtils.getVIntSize(keyLength);
+            KeyValue keyValue =
+                    KeyValueUtil.newKeyValue(ptr.get(), ptr.getOffset(), ptr.getLength(),
+                        QueryConstants.SINGLE_COLUMN_FAMILY, QueryConstants.SINGLE_COLUMN,
+                        QueryConstants.AGG_TIMESTAMP, data, vIntKeyLength + keyLength + vIntValLength, valueLength);
+            Tuple result = new SingleKeyValueTuple(keyValue);
+            TupleUtil.getAggregateValue(result, ptr);
+            KeyValueSchema schema = aggregators.getValueSchema();
+            ValueBitSet tempValueSet = ValueBitSet.newInstance(schema);
+            tempValueSet.clear();
+            tempValueSet.or(ptr);
+
+            int i = 0, maxOffset = ptr.getOffset() + ptr.getLength();
+            SingleAggregateFunction[] funcArray = aggregators.getFunctions();
+            Aggregator[] sAggs = new Aggregator[funcArray.length];
+            Boolean hasValue;
+            schema.iterator(ptr);
+            while ((hasValue = schema.next(ptr, i, maxOffset, tempValueSet)) != null) {
+                SingleAggregateFunction func = funcArray[i];
+                sAggs[i++] =
+                        hasValue ? func.newServerAggregator(conf, ptr) : func
+                                .newServerAggregator(conf);
+            }
+            return sAggs;
+
+        } finally {
+            Closeables.closeQuietly(input);
+        }
+    }
+
+    /**
+     * Helper function to deserialize a byte array into a CacheEntry
+     * @param <K>
+     * @param bytes
+     * @throws IOException
+     */
+    @SuppressWarnings("unchecked")
+    public <K extends ImmutableBytesWritable> CacheEntry<K> toCacheEntry(byte[] bytes)
+            throws IOException {
+        ImmutableBytesPtr key = SpillManager.getKey(bytes);
+        Aggregator[] aggs = getAggregators(bytes);
+
+        return new CacheEntry<K>((K) key, aggs);
+    }
+
+    // Determines the partition, i.e. spillFile the tuple should get spilled to.
+    private int getPartition(ImmutableBytesWritable key) {
+        // Simple implementation hash mod numFiles
+        return Math.abs(key.hashCode()) % numSpillFiles;
+    }
+
+    /**
+     * Function that spills a key/value groupby tuple into a partition Spilling always triggers a
+     * serialize call
+     * @param key
+     * @param value
+     * @throws IOException
+     */
+    public void spill(ImmutableBytesWritable key, Aggregator[] value) throws IOException {
+        SpillMap spillMap = spillMaps.get(getPartition(key));
+        ImmutableBytesPtr keyPtr = new ImmutableBytesPtr(key);
+        byte[] data = serialize(keyPtr, value, aggregators);
+        spillMap.put(keyPtr, data);
+    }
+
+    /**
+     * Function that loads a spilled key/value groupby tuple from one of the spill partitions into
+     * the LRU cache. Loading always involves deserialization
+     * @throws IOException
+     */
+    public Aggregator[] loadEntry(ImmutableBytesWritable key) throws IOException {
+        SpillMap spillMap = spillMaps.get(getPartition(key));
+        byte[] data = spillMap.get(key);
+        if (data != null) {
+            return getAggregators(data);
+        }
+        return null;
+    }
+
+    /**
+     * Close the attached spillMap
+     */
+    @Override
+    public void close() {
+        for (int i = 0; i < spillMaps.size(); i++) {
+            Closeables.closeQuietly(spillMaps.get(i).getSpillFile());
+        }
+    }
+
+    /**
+     * Function returns an iterator over all spilled Tuples
+     */
+    public SpillMapIterator newDataIterator() {
+        return new SpillMapIterator();
+    }
+
+    private final class SpillMapIterator implements Iterator<byte[]> {
+
+        int index = 0;
+        Iterator<byte[]> spillIter = spillMaps.get(index).iterator();
+
+        @Override
+        public boolean hasNext() {
+            if (!spillIter.hasNext()) {
+                if (index < (numSpillFiles - 1)) {
+                    // Current spillFile exhausted get iterator over new one
+                    spillIter = spillMaps.get(++index).iterator();
+                }
+            }
+            return spillIter.hasNext();
+        }
+
+        @Override
+        public byte[] next() {
+            return spillIter.next();
+        }
+
+        @Override
+        public void remove() {
+            throw new IllegalAccessError("Remove is not supported for this type of iterator");
+        }
+    }
+}


[44/51] [partial] Initial commit of master branch from github

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/cache/aggcache/SpillMap.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/cache/aggcache/SpillMap.java b/phoenix-core/src/main/java/org/apache/phoenix/cache/aggcache/SpillMap.java
new file mode 100644
index 0000000..d93d08c
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/cache/aggcache/SpillMap.java
@@ -0,0 +1,494 @@
+package org.apache.phoenix.cache.aggcache;
+
+import java.io.IOException;
+import java.nio.BufferOverflowException;
+import java.nio.MappedByteBuffer;
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.hadoop.hbase.util.Bytes;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.hash.BloomFilter;
+import com.google.common.hash.Funnels;
+import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+
+/**
+ * Class implements an active spilled partition serialized tuples are first written into an in-memory data structure
+ * that represents a single page. As the page fills up, it is written to the current spillFile or spill partition For
+ * fast tuple discovery, the class maintains a per page bloom-filter and never de-serializes elements. The element
+ * spilling employs an extentible hashing technique.
+ */
+public class SpillMap extends AbstractMap<ImmutableBytesPtr, byte[]> implements Iterable<byte[]> {
+
+    // Threshold is typically the page size
+    private final int thresholdBytes;
+    private final int pageInserts;
+    // Global directory depth
+    private int globalDepth;
+    private int curMapBufferIndex;
+    private SpillFile spillFile;
+    // Directory of hash buckets --> extendible hashing implementation
+    private MappedByteBufferMap[] directory;
+    private final SpillableGroupByCache.QueryCache cache;
+
+    public SpillMap(SpillFile file, int thresholdBytes, int estValueSize, SpillableGroupByCache.QueryCache cache)
+            throws IOException {
+        this.thresholdBytes = thresholdBytes - Bytes.SIZEOF_INT;
+        this.pageInserts = thresholdBytes / estValueSize;
+        this.spillFile = file;
+        this.cache = cache;
+
+        // Init the e-hashing directory structure
+        globalDepth = 1;
+        directory = new MappedByteBufferMap[(1 << globalDepth)];
+
+        for (int i = 0; i < directory.length; i++) {
+            // Create an empty bucket list
+            directory[i] = new MappedByteBufferMap(i, this.thresholdBytes, pageInserts, file);
+            directory[i].flushBuffer();
+        }
+        directory[0].pageIn();
+        curMapBufferIndex = 0;
+    }
+
+    // Get the directoy index for a specific key
+    private int getBucketIndex(ImmutableBytesPtr key) {
+        // Get key hash
+        int hashCode = key.hashCode();
+
+        // Mask all but globalDepth low n bits
+        return hashCode & ((1 << globalDepth) - 1);
+    }
+
+    // Function redistributes the elements in the current index
+    // to two new buckets, based on the bit at localDepth + 1 position.
+    // Optionally this function also doubles the directory to allow
+    // for bucket splits
+    private void redistribute(int index, ImmutableBytesPtr keyNew, byte[] valueNew) {
+        // Get the respective bucket
+        MappedByteBufferMap byteMap = directory[index];
+
+        // Get the actual bucket index, that the directory index points to
+        int mappedIdx = byteMap.pageIndex;
+
+        int localDepth = byteMap.localDepth;
+        ArrayList<Integer> buckets = Lists.newArrayList();
+        // Get all directory entries that point to the same bucket.
+        // TODO: can be made faster!
+        for (int i = 0; i < directory.length; i++) {
+            if (directory[i].pageIndex == mappedIdx) {
+                buckets.add(i);
+            }
+        }
+
+        // Assuming no directory doubling for now
+        // compute the two new bucket Ids for splitting
+        // SpillFile adds new files dynamically in case the directory points to pageIDs
+        // that exceed the size limit of a single file.
+
+        // TODO verify if some sort of de-fragmentation might be helpful
+        int tmpIndex = index ^ ((1 << localDepth));
+        int b1Index = Math.min(index, tmpIndex);
+        int b2Index = Math.max(index, tmpIndex);
+
+        // Create two new split buckets
+        MappedByteBufferMap b1 = new MappedByteBufferMap(b1Index, thresholdBytes, pageInserts, spillFile);
+        MappedByteBufferMap b2 = new MappedByteBufferMap(b2Index, thresholdBytes, pageInserts, spillFile);
+
+        // redistribute old elements into b1 and b2
+        for (Entry<ImmutableBytesPtr, byte[]> element : byteMap.pageMap.entrySet()) {
+            ImmutableBytesPtr key = element.getKey();
+            byte[] value = element.getValue();
+            // Only add key during redistribution if its not in the cache
+            // Otherwise this is an good point to reduce the number of spilled elements
+            if (!cache.isKeyContained(key)) {
+                // Re-distribute element onto the new 2 split buckets
+                if ((key.hashCode() & ((1 << localDepth))) != 0) {
+                    b2.addElement(null, key, value);
+                } else {
+                    b1.addElement(null, key, value);
+                }
+            }
+        }
+
+        // Clear and GC the old now redistributed bucket
+        byteMap.pageMap.clear();
+        byteMap = null;
+
+        // Increase local bucket depths
+        b1.localDepth = localDepth + 1;
+        b2.localDepth = localDepth + 1;
+        boolean doubleDir = false;
+
+        if (globalDepth < (localDepth + 1)) {
+            // Double directory structure and re-adjust pointers
+            doubleDir = true;
+
+            b2Index = doubleDirectory(b2Index, keyNew);
+        }
+
+        if (!doubleDir) {
+            // This is a bit more tricky, we have to cover scenarios where
+            // globalDepth - localDepth > 1
+            // Here even after bucket splitting, multiple directory entries point to
+            // the new buckets
+            for (int i = 0; i < buckets.size(); i++) {
+                if ((buckets.get(i) & (1 << (localDepth))) != 0) {
+                    directory[buckets.get(i)] = b2;
+                } else {
+                    directory[buckets.get(i)] = b1;
+                }
+            }
+        } else {
+            // Update the directory indexes in case of directory doubling
+            directory[b1Index] = b1;
+            directory[b2Index] = b2;
+        }
+    }
+
+    // Doubles the directory and readjusts pointers.
+    private int doubleDirectory(int b2Index, ImmutableBytesPtr keyNew) {
+        // Double the directory in size, second half points to original first half
+        int newDirSize = 1 << (globalDepth + 1);
+
+        // Ensure that the new directory size does not exceed size limits
+        Preconditions.checkArgument(newDirSize < Integer.MAX_VALUE);
+
+        // Double it!
+        MappedByteBufferMap[] newDirectory = new MappedByteBufferMap[newDirSize];
+        for (int i = 0; i < directory.length; i++) {
+            newDirectory[i] = directory[i];
+            newDirectory[i + directory.length] = directory[i];
+        }
+        directory = newDirectory;
+        newDirectory = null;
+
+        // Adjust the index for new split bucket, according to the directory double
+        b2Index = (keyNew.hashCode() & ((1 << globalDepth) - 1)) | (1 << globalDepth);
+
+        // Increment global depth
+        globalDepth++;
+
+        return b2Index;
+    }
+
+    /**
+     * Get a key from the spillable data structures. page is determined via hash partitioning, and a bloomFilter check
+     * is used to determine if its worth paging in the data.
+     */
+    @Override
+    public byte[] get(Object key) {
+        if (!(key instanceof ImmutableBytesPtr)) {
+            // TODO ... work on type safety
+        }
+        ImmutableBytesPtr ikey = (ImmutableBytesPtr)key;
+        byte[] value = null;
+
+        int bucketIndex = getBucketIndex(ikey);
+        MappedByteBufferMap byteMap = directory[bucketIndex];
+
+        // Decision based on bucket ID, not the directory ID due to the n:1 relationship
+        if (directory[curMapBufferIndex].pageIndex != byteMap.pageIndex) {
+            // map not paged in
+            MappedByteBufferMap curByteMap = directory[curMapBufferIndex];
+
+            // Use bloomFilter to check if key was spilled before
+            if (byteMap.containsKey(ikey.copyBytesIfNecessary())) {
+                // ensure consistency and flush current memory page to disk
+                // fflush current buffer
+                curByteMap.flushBuffer();
+                // page in new buffer
+                byteMap.pageIn();
+                // update index
+                curMapBufferIndex = bucketIndex;
+            }
+        }
+        // get KV from current map
+        value = byteMap.getPagedInElement(ikey);
+        return value;
+    }
+
+    // Similar as get(Object key) function, however
+    // always pages in page a key is spilled to, no bloom filter decision
+    private byte[] getAlways(ImmutableBytesPtr key) {
+        byte[] value = null;
+        int bucketIndex = getBucketIndex(key);
+        MappedByteBufferMap byteMap = directory[bucketIndex];
+
+        if (directory[curMapBufferIndex].pageIndex != byteMap.pageIndex) {
+            MappedByteBufferMap curByteMap = directory[curMapBufferIndex];
+            // ensure consistency and flush current memory page to disk
+            curByteMap.flushBuffer();
+
+            byteMap.pageIn();
+            curMapBufferIndex = bucketIndex;
+        }
+        // get KV from current queue
+        value = byteMap.getPagedInElement(key);
+        return value;
+    }
+
+    /**
+     * Spill a key First we discover if the key has been spilled before and load it into memory: #ref get() if it was
+     * loaded before just replace the old value in the memory page if it was not loaded before try to store it in the
+     * current page alternatively if not enough memory available, request new page.
+     */
+    @Override
+    public byte[] put(ImmutableBytesPtr key, byte[] value) {
+        boolean redistributed = false;
+        // page in element and replace if present
+        byte[] spilledValue = getAlways(key);
+
+        MappedByteBufferMap byteMap = directory[curMapBufferIndex];
+        int index = curMapBufferIndex;
+
+        // TODO: We split buckets until the new element fits onto a
+        // one of the new buckets. Might consider the use of an overflow
+        // bucket, especially in case the directory runs out of page IDs.
+        while (!byteMap.canFit(spilledValue, value)) {
+            // Element does not fit... Split the bucket!
+            redistribute(index, key, value);
+            redistributed = true;
+
+            index = getBucketIndex(key);
+            byteMap = directory[index];
+        }
+        // Ensure that all pages that were paged in during redistribution are flushed back out
+        // to disk to keep memory footprint small.
+        if (redistributed) {
+            for (int i = 0; i < directory.length; i++) {
+                if (directory[i].pageIndex != byteMap.pageIndex) {
+                    directory[i].flushBuffer();
+                }
+            }
+            // Ensure the page that receives the new key is in memory
+            spilledValue = getAlways(key);
+        }
+        byteMap.addElement(spilledValue, key, value);
+
+        return value;
+    }
+
+    /**
+     * Function returns the current spill file
+     */
+    public SpillFile getSpillFile() {
+        return spillFile;
+    }
+
+    /**
+     * This inner class represents the currently mapped file region. It uses a Map to represent the current in memory
+     * page for easy get() and update() calls on an individual key The class keeps track of the current size of the in
+     * memory page and handles flushing and paging in respectively
+     */
+    private static class MappedByteBufferMap {
+        private SpillFile spillFile;
+        private int pageIndex;
+        private final int thresholdBytes;
+        private long totalResultSize;
+        private boolean pagedIn;
+        private int localDepth;
+        // dirtyPage flag tracks if a paged in page was modified
+        // if not, no need to flush it back out to disk
+        private boolean dirtyPage;
+        // Use a map for in memory page representation
+        Map<ImmutableBytesPtr, byte[]> pageMap = Maps.newHashMap();
+        // Used to determine is an element was written to this page before or not
+        BloomFilter<byte[]> bFilter;
+
+        public MappedByteBufferMap(int id, int thresholdBytes, int pageInserts, SpillFile spillFile) {
+            this.spillFile = spillFile;
+            // size threshold of a page
+            this.thresholdBytes = thresholdBytes;
+            this.pageIndex = id;
+            pageMap.clear();
+            bFilter = BloomFilter.create(Funnels.byteArrayFunnel(), pageInserts);
+            pagedIn = true;
+            totalResultSize = 0;
+            localDepth = 1;
+            dirtyPage = true;
+        }
+
+        private boolean containsKey(byte[] key) {
+            return bFilter.mightContain(key);
+        }
+
+        private boolean canFit(byte[] curValue, byte[] newValue) {
+            if (thresholdBytes < newValue.length) {
+                // TODO resize page size if single element is too big,
+                // Can this ever happen?
+                throw new RuntimeException("page size too small to store a single KV element");
+            }
+
+            int resultSize = newValue.length + Bytes.SIZEOF_INT;
+            if (curValue != null) {
+                // Key existed before
+                // Ensure to compensate for potential larger byte[] for agg
+                resultSize = Math.max(0, resultSize - (curValue.length + Bytes.SIZEOF_INT));
+            }
+
+            if ((thresholdBytes - totalResultSize) <= (resultSize)) {
+                // KV does not fit
+                return false;
+            }
+            // KV fits
+            return true;
+        }
+
+        // Flush the current page to the memory mapped byte buffer
+        private void flushBuffer() throws BufferOverflowException {
+            if (pagedIn) {
+                MappedByteBuffer buffer;
+                // Only flush if page was changed
+                if (dirtyPage) {
+                    Collection<byte[]> values = pageMap.values();
+                    buffer = spillFile.getPage(pageIndex);
+                    buffer.clear();
+                    // number of elements
+                    buffer.putInt(values.size());
+                    for (byte[] value : values) {
+                        // element length
+                        buffer.putInt(value.length);
+                        // element
+                        buffer.put(value, 0, value.length);
+                    }
+                }
+                buffer = null;
+                // Reset page stats
+                pageMap.clear();
+                totalResultSize = 0;
+            }
+            pagedIn = false;
+            dirtyPage = false;
+        }
+
+        // load memory mapped region into a map for fast element access
+        private void pageIn() throws IndexOutOfBoundsException {
+            if (!pagedIn) {
+                // Map the memory region
+                MappedByteBuffer buffer = spillFile.getPage(pageIndex);
+                int numElements = buffer.getInt();
+                for (int i = 0; i < numElements; i++) {
+                    int kvSize = buffer.getInt();
+                    byte[] data = new byte[kvSize];
+                    buffer.get(data, 0, kvSize);
+                    try {
+                        pageMap.put(SpillManager.getKey(data), data);
+                        totalResultSize += (data.length + Bytes.SIZEOF_INT);
+                    } catch (IOException ioe) {
+                        // Error during key access on spilled resource
+                        // TODO rework error handling
+                        throw new RuntimeException(ioe);
+                    }
+                }
+                pagedIn = true;
+                dirtyPage = false;
+            }
+        }
+
+        /**
+         * Return a cache element currently page into memory Direct access via mapped page map
+         * 
+         * @param key
+         * @return
+         */
+        public byte[] getPagedInElement(ImmutableBytesPtr key) {
+            return pageMap.get(key);
+        }
+
+        /**
+         * Inserts / Replaces cache element in the currently loaded page. Direct access via mapped page map
+         * 
+         * @param key
+         * @param value
+         */
+        public void addElement(byte[] spilledValue, ImmutableBytesPtr key, byte[] value) {
+
+            // put Element into map
+            pageMap.put(key, value);
+            // Update bloom filter
+            bFilter.put(key.copyBytesIfNecessary());
+            // track current Map size to prevent Buffer overflows
+            if (spilledValue != null) {
+                // if previous key was present, just add the size difference
+                totalResultSize += Math.max(0, value.length - (spilledValue.length));
+            } else {
+                // Add new size information
+                totalResultSize += (value.length + Bytes.SIZEOF_INT);
+            }
+
+            dirtyPage = true;
+        }
+
+        /**
+         * Returns a value iterator over the pageMap
+         */
+        public Iterator<byte[]> getPageMapEntries() {
+            pageIn();
+            return pageMap.values().iterator();
+        }
+    }
+
+    /**
+     * Iterate over all spilled elements, including the ones that are currently paged into memory
+     */
+    @Override
+    public Iterator<byte[]> iterator() {
+        directory[curMapBufferIndex].flushBuffer();
+
+        return new Iterator<byte[]>() {
+            int pageIndex = 0;
+            Iterator<byte[]> entriesIter = directory[pageIndex].getPageMapEntries();
+            HashSet<Integer> dups = new HashSet<Integer>();
+
+            @Override
+            public boolean hasNext() {
+                if (!entriesIter.hasNext()) {
+                    boolean found = false;
+                    // Clear in memory map
+
+                    while (!found) {
+                        pageIndex++;
+                        if (pageIndex >= directory.length) { return false; }
+                        directory[pageIndex - 1].pageMap.clear();
+                        // get keys from all spilled pages
+                        if (!dups.contains(directory[pageIndex].pageIndex)) {
+                            dups.add(directory[pageIndex].pageIndex);
+                            entriesIter = directory[pageIndex].getPageMapEntries();
+                            if (entriesIter.hasNext()) {
+                                found = true;
+                            }
+                        }
+                    }
+                }
+                dups.add(directory[pageIndex].pageIndex);
+                return true;
+            }
+
+            @Override
+            public byte[] next() {
+                // get elements from in memory map first
+                return entriesIter.next();
+            }
+
+            @Override
+            public void remove() {
+                throw new IllegalAccessError("Iterator does not support removal operation");
+            }
+        };
+    }
+
+    // TODO implement this method to make the SpillMap a true Map implementation
+    @Override
+    public Set<java.util.Map.Entry<ImmutableBytesPtr, byte[]>> entrySet() {
+        throw new IllegalAccessError("entrySet is not supported for this type of cache");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/cache/aggcache/SpillableGroupByCache.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/cache/aggcache/SpillableGroupByCache.java b/phoenix-core/src/main/java/org/apache/phoenix/cache/aggcache/SpillableGroupByCache.java
new file mode 100644
index 0000000..41a4e65
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/cache/aggcache/SpillableGroupByCache.java
@@ -0,0 +1,361 @@
+package org.apache.phoenix.cache.aggcache;
+
+import static org.apache.phoenix.query.QueryConstants.AGG_TIMESTAMP;
+import static org.apache.phoenix.query.QueryConstants.SINGLE_COLUMN;
+import static org.apache.phoenix.query.QueryConstants.SINGLE_COLUMN_FAMILY;
+import static org.apache.phoenix.query.QueryServices.GROUPBY_MAX_CACHE_SIZE_ATTRIB;
+import static org.apache.phoenix.query.QueryServices.GROUPBY_SPILL_FILES_ATTRIB;
+import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_GROUPBY_MAX_CACHE_MAX;
+import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_GROUPBY_SPILL_FILES;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.io.Closeables;
+import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.cache.GlobalCache;
+import org.apache.phoenix.cache.TenantCache;
+import org.apache.phoenix.cache.aggcache.SpillManager.CacheEntry;
+import org.apache.phoenix.coprocessor.BaseRegionScanner;
+import org.apache.phoenix.coprocessor.GroupByCache;
+import org.apache.phoenix.coprocessor.GroupedAggregateRegionObserver;
+import org.apache.phoenix.expression.aggregator.Aggregator;
+import org.apache.phoenix.expression.aggregator.ServerAggregators;
+import org.apache.phoenix.memory.InsufficientMemoryException;
+import org.apache.phoenix.memory.MemoryManager.MemoryChunk;
+import org.apache.phoenix.util.KeyValueUtil;
+
+/**
+ * The main entry point is in GroupedAggregateRegionObserver. It instantiates a SpillableGroupByCache and invokes a
+ * get() method on it. There is no: "if key not exists -> put into map" case, since the cache is a Loading cache and
+ * therefore handles the put under the covers. I tried to implement the final cache element accesses (RegionScanner
+ * below) streaming, i.e. there is just an iterator on it and removed the existing result materialization.
+ * SpillableGroupByCache implements a LRU cache using a LinkedHashMap with access order. There is a configurable an
+ * upper and lower size limit in bytes which are used as follows to compute the initial cache size in number of
+ * elements: Max(lowerBoundElements, Min(upperBoundElements, estimatedCacheSize)). Once the number of cached elements
+ * exceeds this number, the cache size is increased by a factor of 1.5. This happens until the additional memory to grow
+ * the cache cannot be requested. At this point the Cache starts spilling elements. As long as no eviction happens no
+ * spillable data structures are allocated, this only happens as soon as the first element is evicted from the cache. We
+ * cannot really make any assumptions on which keys arrive at the map, but assume the LRU would at least cover the cases
+ * where some keys have a slight skew and they should stay memory resident. Once a key gets evicted, the spillManager is
+ * instantiated. It basically takes care of spilling an element to disk and does all the SERDE work. It pre-allocates a
+ * configurable number of SpillFiles (spill partition) which are memory mapped temp files. The SpillManager keeps a list
+ * of these and hash distributes the keys within this list. Once an element gets spilled, it is serialized and will only
+ * get deserialized again, when it is requested from the client, i.e. loaded back into the LRU cache. The SpillManager
+ * holds a single SpillMap object in memory for every spill partition (SpillFile). The SpillMap is an in memory Map
+ * representation of a single page of spilled serialized key/value pairs. To achieve fast key lookup the key is hash
+ * partitioned into random pages of the current spill file. The code implements an extendible hashing approach which
+ * dynamically adjusts the hash function, in order to adapt to growing number of storage pages and avoiding long chains
+ * of overflow buckets. For an excellent discussion of the algorithm please refer to the following online resource:
+ * http://db.inf.uni-tuebingen.de/files/teaching/ws1011/db2/db2-hash-indexes.pdf . For this, each SpillFile keeps a
+ * directory of pointers to Integer.MAX_VALUE 4K pages in memory, which allows each directory to address more pages than
+ * a single memory mapped temp file could theoretically store. In case directory doubling, requests a page index that
+ * exceeds the limits of the initial temp file limits, the implementation dynamically allocates additional temp files to
+ * the SpillFile. The directory starts with a global depth of 1 and therefore a directory size of 2 buckets. Only during
+ * bucket split and directory doubling more than one page is temporarily kept in memory until all elements have been
+ * redistributed. The current implementation conducts bucket splits as long as an element does not fit onto a page. No
+ * overflow chain is created, which might be an alternative. For get requests, each directory entry maintains a
+ * bloomFilter to prevent page-in operations in case an element has never been spilled before. The deserialization is
+ * only triggered when a key a loaded back into the LRU cache. The aggregators are returned from the LRU cache and the
+ * next value is computed. In case the key is not found on any page, the Loader create new aggregators for it.
+ */
+
+public class SpillableGroupByCache implements GroupByCache {
+
+    private static final Logger logger = LoggerFactory.getLogger(SpillableGroupByCache.class);
+
+    // Min size of 1st level main memory cache in bytes --> lower bound
+    private static final int SPGBY_CACHE_MIN_SIZE = 4096; // 4K
+
+    // TODO Generally better to use Collection API with generics instead of
+    // array types
+    private final LinkedHashMap<ImmutableBytesWritable, Aggregator[]> cache;
+    private SpillManager spillManager = null;
+    private int curNumCacheElements;
+    private final ServerAggregators aggregators;
+    private final RegionCoprocessorEnvironment env;
+    private final MemoryChunk chunk;
+
+    /*
+     * inner class that makes cache queryable for other classes that should not get the full instance. Queryable view of
+     * the cache
+     */
+    public class QueryCache {
+        public boolean isKeyContained(ImmutableBytesPtr key) {
+            return cache.containsKey(key);
+        }
+    }
+
+    /**
+     * Instantiates a Loading LRU Cache that stores key / aggregator[] tuples used for group by queries
+     * 
+     * @param estSize
+     * @param estValueSize
+     * @param aggs
+     * @param ctxt
+     */
+    public SpillableGroupByCache(final RegionCoprocessorEnvironment env, ImmutableBytesWritable tenantId,
+            ServerAggregators aggs, final int estSizeNum) {
+        curNumCacheElements = 0;
+        this.aggregators = aggs;
+        this.env = env;
+
+        final int estValueSize = aggregators.getEstimatedByteSize();
+        final TenantCache tenantCache = GlobalCache.getTenantCache(env, tenantId);
+
+        // Compute Map initial map
+        final Configuration conf = env.getConfiguration();
+        final long maxCacheSizeConf = conf.getLong(GROUPBY_MAX_CACHE_SIZE_ATTRIB, DEFAULT_GROUPBY_MAX_CACHE_MAX);
+        final int numSpillFilesConf = conf.getInt(GROUPBY_SPILL_FILES_ATTRIB, DEFAULT_GROUPBY_SPILL_FILES);
+
+        final int maxSizeNum = (int)(maxCacheSizeConf / estValueSize);
+        final int minSizeNum = (SPGBY_CACHE_MIN_SIZE / estValueSize);
+
+        // use upper and lower bounds for the cache size
+        final int maxCacheSize = Math.max(minSizeNum, Math.min(maxSizeNum, estSizeNum));
+        final int estSize = GroupedAggregateRegionObserver.sizeOfUnorderedGroupByMap(maxCacheSize, estValueSize);
+        try {
+            this.chunk = tenantCache.getMemoryManager().allocate(estSize);
+        } catch (InsufficientMemoryException ime) {
+            logger.error("Requested Map size exceeds memory limit, please decrease max size via config paramter: "
+                    + GROUPBY_MAX_CACHE_SIZE_ATTRIB);
+            throw ime;
+        }
+
+        if (logger.isDebugEnabled()) {
+            logger.debug("Instantiating LRU groupby cache of element size: " + maxCacheSize);
+        }
+
+        // LRU cache implemented as LinkedHashMap with access order
+        cache = new LinkedHashMap<ImmutableBytesWritable, Aggregator[]>(maxCacheSize, 0.75f, true) {
+            boolean spill = false;
+            int cacheSize = maxCacheSize;
+
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<ImmutableBytesWritable, Aggregator[]> eldest) {
+                if (!spill && size() > cacheSize) { // increase allocation
+                    cacheSize *= 1.5f;
+                    int estSize = GroupedAggregateRegionObserver.sizeOfUnorderedGroupByMap(cacheSize, estValueSize);
+                    try {
+                        chunk.resize(estSize);
+                    } catch (InsufficientMemoryException im) {
+                        // Cannot extend Map anymore, start spilling
+                        spill = true;
+                    }
+                }
+
+                if (spill) {
+                    try {
+                        if (spillManager == null) {
+                            // Lazy instantiation of spillable data
+                            // structures
+                            //
+                            // Only create spill data structs if LRU
+                            // cache is too small
+                            spillManager = new SpillManager(numSpillFilesConf, aggregators, env.getConfiguration(),
+                                    new QueryCache());
+                        }
+                        spillManager.spill(eldest.getKey(), eldest.getValue());
+                        // keep track of elements in cache
+                        curNumCacheElements--;
+                    } catch (IOException ioe) {
+                        // Ensure that we always close and delete the temp files
+                        try {
+                            throw new RuntimeException(ioe);
+                        } finally {
+                            Closeables.closeQuietly(SpillableGroupByCache.this);
+                        }
+                    }
+                    return true;
+                }
+
+                return false;
+            }
+        };
+    }
+
+    /**
+     * Size function returns the estimate LRU cache size in bytes
+     */
+    @Override
+    public int size() {
+        return curNumCacheElements * aggregators.getEstimatedByteSize();
+    }
+
+    /**
+     * Extract an element from the Cache If element is not present in in-memory cache / or in spill files cache
+     * implements an implicit put() of a new key/value tuple and loads it into the cache
+     */
+    @Override
+    public Aggregator[] cache(ImmutableBytesWritable cacheKey) {
+        ImmutableBytesPtr key = new ImmutableBytesPtr(cacheKey);
+        Aggregator[] rowAggregators = cache.get(key);
+        if (rowAggregators == null) {
+            // If Aggregators not found for this distinct
+            // value, clone our original one (we need one
+            // per distinct value)
+            if (spillManager != null) {
+                // Spill manager present, check if key has been
+                // spilled before
+                try {
+                    rowAggregators = spillManager.loadEntry(key);
+                } catch (IOException ioe) {
+                    // Ensure that we always close and delete the temp files
+                    try {
+                        throw new RuntimeException(ioe);
+                    } finally {
+                        Closeables.closeQuietly(SpillableGroupByCache.this);
+                    }
+                }
+            }
+            if (rowAggregators == null) {
+                // No, key never spilled before, create a new tuple
+                rowAggregators = aggregators.newAggregators(env.getConfiguration());
+                if (logger.isDebugEnabled()) {
+                    logger.debug("Adding new aggregate bucket for row key "
+                            + Bytes.toStringBinary(key.get(), key.getOffset(), key.getLength()));
+                }
+            }
+            cache.put(key, rowAggregators);
+            // keep track of elements in cache
+            curNumCacheElements++;
+        }
+        return rowAggregators;
+    }
+
+    /**
+     * Iterator over the cache and the spilled data structures by returning CacheEntries. CacheEntries are either
+     * extracted from the LRU cache or from the spillable data structures.The key/value tuples are returned in
+     * non-deterministic order.
+     */
+    private final class EntryIterator implements Iterator<Map.Entry<ImmutableBytesWritable, Aggregator[]>> {
+        final Iterator<Map.Entry<ImmutableBytesWritable, Aggregator[]>> cacheIter;
+        final Iterator<byte[]> spilledCacheIter;
+
+        private EntryIterator() {
+            cacheIter = cache.entrySet().iterator();
+            if (spillManager != null) {
+                spilledCacheIter = spillManager.newDataIterator();
+            } else {
+                spilledCacheIter = null;
+            }
+        }
+
+        @Override
+        public boolean hasNext() {
+            return cacheIter.hasNext();
+        }
+
+        @Override
+        public Map.Entry<ImmutableBytesWritable, Aggregator[]> next() {
+            if (spilledCacheIter != null && spilledCacheIter.hasNext()) {
+                try {
+                    byte[] value = spilledCacheIter.next();
+                    // Deserialize into a CacheEntry
+                    Map.Entry<ImmutableBytesWritable, Aggregator[]> spilledEntry = spillManager.toCacheEntry(value);
+
+                    boolean notFound = false;
+                    // check against map and return only if not present
+                    while (cache.containsKey(spilledEntry.getKey())) {
+                        // LRU Cache entries always take precedence,
+                        // since they are more up to date
+                        if (spilledCacheIter.hasNext()) {
+                            value = spilledCacheIter.next();
+                            spilledEntry = spillManager.toCacheEntry(value);
+                        } else {
+                            notFound = true;
+                            break;
+                        }
+                    }
+                    if (!notFound) {
+                        // Return a spilled entry, this only happens if the
+                        // entry was not
+                        // found in the LRU cache
+                        return spilledEntry;
+                    }
+                } catch (IOException ioe) {
+                    // TODO rework error handling
+                    throw new RuntimeException(ioe);
+                }
+            }
+            // Spilled elements exhausted
+            // Finally return all elements from LRU cache
+            Map.Entry<ImmutableBytesWritable, Aggregator[]> entry = cacheIter.next();
+            return new CacheEntry<ImmutableBytesWritable>(entry.getKey(), entry.getValue());
+        }
+
+        /**
+         * Remove??? Denied!!!
+         */
+        @Override
+        public void remove() {
+            throw new IllegalAccessError("Remove is not supported for this type of iterator");
+        }
+    }
+
+    /**
+     * Closes cache and releases spill resources
+     * 
+     * @throws IOException
+     */
+    @Override
+    public void close() throws IOException {
+        // Close spillable resources
+        Closeables.closeQuietly(spillManager);
+        Closeables.closeQuietly(chunk);
+    }
+
+    @Override
+    public RegionScanner getScanner(final RegionScanner s) {
+        final Iterator<Entry<ImmutableBytesWritable, Aggregator[]>> cacheIter = new EntryIterator();
+
+        // scanner using the spillable implementation
+        return new BaseRegionScanner() {
+            @Override
+            public HRegionInfo getRegionInfo() {
+                return s.getRegionInfo();
+            }
+
+            @Override
+            public void close() throws IOException {
+                try {
+                    s.close();
+                } finally {
+                    // Always close gbCache and swallow possible Exceptions
+                    Closeables.closeQuietly(SpillableGroupByCache.this);
+                }
+            }
+
+            @Override
+            public boolean next(List<KeyValue> results) throws IOException {
+                if (!cacheIter.hasNext()) { return false; }
+                Map.Entry<ImmutableBytesWritable, Aggregator[]> ce = cacheIter.next();
+                ImmutableBytesWritable key = ce.getKey();
+                Aggregator[] aggs = ce.getValue();
+                byte[] value = aggregators.toBytes(aggs);
+                if (logger.isDebugEnabled()) {
+                    logger.debug("Adding new distinct group: "
+                            + Bytes.toStringBinary(key.get(), key.getOffset(), key.getLength()) + " with aggregators "
+                            + aggs.toString() + " value = " + Bytes.toStringBinary(value));
+                }
+                results.add(KeyValueUtil.newKeyValue(key.get(), key.getOffset(), key.getLength(), SINGLE_COLUMN_FAMILY,
+                        SINGLE_COLUMN, AGG_TIMESTAMP, value, 0, value.length));
+                return cacheIter.hasNext();
+            }
+        };
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/client/ClientKeyValue.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/client/ClientKeyValue.java b/phoenix-core/src/main/java/org/apache/phoenix/client/ClientKeyValue.java
new file mode 100644
index 0000000..3288c4d
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/client/ClientKeyValue.java
@@ -0,0 +1,503 @@
+/**
+ * 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.client;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * {@link KeyValue} that should only be used from the client side. Enables clients to be more
+ * flexible with the byte arrays they use when building a {@link KeyValue}, but still wire
+ * compatible.
+ * <p>
+ * All<tt> byte[]</tt> (or {@link ImmutableBytesWritable}) passed into the constructor are only ever
+ * read once - when writing <tt>this</tt> onto the wire. They are never copied into another array or
+ * reused. This has the advantage of being much more efficient than the usual {@link KeyValue}
+ * <p>
+ * The down side is that we no longer can support some of the usual methods like
+ * {@link #getBuffer()} or {@link #getKey()} since its is backed with multiple <tt>byte[]</tt> and
+ * <i>should only be used by the client to <b>send</b> information</i>
+ * <p>
+ * <b>WARNING:</b> should only be used by advanced users who know how to construct their own
+ * KeyValues
+ */
+public class ClientKeyValue extends KeyValue {
+
+  private static ImmutableBytesWritable NULL = new ImmutableBytesWritable(new byte[0]);
+  private ImmutableBytesWritable row;
+  private ImmutableBytesWritable family;
+  private ImmutableBytesWritable qualifier;
+  private Type type;
+  private long ts;
+  private ImmutableBytesWritable value;
+
+  /**
+   * @param row must not be <tt>null</tt>
+   * @param type must not be <tt>null</tt>
+   */
+  public ClientKeyValue(ImmutableBytesWritable row, ImmutableBytesWritable family,
+      ImmutableBytesWritable qualifier, long ts, Type type, ImmutableBytesWritable value) {
+    this.row = row;
+    this.family = family == null ? NULL : family;
+    this.qualifier = qualifier == null ? NULL : qualifier;
+    this.type = type;
+    this.ts = ts;
+    this.value = value == null ? NULL : value;
+  }
+
+  /**
+   * Convenience constructor that just wraps all the bytes in {@link ImmutableBytesWritable}
+   */
+  public ClientKeyValue(byte[] row, byte[] family, byte[] qualifier, long ts, Type t, byte[] value) {
+    this(wrap(row), wrap(family), wrap(qualifier), ts, t, wrap(value));
+  }
+
+  /**
+   * Convenience constructor that just wraps all the bytes in {@link ImmutableBytesWritable}
+   */
+  public ClientKeyValue(byte[] row, byte[] family, byte[] qualifier, long ts, Type t) {
+    this(wrap(row), wrap(family), wrap(qualifier), ts, t, null);
+  }
+
+  private static ImmutableBytesWritable wrap(byte[] b) {
+    return b == null ? NULL : new ImmutableBytesWritable(b);
+  }
+
+  @Override
+  public KeyValue clone() {
+    return new ClientKeyValue(copy(row), copy(family), copy(qualifier), ts, type, copy(value));
+  }
+
+  private ImmutableBytesWritable copy(ImmutableBytesWritable bytes) {
+    return new ImmutableBytesWritable(bytes.copyBytes());
+  }
+
+  private static byte[] copyIfNecessary(ImmutableBytesWritable bytes) {
+    byte[] b = bytes.get();
+    if (bytes.getLength() == b.length && bytes.getOffset() == 0) {
+      return b;
+    }
+    return Arrays.copyOfRange(b, bytes.getOffset(), bytes.getOffset() + bytes.getLength());
+  }
+
+  @Override
+  public KeyValue shallowCopy() {
+    return new ClientKeyValue(row, family, qualifier, ts, type, value);
+  }
+
+  @Override
+  public int getValueOffset() {
+    return value.getOffset();
+  }
+
+  @Override
+  public int getValueLength() {
+    return value.getLength();
+  }
+
+  @Override
+  public int getRowOffset() {
+    return row.getOffset();
+  }
+
+  @Override
+  public short getRowLength() {
+    return (short) row.getLength();
+  }
+
+  @Override
+  public int getFamilyOffset() {
+    return family.getOffset();
+  }
+
+  @Override
+  public byte getFamilyLength() {
+    return (byte) family.getLength();
+  }
+
+  @Override
+  public byte getFamilyLength(int foffset) {
+    return this.getFamilyLength();
+  }
+
+  @Override
+  public int getQualifierOffset() {
+    return qualifier.getOffset();
+  }
+
+  @Override
+  public int getQualifierLength() {
+    return qualifier.getLength();
+  }
+
+  @Override
+  public int getQualifierLength(int rlength, int flength) {
+    return this.getQualifierLength();
+  }
+
+  @Override
+  public int getTotalColumnLength(int rlength, int foffset) {
+    return this.getFamilyLength() + this.getQualifierLength();
+  }
+
+  @Override
+  public int getTotalColumnLength() {
+    return qualifier.getLength() + family.getLength();
+  }
+
+  @Override
+  public byte[] getValue() {
+    return copyIfNecessary(value);
+  }
+
+  @Override
+  public byte[] getRow() {
+    return copyIfNecessary(row);
+  }
+
+  @Override
+  public long getTimestamp() {
+    return ts;
+  }
+
+  @Override
+  public byte[] getFamily() {
+    return copyIfNecessary(family);
+  }
+
+  @Override
+  public byte[] getQualifier() {
+    return copyIfNecessary(qualifier);
+  }
+
+  @Override
+  public byte getType() {
+    return this.type.getCode();
+  }
+
+  @Override
+  public boolean matchingFamily(byte[] family) {
+    if (family == null) {
+      if (this.family.getLength() == 0) {
+        return true;
+      }
+      return false;
+    }
+    return matchingFamily(family, 0, family.length);
+  }
+
+  @Override
+  public boolean matchingFamily(byte[] family, int offset, int length) {
+    if (family == null) {
+      if (this.family.getLength() == 0) {
+        return true;
+      }
+      return false;
+    }
+    return matches(family, offset, length, this.family);
+  }
+
+  @Override
+  public boolean matchingFamily(KeyValue other) {
+    if(other == null) {
+      return false;
+    }
+    if(other instanceof ClientKeyValue) {
+      ClientKeyValue kv = (ClientKeyValue)other;
+      return this.family.compareTo(kv.family) == 0;
+    }
+    return matchingFamily(other.getBuffer(), other.getFamilyOffset(), other.getFamilyLength());
+  }
+
+  private boolean matches(byte[] b, int offset, int length, ImmutableBytesWritable bytes) {
+    return Bytes.equals(b, offset, length, bytes.get(), bytes.getOffset(), bytes.getLength());
+  }
+
+  @Override
+  public boolean matchingQualifier(byte[] qualifier) {
+    if (qualifier == null) {
+      if (this.qualifier.getLength() == 0) {
+        return true;
+      }
+      return false;
+    }
+    return matchingQualifier(qualifier, 0, qualifier.length);
+  }
+
+  @Override
+  public boolean matchingQualifier(byte[] qualifier, int offset, int length) {
+    if (qualifier == null) {
+      if (this.qualifier.getLength() == 0) {
+        return true;
+      }
+      return false;
+    }
+    return matches(qualifier, offset, length, this.qualifier);
+  }
+
+  @Override
+  public boolean matchingQualifier(KeyValue other) {
+    if (other == null) {
+      return false;
+    }
+    if (other instanceof ClientKeyValue) {
+      ClientKeyValue kv = (ClientKeyValue) other;
+      return this.row.compareTo(kv.row) == 0;
+    }
+    return matchingQualifier(other.getBuffer(), other.getQualifierOffset(),
+      other.getQualifierLength());
+  }
+
+  @Override
+  public boolean matchingRow(byte[] row){
+    if (row == null) {
+      return false;
+    }
+    return matches(row, 0, row.length, this.row);
+  }
+
+  @Override
+  public boolean matchingRow(byte[] row, int offset, int length) {
+    if (row == null) {
+      return false;
+    }
+    return matches(row, offset, length, this.row);
+  }
+
+  @Override
+  public boolean matchingRow(KeyValue other) {
+    return matchingRow(other.getBuffer(), other.getRowOffset(), other.getRowLength());
+  }
+
+  @Override
+  public boolean matchingColumnNoDelimiter(byte[] column) {
+    // match both the family and qualifier
+    if (matchingFamily(column, 0, this.family.getLength())) {
+      return matchingQualifier(column, family.getLength(), column.length - family.getLength());
+    }
+    return false;
+  }
+
+  @Override
+  public boolean matchingColumn(byte[] family, byte[] qualifier) {
+    return this.matchingFamily(family) && matchingQualifier(qualifier);
+  }
+
+  @Override
+  public boolean nonNullRowAndColumn() {
+    return (this.row != null && row.getLength() > 0) && !isEmptyColumn();
+  }
+
+  @Override
+  public boolean isEmptyColumn() {
+    return this.qualifier != null && this.qualifier.getLength() > 0;
+  }
+
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    // we need to simulate the keyvalue writing, but actually step through each buffer.
+    //start with keylength
+    long longkeylength = KeyValue.KEY_INFRASTRUCTURE_SIZE + row.getLength() + family.getLength()
+        + qualifier.getLength();
+    if (longkeylength > Integer.MAX_VALUE) {
+      throw new IllegalArgumentException("keylength " + longkeylength + " > " + Integer.MAX_VALUE);
+    }
+    // need to figure out the max length before we start
+    int length = this.getLength();
+    out.writeInt(length);
+
+    // write the actual data
+    int keylength = (int) longkeylength;
+    out.writeInt(keylength);
+    int vlength = value == null ? 0 : value.getLength();
+    out.writeInt(vlength);
+    out.writeShort((short) (row.getLength() & 0x0000ffff));
+    out.write(this.row.get(), this.row.getOffset(), this.row.getLength());
+    out.writeByte((byte) (family.getLength() & 0x0000ff));
+    if (family.getLength() != 0) {
+      out.write(this.family.get(), this.family.getOffset(), this.family.getLength());
+    }
+    if (qualifier != NULL) {
+      out.write(this.qualifier.get(), this.qualifier.getOffset(), this.qualifier.getLength());
+    }
+    out.writeLong(ts);
+    out.writeByte(this.type.getCode());
+    if (this.value != NULL) {
+      out.write(this.value.get(), this.value.getOffset(), this.value.getLength());
+    }
+  }
+
+  @Override
+  public int getLength() {
+    return KEYVALUE_INFRASTRUCTURE_SIZE + KeyValue.ROW_LENGTH_SIZE + row.getLength()
+        + KeyValue.FAMILY_LENGTH_SIZE + family.getLength() + qualifier.getLength()
+        + KeyValue.TIMESTAMP_SIZE + KeyValue.TYPE_SIZE + value.getLength();
+  }
+
+  @Override
+  public String toString() {
+    return keyToString() + "/vlen=" + getValueLength() + "/ts=" + getMemstoreTS();
+  }
+
+  private String keyToString() {
+    String row = Bytes.toStringBinary(this.row.get(), this.row.getOffset(), this.row.getLength());
+    String family = this.family.getLength() == 0 ? "" : Bytes.toStringBinary(this.family.get(),
+      this.family.getOffset(), this.family.getLength());
+    String qualifier = this.qualifier.getLength() == 0 ? "" : Bytes.toStringBinary(
+      this.qualifier.get(), this.qualifier.getOffset(), this.qualifier.getLength());
+    String timestampStr = Long.toString(ts);
+    byte type = this.type.getCode();
+    return row + "/" + family + (family != null && family.length() > 0 ? ":" : "") + qualifier
+        + "/" + timestampStr + "/" + Type.codeToType(type);
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) return true;
+    if (!super.equals(obj)) return false;
+    if (getClass() != obj.getClass()) return false;
+    ClientKeyValue other = (ClientKeyValue) obj;
+    if (family == null) {
+      if (other.family != null) return false;
+    } else if (!family.equals(other.family)) return false;
+    if (qualifier == null) {
+      if (other.qualifier != null) return false;
+    } else if (!qualifier.equals(other.qualifier)) return false;
+    if (row == null) {
+      if (other.row != null) return false;
+    } else if (!row.equals(other.row)) return false;
+    if (ts != other.ts) return false;
+    if (type != other.type) return false;
+    if (value == null) {
+      if (other.value != null) return false;
+    } else if (!value.equals(other.value)) return false;
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    // TODO do we need to keep the same hashcode logic as KeyValue? Everywhere else we don't keep
+    // them by reference, but presumably clients might hash them.
+    final int prime = 31;
+    int result = super.hashCode();
+    result = prime * result + family.hashCode();
+    result = prime * result + qualifier.hashCode();
+    result = prime * result + row.hashCode();
+    result = prime * result + (int) (ts ^ (ts >>> 32));
+    result = prime * result + type.hashCode();
+    result = prime * result + value.hashCode();
+    return result;
+  }
+
+  @Override
+  public void readFields(int length, DataInput in) throws IOException {
+    throw new UnsupportedOperationException(ClientKeyValue.class.getSimpleName()
+        + " should not be used for server-side operations");
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    throw new UnsupportedOperationException(ClientKeyValue.class.getSimpleName()
+        + " should not be used for server-side operations");
+  }
+
+  @Override
+  public int getKeyOffset() {
+    return 0;
+  }
+
+
+  @Override
+  public int getFamilyOffset(int rlength) {
+    return 0;
+  }
+
+  @Override
+  public int getQualifierOffset(int foffset) {
+    return 0;
+  }
+
+  @Override
+  public int getTimestampOffset() {
+    return 0;
+  }
+
+  @Override
+  public int getTimestampOffset(int keylength) {
+    return 0;
+  }
+
+  @Override
+  public int getOffset() {
+    return 0;
+  }
+
+  @Override
+  public boolean updateLatestStamp(byte[] now) {
+    if (this.isLatestTimestamp()) {
+      // unfortunately, this is a bit slower than the usual kv, but we don't expect this to happen
+      // all that often on the client (unless users are updating the ts this way), as it generally
+      // happens on the server
+      this.ts = Bytes.toLong(now);
+      return true;
+    }
+    return false;
+  }
+
+  @Override
+  public boolean isLatestTimestamp() {
+    return this.ts == HConstants.LATEST_TIMESTAMP;
+  }
+
+  @Override
+  public int getKeyLength() {
+    return KEY_INFRASTRUCTURE_SIZE + getRowLength() + getFamilyLength() + getQualifierLength();
+  }
+
+  @Override
+  public byte[] getKey() {
+    throw new UnsupportedOperationException(ClientKeyValue.class.getSimpleName()
+        + " does not support a single backing buffer.");
+  }
+
+  @Override
+  public String getKeyString() {
+    throw new UnsupportedOperationException(ClientKeyValue.class.getSimpleName()
+        + " does not support a single backing buffer.");
+  }
+
+  @Override
+  public SplitKeyValue split() {
+    throw new UnsupportedOperationException(ClientKeyValue.class.getSimpleName()
+        + " should not be used for server-side operations");
+  }
+
+  @Override
+  public byte[] getBuffer() {
+    throw new UnsupportedOperationException(ClientKeyValue.class.getSimpleName()
+        + " does not support a single backing buffer.");
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/client/ClientKeyValueBuilder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/client/ClientKeyValueBuilder.java b/phoenix-core/src/main/java/org/apache/phoenix/client/ClientKeyValueBuilder.java
new file mode 100644
index 0000000..09e295a
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/client/ClientKeyValueBuilder.java
@@ -0,0 +1,67 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.client;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValue.Type;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+
+/**
+ * A {@link KeyValueBuilder} that builds {@link ClientKeyValue}, eliminating the extra byte copies
+ * inherent in the standard {@link KeyValue} implementation.
+ * <p>
+ * This {@link KeyValueBuilder} is only supported in HBase 0.94.14+ (
+ * {@link PhoenixDatabaseMetaData#CLIENT_KEY_VALUE_BUILDER_THRESHOLD}), with the addition of
+ * HBASE-9834.
+ */
+public class ClientKeyValueBuilder extends KeyValueBuilder {
+
+    public static final KeyValueBuilder INSTANCE = new ClientKeyValueBuilder();
+
+  private ClientKeyValueBuilder() {
+    // private ctor for singleton
+  }
+
+  @Override
+  public KeyValue buildPut(ImmutableBytesWritable row, ImmutableBytesWritable family,
+      ImmutableBytesWritable qualifier, long ts, ImmutableBytesWritable value) {
+    return new ClientKeyValue(row, family, qualifier, ts, Type.Put, value);
+  }
+
+  @Override
+  public KeyValue buildDeleteFamily(ImmutableBytesWritable row, ImmutableBytesWritable family,
+            ImmutableBytesWritable qualifier, long ts) {
+        return new ClientKeyValue(row, family, qualifier, ts, Type.DeleteFamily, null);
+  }
+
+  @Override
+  public KeyValue buildDeleteColumns(ImmutableBytesWritable row, ImmutableBytesWritable family,
+            ImmutableBytesWritable qualifier, long ts) {
+        return new ClientKeyValue(row, family, qualifier, ts, Type.DeleteColumn, null);
+  }
+
+  @Override
+  public KeyValue buildDeleteColumn(ImmutableBytesWritable row, ImmutableBytesWritable family,
+            ImmutableBytesWritable qualifier, long ts) {
+        return new ClientKeyValue(row, family, qualifier, ts, Type.Delete, null);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/client/GenericKeyValueBuilder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/client/GenericKeyValueBuilder.java b/phoenix-core/src/main/java/org/apache/phoenix/client/GenericKeyValueBuilder.java
new file mode 100644
index 0000000..b3771ca
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/client/GenericKeyValueBuilder.java
@@ -0,0 +1,71 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.client;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValue.Type;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import static org.apache.hadoop.hbase.index.util.ImmutableBytesPtr.copyBytesIfNecessary;
+
+/**
+ * {@link KeyValueBuilder} that does simple byte[] copies to build the underlying key-value. This is
+ * exactly the same behavior as currently used in {@link Delete} and {@link Put}.
+ */
+public class GenericKeyValueBuilder extends KeyValueBuilder {
+
+  public static final KeyValueBuilder INSTANCE = new GenericKeyValueBuilder();
+
+  private GenericKeyValueBuilder() {
+    // private ctor for singleton
+  }
+
+  @Override
+  public KeyValue buildPut(ImmutableBytesWritable row, ImmutableBytesWritable family,
+      ImmutableBytesWritable qualifier, long ts, ImmutableBytesWritable value) {
+    return build(row, family, qualifier, ts, Type.Put, value);
+  }
+
+  @Override
+  public KeyValue buildDeleteFamily(ImmutableBytesWritable row, ImmutableBytesWritable family,
+      ImmutableBytesWritable qualifier, long ts) {
+    return build(row, family, qualifier, ts, Type.DeleteFamily, null);
+  }
+
+  @Override
+  public KeyValue buildDeleteColumns(ImmutableBytesWritable row, ImmutableBytesWritable family,
+      ImmutableBytesWritable qualifier, long ts) {
+    return build(row, family, qualifier, ts, Type.DeleteColumn, null);
+  }
+
+  @Override
+  public KeyValue buildDeleteColumn(ImmutableBytesWritable row, ImmutableBytesWritable family,
+            ImmutableBytesWritable qualifier, long ts) {
+    return build(row, family, qualifier, ts, Type.Delete, null);
+  }
+
+  private KeyValue build(ImmutableBytesWritable row, ImmutableBytesWritable family,
+      ImmutableBytesWritable qualifier, long ts, KeyValue.Type type, ImmutableBytesWritable value) {
+    return new KeyValue(copyBytesIfNecessary(row), copyBytesIfNecessary(family),
+        copyBytesIfNecessary(qualifier), ts, type, value == null? null: copyBytesIfNecessary(value));
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/client/KeyValueBuilder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/client/KeyValueBuilder.java b/phoenix-core/src/main/java/org/apache/phoenix/client/KeyValueBuilder.java
new file mode 100644
index 0000000..48608ff
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/client/KeyValueBuilder.java
@@ -0,0 +1,108 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.client;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.util.MetaDataUtil;
+
+/**
+ * Build {@link KeyValue} in an efficient way
+ */
+public abstract class KeyValueBuilder {
+
+    /**
+     * Helper method for a {@link KeyValueBuilder} that catches an IOException from a {@link Put}
+     * when adding a {@link KeyValue} generated by the KeyValueBuilder.
+     * @throws RuntimeException if there is an IOException thrown from the underlying {@link Put}
+     */
+    @SuppressWarnings("javadoc")
+    public static void addQuietly(Put put, KeyValueBuilder builder, KeyValue kv) {
+        try {
+            put.add(kv);
+        } catch (IOException e) {
+            throw new RuntimeException("KeyValue Builder " + builder + " created an invalid kv: "
+                    + kv + "!");
+        }
+    }
+
+    /**
+     * Helper method for a {@link KeyValueBuilder} that catches an IOException from a {@link Put}
+     * when adding a {@link KeyValue} generated by the KeyValueBuilder.
+     * @throws RuntimeException if there is an IOException thrown from the underlying {@link Put}
+     */
+    @SuppressWarnings("javadoc")
+    public static void deleteQuietly(Delete delete, KeyValueBuilder builder, KeyValue kv) {
+        try {
+            delete.addDeleteMarker(kv);
+        } catch (IOException e) {
+            throw new RuntimeException("KeyValue Builder " + builder + " created an invalid kv: "
+                    + kv + "!");
+        }
+    }
+
+    private static final int CUSTOM_KEY_VALUE_MIN_VERSION = MetaDataUtil.encodeVersion("0.94.14");
+
+    public static KeyValueBuilder get(String hbaseVersion) {
+        int version = MetaDataUtil.encodeVersion(hbaseVersion);
+        if (version >= CUSTOM_KEY_VALUE_MIN_VERSION) {
+            return ClientKeyValueBuilder.INSTANCE;
+        }
+        return GenericKeyValueBuilder.INSTANCE;
+    }
+
+  public KeyValue buildPut(ImmutableBytesWritable row, ImmutableBytesWritable family,
+      ImmutableBytesWritable qualifier, ImmutableBytesWritable value) {
+    return buildPut(row, family, qualifier, HConstants.LATEST_TIMESTAMP, value);
+  }
+
+  public abstract KeyValue buildPut(ImmutableBytesWritable row, ImmutableBytesWritable family,
+      ImmutableBytesWritable qualifier, long ts, ImmutableBytesWritable value);
+
+  public KeyValue buildDeleteFamily(ImmutableBytesWritable row, ImmutableBytesWritable family,
+            ImmutableBytesWritable qualifier) {
+        return buildDeleteFamily(row, family, qualifier, HConstants.LATEST_TIMESTAMP);
+  }
+
+  public abstract KeyValue buildDeleteFamily(ImmutableBytesWritable row,
+            ImmutableBytesWritable family, ImmutableBytesWritable qualifier, long ts);
+
+  public KeyValue buildDeleteColumns(ImmutableBytesWritable row, ImmutableBytesWritable family,
+            ImmutableBytesWritable qualifier) {
+        return buildDeleteColumns(row, family, qualifier, HConstants.LATEST_TIMESTAMP);
+  }
+
+  public abstract KeyValue buildDeleteColumns(ImmutableBytesWritable row,
+            ImmutableBytesWritable family, ImmutableBytesWritable qualifier, long ts);
+
+  public KeyValue buildDeleteColumn(ImmutableBytesWritable row, ImmutableBytesWritable family,
+            ImmutableBytesWritable qualifier) {
+        return buildDeleteColumn(row, family, qualifier, HConstants.LATEST_TIMESTAMP);
+  }
+
+  public abstract KeyValue buildDeleteColumn(ImmutableBytesWritable row,
+            ImmutableBytesWritable family, ImmutableBytesWritable qualifier, long ts);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/compile/AggregationManager.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/AggregationManager.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/AggregationManager.java
new file mode 100644
index 0000000..70ea3a2
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/AggregationManager.java
@@ -0,0 +1,53 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.compile;
+
+import org.apache.phoenix.expression.aggregator.ClientAggregators;
+
+/**
+ * 
+ * Class that manages aggregations during query compilation
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class AggregationManager {
+    private ClientAggregators aggregators;
+    private int position = 0;
+    
+    public AggregationManager() {
+    }
+
+    public ClientAggregators getAggregators() {
+        return aggregators;
+    }
+    
+    /**
+     * @return allocate the next available zero-based positional index
+     * for the client-side aggregate function.
+     */
+    protected int nextPosition() {
+        return position++;
+    }
+    
+    public void setAggregators(ClientAggregators clientAggregator) {
+        this.aggregators = clientAggregator;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/compile/BindManager.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/BindManager.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/BindManager.java
new file mode 100644
index 0000000..859d233
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/BindManager.java
@@ -0,0 +1,82 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.compile;
+
+import java.sql.ParameterMetaData;
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+import org.apache.phoenix.jdbc.PhoenixParameterMetaData;
+import org.apache.phoenix.parse.BindParseNode;
+import org.apache.phoenix.schema.PDatum;
+
+
+/**
+ * 
+ * Class that manages binding parameters and checking type matching. There are
+ * two main usages:
+ * 
+ * 1) the standard query case where we have the values for the binds.
+ * 2) the retrieve param metadata case where we don't have the bind values.
+ * 
+ * In both cases, during query compilation we figure out what type the bind variable
+ * "should" be, based on how it's used in the query. For example foo < ? would expect
+ * that the bind variable type matches or can be coerced to the type of foo. For (1),
+ * we check that the bind value has the correct type and for (2) we set the param
+ * metadata type.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public class BindManager {
+    public static final Object UNBOUND_PARAMETER = new Object();
+
+    private final List<Object> binds;
+    private final PhoenixParameterMetaData bindMetaData;
+
+    public BindManager(List<Object> binds) {
+        this.binds = binds;
+        this.bindMetaData = new PhoenixParameterMetaData(binds.size());
+    }
+
+    public ParameterMetaData getParameterMetaData() {
+        return bindMetaData;
+    }
+    
+    public Object getBindValue(BindParseNode node) throws SQLException {
+        int index = node.getIndex();
+        if (index < 0 || index >= binds.size()) {
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode.PARAM_INDEX_OUT_OF_BOUND)
+                .setMessage("binds size: " + binds.size() + "; index: " + index).build().buildException();
+        }
+        Object value = binds.get(index);
+        if (value == UNBOUND_PARAMETER) {
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode.PARAM_VALUE_UNBOUND)
+            .setMessage(node.toString()).build().buildException();
+        }
+        return value;
+    }
+
+    public void addParamMetaData(BindParseNode bind, PDatum column) throws SQLException {
+        bindMetaData.addParam(bind,column);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/compile/ColumnProjector.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/ColumnProjector.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/ColumnProjector.java
new file mode 100644
index 0000000..8c1697b
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/ColumnProjector.java
@@ -0,0 +1,70 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.compile;
+
+import java.sql.SQLException;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+
+
+/**
+ * 
+ * Interface used to access the value of a projected column.
+ * 
+ * @author jtaylor
+ * @since 0.1
+ */
+public interface ColumnProjector {
+    /**
+     * Get the column name as it was referenced in the query
+     * @return the database column name
+     */
+    String getName();
+    
+    /**
+     * Get the expression
+     * @return the expression for the column projector
+     */
+    public Expression getExpression();
+    
+    // TODO: An expression may contain references to multiple tables.
+    /**
+     * Get the name of the hbase table containing the column
+     * @return the hbase table name
+     */
+    String getTableName();
+    
+    /**
+     * Get the value of the column, coercing it if necessary to the specified type
+     * @param tuple the row containing the column
+     * @param type the type to which to coerce the binary value
+     * @param ptr used to retrieve the value
+     * @return the object representation of the column value.
+     * @throws SQLException
+     */
+    Object getValue(Tuple tuple, PDataType type, ImmutableBytesWritable ptr) throws SQLException;
+    
+    boolean isCaseSensitive();
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/compile/ColumnResolver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/ColumnResolver.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/ColumnResolver.java
new file mode 100644
index 0000000..49a1947
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/ColumnResolver.java
@@ -0,0 +1,55 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.compile;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.phoenix.schema.ColumnRef;
+import org.apache.phoenix.schema.TableRef;
+
+
+
+/**
+ * 
+ * Interface used to resolve column references occurring
+ * in the select statement.
+ *
+ * @author jtaylor
+ * @since 0.1
+ */
+public interface ColumnResolver {
+    
+    /**
+     * Returns the collection of resolved tables in the FROM clause.
+     */
+    public List<TableRef> getTables();
+    
+    /**
+     * Resolves column using name and alias.
+     * @param schemaName TODO
+     * @param tableName TODO
+     * @param colName TODO
+     * @return the resolved ColumnRef
+     * @throws ColumnNotFoundException if the column could not be resolved
+     * @throws AmbiguousColumnException if the column name is ambiguous
+     */
+    public ColumnRef resolveColumn(String schemaName, String tableName, String colName) throws SQLException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/50d523f6/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateIndexCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateIndexCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateIndexCompiler.java
new file mode 100644
index 0000000..718d09a
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateIndexCompiler.java
@@ -0,0 +1,88 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.compile;
+
+import java.sql.ParameterMetaData;
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.hbase.client.Scan;
+
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+import org.apache.phoenix.execute.MutationState;
+import org.apache.phoenix.expression.LiteralExpression;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixStatement;
+import org.apache.phoenix.parse.CreateIndexStatement;
+import org.apache.phoenix.parse.ParseNode;
+import org.apache.phoenix.schema.MetaDataClient;
+
+public class CreateIndexCompiler {
+    private final PhoenixStatement statement;
+
+    public CreateIndexCompiler(PhoenixStatement statement) {
+        this.statement = statement;
+    }
+
+    public MutationPlan compile(final CreateIndexStatement create) throws SQLException {
+        final PhoenixConnection connection = statement.getConnection();
+        final ColumnResolver resolver = FromCompiler.getResolver(create, connection);
+        Scan scan = new Scan();
+        final StatementContext context = new StatementContext(statement, resolver, statement.getParameters(), scan);
+        ExpressionCompiler expressionCompiler = new ExpressionCompiler(context);
+        List<ParseNode> splitNodes = create.getSplitNodes();
+        final byte[][] splits = new byte[splitNodes.size()][];
+        for (int i = 0; i < splits.length; i++) {
+            ParseNode node = splitNodes.get(i);
+            if (!node.isStateless()) {
+                throw new SQLExceptionInfo.Builder(SQLExceptionCode.SPLIT_POINT_NOT_CONSTANT)
+                    .setMessage("Node: " + node).build().buildException();
+            }
+            LiteralExpression expression = (LiteralExpression)node.accept(expressionCompiler);
+            splits[i] = expression.getBytes();
+        }
+        final MetaDataClient client = new MetaDataClient(connection);
+        
+        return new MutationPlan() {
+
+            @Override
+            public ParameterMetaData getParameterMetaData() {
+                return context.getBindManager().getParameterMetaData();
+            }
+
+            @Override
+            public PhoenixConnection getConnection() {
+                return connection;
+            }
+
+            @Override
+            public MutationState execute() throws SQLException {
+                return client.createIndex(create, splits);
+            }
+
+            @Override
+            public ExplainPlan getExplainPlan() throws SQLException {
+                return new ExplainPlan(Collections.singletonList("CREATE INDEX"));
+            }
+        };
+    }
+}