You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ra...@apache.org on 2015/04/27 10:33:45 UTC

[1/7] phoenix git commit: PHOENIX-538 Support UDFs(Rajeshbabu Chintaguntla)

Repository: phoenix
Updated Branches:
  refs/heads/master cd29be2e9 -> 66bd3e35c


http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/phoenix-core/src/main/java/org/apache/phoenix/schema/NewerFunctionAlreadyExistsException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/NewerFunctionAlreadyExistsException.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/NewerFunctionAlreadyExistsException.java
new file mode 100644
index 0000000..85457a0
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/NewerFunctionAlreadyExistsException.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.parse.PFunction;
+
+public class NewerFunctionAlreadyExistsException extends FunctionAlreadyExistsException {
+    private static final long serialVersionUID = 1L;
+    private final PFunction function;
+
+    public NewerFunctionAlreadyExistsException(String functionName) {
+        this(functionName, null);
+    }
+
+    public NewerFunctionAlreadyExistsException(String functionName, PFunction function) {
+        super(functionName);
+        this.function = function;
+    }
+
+    public PFunction getFunction() {
+        return this.function;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/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
index c104473..f015177 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaData.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaData.java
@@ -17,15 +17,19 @@
  */
 package org.apache.phoenix.schema;
 
+import org.apache.phoenix.parse.PFunction;
 import org.apache.phoenix.query.MetaDataMutated;
 
-
 public interface PMetaData extends MetaDataMutated, Iterable<PTable>, Cloneable {
+
     public static interface Pruner {
         public boolean prune(PTable table);
+        public boolean prune(PFunction function);
     }
     public int size();
     public PMetaData clone();
     public PTable getTable(PTableKey key) throws TableNotFoundException;
     public PMetaData pruneTables(Pruner pruner);
+    public PFunction getFunction(PTableKey key) throws FunctionNotFoundException;
+    public PMetaData pruneFunctions(Pruner pruner);
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataEntity.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataEntity.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataEntity.java
new file mode 100644
index 0000000..0901777
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataEntity.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 PMetaDataEntity {
+    public int getEstimatedSize();
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java
index 2f84c95..c4e3674 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java
@@ -24,6 +24,7 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.phoenix.parse.PFunction;
 import org.apache.phoenix.util.TimeKeeper;
 
 import com.google.common.collect.Lists;
@@ -60,7 +61,7 @@ public class PMetaDataImpl implements PMetaData {
             }
         }
 
-        private static class PTableCache implements Cloneable {
+        private static class PMetaDataCache implements Cloneable {
             private static final int MIN_REMOVAL_SIZE = 3;
             private static final Comparator<PTableRef> COMPARATOR = new Comparator<PTableRef>() {
                 @Override
@@ -76,6 +77,7 @@ public class PMetaDataImpl implements PMetaData {
             private final TimeKeeper timeKeeper;
 
             private final Map<PTableKey,PTableRef> tables;
+            private final Map<PTableKey,PFunction> functions;
             
             private static Map<PTableKey,PTableRef> newMap(int expectedCapacity) {
                 // Use regular HashMap, as we cannot use a LinkedHashMap that orders by access time
@@ -84,6 +86,13 @@ public class PMetaDataImpl implements PMetaData {
                 return Maps.newHashMapWithExpectedSize(expectedCapacity);
             }
 
+            private static Map<PTableKey,PFunction> newFunctionMap(int expectedCapacity) {
+                // Use regular HashMap, as we cannot use a LinkedHashMap that orders by access time
+                // safely across multiple threads (as the underlying collection is not thread safe).
+                // Instead, we track access time and prune it based on the copy we've made.
+                return Maps.newHashMapWithExpectedSize(expectedCapacity);
+            }
+
             private static Map<PTableKey,PTableRef> cloneMap(Map<PTableKey,PTableRef> tables, int expectedCapacity) {
                 Map<PTableKey,PTableRef> newTables = newMap(Math.max(tables.size(),expectedCapacity));
                 // Copy value so that access time isn't changing anymore
@@ -93,24 +102,34 @@ public class PMetaDataImpl implements PMetaData {
                 return newTables;
             }
 
-            private PTableCache(PTableCache toClone) {
+            private static Map<PTableKey,PFunction> cloneFunctionsMap(Map<PTableKey,PFunction> functions, int expectedCapacity) {
+                Map<PTableKey,PFunction> newFunctions = newFunctionMap(Math.max(functions.size(),expectedCapacity));
+                for (PFunction functionAccess : functions.values()) {
+                    newFunctions.put(functionAccess.getKey(), new PFunction(functionAccess));
+                }
+                return newFunctions;
+            }
+
+            private PMetaDataCache(PMetaDataCache toClone) {
                 this.timeKeeper = toClone.timeKeeper;
                 this.maxByteSize = toClone.maxByteSize;
                 this.currentByteSize = toClone.currentByteSize;
                 this.expectedCapacity = toClone.expectedCapacity;
-                this.tables = cloneMap(toClone.tables, toClone.expectedCapacity);
+                this.tables = cloneMap(toClone.tables, expectedCapacity);
+                this.functions = cloneFunctionsMap(toClone.functions, expectedCapacity);
             }
             
-            public PTableCache(int initialCapacity, long maxByteSize, TimeKeeper timeKeeper) {
+            public PMetaDataCache(int initialCapacity, long maxByteSize, TimeKeeper timeKeeper) {
                 this.currentByteSize = 0;
                 this.maxByteSize = maxByteSize;
                 this.expectedCapacity = initialCapacity;
-                this.tables = newMap(initialCapacity);
+                this.tables = newMap(this.expectedCapacity);
+                this.functions = newFunctionMap(this.expectedCapacity);
                 this.timeKeeper = timeKeeper;
             }
             
             public PTableRef get(PTableKey key) {
-                PTableRef tableAccess = tables.get(key);
+                PTableRef tableAccess = this.tables.get(key);
                 if (tableAccess == null) {
                     return null;
                 }
@@ -119,8 +138,8 @@ public class PMetaDataImpl implements PMetaData {
             }
             
             @Override
-            public PTableCache clone() {
-                return new PTableCache(this);
+            public PMetaDataCache clone() {
+                return new PMetaDataCache(this);
             }
 
             /**
@@ -128,16 +147,16 @@ public class PMetaDataImpl implements PMetaData {
              * Removes least recently used tables to get size of cache below its max size by
              * the overage amount.
              */
-            public PTableCache cloneMinusOverage(long overage) {
+            public PMetaDataCache cloneMinusOverage(long overage) {
                 assert(overage > 0);
                 int nToRemove = Math.max(MIN_REMOVAL_SIZE, (int)Math.ceil((currentByteSize-maxByteSize) / ((double)currentByteSize / size())) + 1);
                 MinMaxPriorityQueue<PTableRef> toRemove = BUILDER.expectedSize(nToRemove).create();
-                PTableCache newCache = new PTableCache(this.size(), this.maxByteSize, this.timeKeeper);
+                PMetaDataCache newCache = new PMetaDataCache(this.size(), this.maxByteSize, this.timeKeeper);
                 
                 long toRemoveBytes = 0;
                 // Add to new cache, but track references to remove when done
                 // to bring cache at least overage amount below it's max size.
-                for (PTableRef tableRef : tables.values()) {
+                for (PTableRef tableRef : this.tables.values()) {
                     newCache.put(tableRef.table.getKey(), new PTableRef(tableRef));
                     toRemove.add(tableRef);
                     toRemoveBytes += tableRef.estSize;
@@ -154,7 +173,7 @@ public class PMetaDataImpl implements PMetaData {
 
             private PTable put(PTableKey key, PTableRef ref) {
                 currentByteSize += ref.estSize;
-                PTableRef oldTableAccess = tables.put(key, ref);
+                PTableRef oldTableAccess = this.tables.put(key, ref);
                 PTable oldTable = null;
                 if (oldTableAccess != null) {
                     currentByteSize -= oldTableAccess.estSize;
@@ -172,7 +191,7 @@ public class PMetaDataImpl implements PMetaData {
             }
             
             public PTable remove(PTableKey key) {
-                PTableRef value = tables.remove(key);
+                PTableRef value = this.tables.remove(key);
                 if (value == null) {
                     return null;
                 }
@@ -181,7 +200,7 @@ public class PMetaDataImpl implements PMetaData {
             }
             
             public Iterator<PTable> iterator() {
-                final Iterator<PTableRef> iterator = tables.values().iterator();
+                final Iterator<PTableRef> iterator = this.tables.values().iterator();
                 return new Iterator<PTable>() {
 
                     @Override
@@ -203,7 +222,7 @@ public class PMetaDataImpl implements PMetaData {
             }
 
             public int size() {
-                return tables.size();
+                return this.tables.size();
             }
 
             public long getCurrentSize() {
@@ -215,18 +234,18 @@ public class PMetaDataImpl implements PMetaData {
             }
         }
             
-    private final PTableCache metaData;
+    private final PMetaDataCache metaData;
     
     public PMetaDataImpl(int initialCapacity, long maxByteSize) {
-        this.metaData = new PTableCache(initialCapacity, maxByteSize, TimeKeeper.SYSTEM);
+        this.metaData = new PMetaDataCache(initialCapacity, maxByteSize, TimeKeeper.SYSTEM);
     }
 
     public PMetaDataImpl(int initialCapacity, long maxByteSize, TimeKeeper timeKeeper) {
-        this.metaData = new PTableCache(initialCapacity, maxByteSize, timeKeeper);
+        this.metaData = new PMetaDataCache(initialCapacity, maxByteSize, timeKeeper);
     }
 
-    private PMetaDataImpl(PTableCache tables) {
-        this.metaData = tables.clone();
+    private PMetaDataImpl(PMetaDataCache metaData) {
+        this.metaData = metaData.clone();
     }
     
     @Override
@@ -244,6 +263,15 @@ public class PMetaDataImpl implements PMetaData {
     }
 
     @Override
+    public PFunction getFunction(PTableKey key) throws FunctionNotFoundException {
+        PFunction function = metaData.functions.get(key);
+        if (function == null) {
+            throw new FunctionNotFoundException(key.getName());
+        }
+        return function;
+    }
+
+    @Override
     public int size() {
         return metaData.size();
     }
@@ -283,18 +311,18 @@ public class PMetaDataImpl implements PMetaData {
             netGain += table.getEstimatedSize();
         }
         long overage = metaData.getCurrentSize() + netGain - metaData.getMaxSize();
-        PTableCache tables = overage <= 0 ? metaData.clone() : metaData.cloneMinusOverage(overage);
+        PMetaDataCache newMetaData = overage <= 0 ? metaData.clone() : metaData.cloneMinusOverage(overage);
         
         if (newParentTable != null) { // Upsert new index table into parent data table list
-            tables.put(newParentTable.getKey(), newParentTable);
-            tables.putDuplicate(table.getKey(), table);
+            newMetaData.put(newParentTable.getKey(), newParentTable);
+            newMetaData.putDuplicate(table.getKey(), table);
         } else {
-            tables.put(table.getKey(), table);
+            newMetaData.put(table.getKey(), table);
         }
         for (PTable index : table.getIndexes()) {
-            tables.putDuplicate(index.getKey(), index);
+            newMetaData.putDuplicate(index.getKey(), index);
         }
-        return new PMetaDataImpl(tables);
+        return new PMetaDataImpl(newMetaData);
     }
 
     @Override
@@ -318,7 +346,7 @@ public class PMetaDataImpl implements PMetaData {
 
     @Override
     public PMetaData removeTable(PName tenantId, String tableName, String parentTableName, long tableTimeStamp) throws SQLException {
-        PTableCache tables = null;
+        PMetaDataCache tables = null;
         PTableRef parentTableRef = null;
         PTableKey key = new PTableKey(tenantId, tableName);
         if (metaData.get(key) == null) {
@@ -371,7 +399,7 @@ public class PMetaDataImpl implements PMetaData {
             return this;
         }
         PTable table = tableRef.table;
-        PTableCache tables = metaData.clone();
+        PMetaDataCache tables = metaData.clone();
         for (PColumn columnToRemove : columnsToRemove) {
             PColumn column;
             String familyName = columnToRemove.getFamilyName().getString();
@@ -414,7 +442,7 @@ public class PMetaDataImpl implements PMetaData {
         if (keysToPrune.isEmpty()) {
             return this;
         }
-        PTableCache tables = metaData.clone();
+        PMetaDataCache tables = metaData.clone();
         for (PTableKey key : keysToPrune) {
             tables.remove(key);
         }
@@ -425,4 +453,36 @@ public class PMetaDataImpl implements PMetaData {
     public Iterator<PTable> iterator() {
         return metaData.iterator();
     }
+
+    @Override
+    public PMetaData addFunction(PFunction function) throws SQLException {
+        this.metaData.functions.put(function.getKey(), function);
+        return this;
+    }
+
+    @Override
+    public PMetaData removeFunction(PName tenantId, String function, long functionTimeStamp)
+            throws SQLException {
+        this.metaData.functions.remove(new PTableKey(tenantId, function));
+        return this;
+    }
+
+    @Override
+    public PMetaData pruneFunctions(Pruner pruner) {
+        List<PTableKey> keysToPrune = Lists.newArrayListWithExpectedSize(this.size());
+        for (PFunction function : this.metaData.functions.values()) {
+            if (pruner.prune(function)) {
+                keysToPrune.add(function.getKey());
+            }
+        }
+        if (keysToPrune.isEmpty()) {
+            return this;
+        }
+        PMetaDataCache clone = metaData.clone();
+        for (PTableKey key : keysToPrune) {
+            clone.functions.remove(key);
+        }
+        return new PMetaDataImpl(clone);
+    
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java
index d0fea88..e46dcb7 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java
@@ -33,7 +33,7 @@ import org.apache.phoenix.schema.stats.PTableStats;
  *
  * @since 0.1
  */
-public interface PTable {
+public interface PTable extends PMetaDataEntity {
     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;
@@ -320,7 +320,6 @@ public interface PTable {
     Short getViewIndexId();
     PTableKey getKey();
 
-    int getEstimatedSize();
     IndexType getIndexType();
     PTableStats getTableStats();
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
index f916f5b..91fd2cb 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
@@ -145,7 +145,12 @@ public class MetaDataUtil {
         Mutation m = getTableHeaderRow(tableMetadata);
         getVarChars(m.getRow(), 3, rowKeyMetaData);
     }
-    
+
+    public static void getTenantIdAndFunctionName(List<Mutation> functionMetadata, byte[][] rowKeyMetaData) {
+        Mutation m = getTableHeaderRow(functionMetadata);
+        getVarChars(m.getRow(), 2, rowKeyMetaData);
+    }
+
     public static byte[] getParentTableName(List<Mutation> tableMetadata) {
         if (tableMetadata.size() == 1) {
             return null;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
index 46da726..aff6b51 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
@@ -22,6 +22,8 @@ import static com.google.common.base.Preconditions.checkNotNull;
 import static com.google.common.base.Strings.isNullOrEmpty;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_STATS_NAME_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_FUNCTION_NAME_BYTES;
+
 
 import java.sql.SQLException;
 import java.sql.Statement;
@@ -201,6 +203,15 @@ public class SchemaUtil {
         return ByteUtil.concat(tenantId, QueryConstants.SEPARATOR_BYTE_ARRAY, schemaName, QueryConstants.SEPARATOR_BYTE_ARRAY, tableName);
     }
 
+    /**
+     * Get the key used in the Phoenix function data row for a function definition
+     * @param tenantId
+     * @param functionName
+     */
+    public static byte[] getFunctionKey(byte[] tenantId, byte[] functionName) {
+        return ByteUtil.concat(tenantId, QueryConstants.SEPARATOR_BYTE_ARRAY, functionName);
+    }
+
     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));
     }
@@ -373,7 +384,11 @@ public class SchemaUtil {
     public static boolean isMetaTable(byte[] tableName) {
         return Bytes.compareTo(tableName, SYSTEM_CATALOG_NAME_BYTES) == 0;
     }
-    
+
+    public static boolean isFunctionTable(byte[] tableName) {
+        return Bytes.compareTo(tableName, SYSTEM_FUNCTION_NAME_BYTES) == 0;
+    }
+
     public static boolean isStatsTable(byte[] tableName) {
         return Bytes.compareTo(tableName, SYSTEM_STATS_NAME_BYTES) == 0;
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/phoenix-core/src/test/java/org/apache/phoenix/parse/QueryParserTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/parse/QueryParserTest.java b/phoenix-core/src/test/java/org/apache/phoenix/parse/QueryParserTest.java
index 21a63c7..eabd789 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/parse/QueryParserTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/parse/QueryParserTest.java
@@ -289,24 +289,6 @@ public class QueryParserTest {
     }
 
     @Test
-    public void testUnknownFunction() throws Exception {
-        String sql = ((
-            "select /*gatherSlowStats*/ bogus_function(ind.key_prefix) from core.search_name_lookup ind\n" +
-            "where (ind.name = 'X')\n" +
-            "and rownum <= 2000\n" +
-            "and (ind.organization_id = '000000000000000')\n" +
-            "and (ind.key_prefix = '00T')\n" +
-            "and (ind.name_type = 't')"
-            ));
-        try {
-            parseQuery(sql);
-            fail();
-        } catch (SQLException e) {
-            assertEquals(SQLExceptionCode.UNKNOWN_FUNCTION.getErrorCode(), e.getErrorCode());
-        }
-    }
-
-    @Test
     public void testNegativeNonBooleanWhere() throws Exception {
         String sql = ((
             "select /*gatherSlowStats*/ max( distinct 1) from core.search_name_lookup ind\n" +

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/phoenix-core/src/test/java/org/apache/phoenix/query/ParallelIteratorsSplitTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/ParallelIteratorsSplitTest.java b/phoenix-core/src/test/java/org/apache/phoenix/query/ParallelIteratorsSplitTest.java
index 6f2a2f1..ecb088a 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/ParallelIteratorsSplitTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/ParallelIteratorsSplitTest.java
@@ -50,6 +50,7 @@ import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixParameterMetaData;
 import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.parse.FilterableStatement;
+import org.apache.phoenix.parse.PFunction;
 import org.apache.phoenix.parse.SelectStatement;
 import org.apache.phoenix.schema.types.PChar;
 import org.apache.phoenix.schema.ColumnRef;
@@ -313,6 +314,11 @@ public class ParallelIteratorsSplitTest extends BaseConnectionlessQueryTest {
             final ScanRanges scanRanges) throws SQLException {
         final List<TableRef> tableRefs = Collections.singletonList(tableRef);
         ColumnResolver resolver = new ColumnResolver() {
+            
+            @Override
+            public List<PFunction> getFunctions() {
+                return Collections.emptyList();
+            }
 
             @Override
             public List<TableRef> getTables() {
@@ -329,7 +335,16 @@ public class ParallelIteratorsSplitTest extends BaseConnectionlessQueryTest {
             public ColumnRef resolveColumn(String schemaName, String tableName, String colName) throws SQLException {
                 throw new UnsupportedOperationException();
             }
+
+            @Override
+            public PFunction resolveFunction(String functionName) throws SQLException {
+                throw new UnsupportedOperationException();
+            }
             
+            @Override
+            public boolean hasUDFs() {
+                return false;
+            }
         };
         PhoenixConnection connection = DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES)).unwrap(PhoenixConnection.class);
         final PhoenixStatement statement = new PhoenixStatement(connection);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/phoenix-protocol/src/main/MetaDataService.proto
----------------------------------------------------------------------
diff --git a/phoenix-protocol/src/main/MetaDataService.proto b/phoenix-protocol/src/main/MetaDataService.proto
index 19d53ef..e79f846 100644
--- a/phoenix-protocol/src/main/MetaDataService.proto
+++ b/phoenix-protocol/src/main/MetaDataService.proto
@@ -25,6 +25,7 @@ option java_generate_equals_and_hash = true;
 option optimize_for = SPEED;
 
 import "PTable.proto";
+import "PFunction.proto";
 
 enum MutationCode {
   TABLE_ALREADY_EXISTS = 0;
@@ -36,7 +37,11 @@ enum MutationCode {
   NEWER_TABLE_FOUND = 6;
   UNALLOWED_TABLE_MUTATION = 7;
   NO_PK_COLUMNS = 8;
-  PARENT_TABLE_NOT_FOUND = 9; 
+  PARENT_TABLE_NOT_FOUND = 9;
+  FUNCTION_ALREADY_EXISTS = 10;
+  FUNCTION_NOT_FOUND = 11;
+  NEWER_FUNCTION_FOUND = 12;
+  FUNCTION_NOT_IN_REGION = 13;
 };
 
 message MetaDataResponse {
@@ -46,6 +51,8 @@ message MetaDataResponse {
   repeated bytes tablesToDelete = 4;
   optional bytes columnName = 5;
   optional bytes familyName = 6;
+  optional bytes functionName = 7;
+  repeated PFunction function = 8;
 }
 
 message GetTableRequest {
@@ -56,11 +63,25 @@ message GetTableRequest {
   required int64 clientTimestamp = 5;
 }
 
+message GetFunctionsRequest {
+  required bytes tenantId = 1;
+  repeated bytes functionNames = 2;
+  repeated int64 functionTimestamps = 3;
+  required int64 clientTimestamp = 4;
+}
+
 // each byte array represents a MutationProto instance
 message CreateTableRequest {
   repeated bytes tableMetadataMutations = 1; 
 }
 
+// each byte array represents a MutationProto instance
+message CreateFunctionRequest {
+  repeated bytes tableMetadataMutations = 1;
+  required bool temporary = 2;
+  optional bool replace = 3;
+}
+
 message DropTableRequest {
   repeated bytes tableMetadataMutations = 1;
   required string tableType = 2;
@@ -75,6 +96,11 @@ message DropColumnRequest {
   repeated bytes tableMetadataMutations = 1;
 }
 
+message DropFunctionRequest {
+  repeated bytes tableMetadataMutations = 1;
+  optional bool ifExists = 2;
+}
+
 message UpdateIndexStateRequest {
   repeated bytes tableMetadataMutations = 1;
 }
@@ -106,12 +132,21 @@ service MetaDataService {
   rpc getTable(GetTableRequest)
     returns (MetaDataResponse);
 
+  rpc getFunctions(GetFunctionsRequest)
+    returns (MetaDataResponse);
+
   rpc createTable(CreateTableRequest)
     returns (MetaDataResponse);
+
+  rpc createFunction(CreateFunctionRequest)
+    returns (MetaDataResponse);
     
   rpc dropTable(DropTableRequest)
     returns (MetaDataResponse);
     
+  rpc dropFunction(DropFunctionRequest)
+    returns (MetaDataResponse);
+
    rpc addColumn(AddColumnRequest)
     returns (MetaDataResponse);
     

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/phoenix-protocol/src/main/PFunction.proto
----------------------------------------------------------------------
diff --git a/phoenix-protocol/src/main/PFunction.proto b/phoenix-protocol/src/main/PFunction.proto
new file mode 100644
index 0000000..07cbac7
--- /dev/null
+++ b/phoenix-protocol/src/main/PFunction.proto
@@ -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.
+ */
+
+option java_package = "org.apache.phoenix.coprocessor.generated";
+option java_outer_classname = "PFunctionProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+message PFunctionArg {
+  required string argumentType = 1;
+  optional bool isArrayType = 2;
+  optional bool isConstant = 3;
+  optional string defaultValue = 4;
+  optional string minValue = 5;
+  optional string maxValue = 6;
+}
+
+message PFunction {
+  required string functionName = 1;
+  repeated PFunctionArg arguments = 2;
+  required string classname = 3;
+  required int64 timeStamp = 4;
+  optional string jarPath = 5;
+  optional bytes tenantId = 6;
+  optional string returnType = 7;
+  optional bool isArrayReturnType = 8;
+}


[6/7] phoenix git commit: PHOENIX-538 Support UDFs(Rajeshbabu Chintaguntla)

Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java
index af6c712..78f54e8 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java
@@ -63,6 +63,7 @@ import org.apache.phoenix.parse.TableName;
 import org.apache.phoenix.parse.TableNode;
 import org.apache.phoenix.parse.TableNodeVisitor;
 import org.apache.phoenix.parse.TableWildcardParseNode;
+import org.apache.phoenix.parse.UDFParseNode;
 import org.apache.phoenix.parse.WildcardParseNode;
 import org.apache.phoenix.schema.ColumnNotFoundException;
 import org.apache.phoenix.schema.ColumnRef;
@@ -688,7 +689,7 @@ public class JoinCompiler {
             if (isSubselect())
                 return SubselectRewriter.applyOrderBy(SubselectRewriter.applyPostFilters(subselect, preFilters, tableNode.getAlias()), orderBy, tableNode.getAlias());
 
-            return NODE_FACTORY.select(tableNode, select.getHint(), false, selectNodes, getPreFiltersCombined(), null, null, orderBy, null, 0, false, select.hasSequence(), Collections.<SelectStatement>emptyList());
+            return NODE_FACTORY.select(tableNode, select.getHint(), false, selectNodes, getPreFiltersCombined(), null, null, orderBy, null, 0, false, select.hasSequence(), Collections.<SelectStatement>emptyList(), select.getUdfParseNodes());
         }
 
         public boolean hasFilters() {
@@ -1177,7 +1178,7 @@ public class JoinCompiler {
             TableRef tableRef = table.getTableRef();
             List<ParseNode> groupBy = tableRef.equals(groupByTableRef) ? select.getGroupBy() : null;
             List<OrderByNode> orderBy = tableRef.equals(orderByTableRef) ? select.getOrderBy() : null;
-            SelectStatement stmt = getSubqueryForOptimizedPlan(select.getHint(), table.getDynamicColumns(), tableRef, join.getColumnRefs(), table.getPreFiltersCombined(), groupBy, orderBy, table.isWildCardSelect(), select.hasSequence());
+            SelectStatement stmt = getSubqueryForOptimizedPlan(select.getHint(), table.getDynamicColumns(), tableRef, join.getColumnRefs(), table.getPreFiltersCombined(), groupBy, orderBy, table.isWildCardSelect(), select.hasSequence(), select.getUdfParseNodes());
             QueryPlan plan = statement.getConnection().getQueryServices().getOptimizer().optimize(statement, stmt);
             if (!plan.getTableRef().equals(tableRef)) {
                 replacement.put(tableRef, plan.getTableRef());
@@ -1247,7 +1248,7 @@ public class JoinCompiler {
     }
 
     private static SelectStatement getSubqueryForOptimizedPlan(HintNode hintNode, List<ColumnDef> dynamicCols, TableRef tableRef, Map<ColumnRef, ColumnRefType> columnRefs, ParseNode where, List<ParseNode> groupBy,
-            List<OrderByNode> orderBy, boolean isWildCardSelect, boolean hasSequence) {
+            List<OrderByNode> orderBy, boolean isWildCardSelect, boolean hasSequence, Map<String, UDFParseNode> udfParseNodes) {
         String schemaName = tableRef.getTable().getSchemaName().getString();
         TableName tName = TableName.create(schemaName.length() == 0 ? null : schemaName, tableRef.getTable().getTableName().getString());
         List<AliasedNode> selectList = new ArrayList<AliasedNode>();
@@ -1267,7 +1268,7 @@ public class JoinCompiler {
         String tableAlias = tableRef.getTableAlias();
         TableNode from = NODE_FACTORY.namedTable(tableAlias == null ? null : '"' + tableAlias + '"', tName, dynamicCols);
 
-        return NODE_FACTORY.select(from, hintNode, false, selectList, where, groupBy, null, orderBy, null, 0, groupBy != null, hasSequence, Collections.<SelectStatement>emptyList());
+        return NODE_FACTORY.select(from, hintNode, false, selectList, where, groupBy, null, orderBy, null, 0, groupBy != null, hasSequence, Collections.<SelectStatement>emptyList(), udfParseNodes);
     }
 
     public static PTable joinProjectedTables(PTable left, PTable right, JoinType type) throws SQLException {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/phoenix-core/src/main/java/org/apache/phoenix/compile/PostDDLCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/PostDDLCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/PostDDLCompiler.java
index 0c586f0..fcbeb7e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/PostDDLCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/PostDDLCompiler.java
@@ -34,6 +34,7 @@ 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.PFunction;
 import org.apache.phoenix.parse.SelectStatement;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.AmbiguousColumnException;
@@ -123,6 +124,11 @@ public class PostDDLCompiler {
                             public List<TableRef> getTables() {
                                 return Collections.singletonList(tableRef);
                             }
+                            
+                            public java.util.List<PFunction> getFunctions() {
+                                return Collections.emptyList();
+                            };
+                            
                             @Override
                             public TableRef resolveTable(String schemaName, String tableName)
                                     throws SQLException {
@@ -135,6 +141,14 @@ public class PostDDLCompiler {
                                         : tableRef.getTable().getColumn(colName);
                                 return new ColumnRef(tableRef, column.getPosition());
                             }
+                            
+                            public PFunction resolveFunction(String functionName) throws SQLException {
+                                throw new UnsupportedOperationException();
+                            };
+
+                            public boolean hasUDFs() {
+                                return false;
+                            };
                         };
                         PhoenixStatement statement = new PhoenixStatement(connection);
                         StatementContext context = new StatementContext(statement, resolver, scan, new SequenceManager(statement));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/phoenix-core/src/main/java/org/apache/phoenix/compile/ProjectionCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/ProjectionCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/ProjectionCompiler.java
index e84ca2a..c39db09 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/ProjectionCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/ProjectionCompiler.java
@@ -458,7 +458,7 @@ public class ProjectionCompiler {
                 projectColumnFamily(table, scan, family);
             }
         }
-        return new RowProjector(projectedColumns, estimatedByteSize, isProjectEmptyKeyValue);
+        return new RowProjector(projectedColumns, estimatedByteSize, isProjectEmptyKeyValue, resolver.hasUDFs());
     }
 
     private static void projectAllColumnFamilies(PTable table, Scan scan) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryCompiler.java
index 3100664..e877e03 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryCompiler.java
@@ -21,6 +21,7 @@ import java.sql.SQLException;
 import java.sql.SQLFeatureNotSupportedException;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Set;
 
@@ -62,6 +63,7 @@ import org.apache.phoenix.parse.SQLParser;
 import org.apache.phoenix.parse.SelectStatement;
 import org.apache.phoenix.parse.SubqueryParseNode;
 import org.apache.phoenix.parse.TableNode;
+import org.apache.phoenix.parse.UDFParseNode;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.AmbiguousColumnException;
 import org.apache.phoenix.schema.ColumnNotFoundException;
@@ -240,13 +242,13 @@ public class QueryCompiler {
                 context.setCurrentTable(table.getTableRef());
                 PTable projectedTable = table.createProjectedTable(!projectPKColumns, context);
                 TupleProjector.serializeProjectorIntoScan(context.getScan(), new TupleProjector(projectedTable));
-                context.setResolver(FromCompiler.getResolverForProjectedTable(projectedTable));
+                context.setResolver(FromCompiler.getResolverForProjectedTable(projectedTable, context.getConnection(), subquery.getUdfParseNodes()));
                 table.projectColumns(context.getScan());
                 return compileSingleQuery(context, subquery, binds, asSubquery, !asSubquery);
             }
             QueryPlan plan = compileSubquery(subquery, false);
             PTable projectedTable = table.createProjectedTable(plan.getProjector());
-            context.setResolver(FromCompiler.getResolverForProjectedTable(projectedTable));
+            context.setResolver(FromCompiler.getResolverForProjectedTable(projectedTable, context.getConnection(), subquery.getUdfParseNodes()));
             return new TupleProjectionPlan(plan, new TupleProjector(plan.getProjector()), table.compilePostFilterExpression(context));
         }
 
@@ -295,7 +297,7 @@ public class QueryCompiler {
                 } else {
                     tables[i] = null;
                 }
-                context.setResolver(FromCompiler.getResolverForProjectedTable(projectedTable));
+                context.setResolver(FromCompiler.getResolverForProjectedTable(projectedTable, context.getConnection(), query.getUdfParseNodes()));
                 joinIds[i] = new ImmutableBytesPtr(emptyByteArray); // place-holder
                 Pair<List<Expression>, List<Expression>> joinConditions = joinSpec.compileJoinConditions(context, subContext, true);
                 joinExpressions[i] = joinConditions.getFirst();
@@ -354,7 +356,7 @@ public class QueryCompiler {
                 tupleProjector = new TupleProjector(plan.getProjector());
             }
             context.setCurrentTable(rhsTableRef);
-            context.setResolver(FromCompiler.getResolverForProjectedTable(rhsProjTable));
+            context.setResolver(FromCompiler.getResolverForProjectedTable(rhsProjTable, context.getConnection(), rhs.getUdfParseNodes()));
             ImmutableBytesPtr[] joinIds = new ImmutableBytesPtr[] {new ImmutableBytesPtr(emptyByteArray)};
             Pair<List<Expression>, List<Expression>> joinConditions = lastJoinSpec.compileJoinConditions(lhsCtx, context, true);
             List<Expression> joinExpressions = joinConditions.getSecond();
@@ -364,7 +366,7 @@ public class QueryCompiler {
             int fieldPosition = needsMerge ? rhsProjTable.getColumns().size() - rhsProjTable.getPKColumns().size() : 0;
             PTable projectedTable = needsMerge ? JoinCompiler.joinProjectedTables(rhsProjTable, lhsTable, type == JoinType.Right ? JoinType.Left : type) : rhsProjTable;
             TupleProjector.serializeProjectorIntoScan(context.getScan(), tupleProjector);
-            context.setResolver(FromCompiler.getResolverForProjectedTable(projectedTable));
+            context.setResolver(FromCompiler.getResolverForProjectedTable(projectedTable, context.getConnection(), rhs.getUdfParseNodes()));
             QueryPlan rhsPlan = compileSingleQuery(context, rhs, binds, asSubquery, !asSubquery && type == JoinType.Right);
             Expression postJoinFilterExpression = joinTable.compilePostFilterExpression(context, rhsTable);
             Integer limit = null;
@@ -413,7 +415,7 @@ public class QueryCompiler {
         int fieldPosition = needsMerge ? lhsProjTable.getColumns().size() - lhsProjTable.getPKColumns().size() : 0;
         PTable projectedTable = needsMerge ? JoinCompiler.joinProjectedTables(lhsProjTable, rhsProjTable, type == JoinType.Right ? JoinType.Left : type) : lhsProjTable;
 
-        ColumnResolver resolver = FromCompiler.getResolverForProjectedTable(projectedTable);
+        ColumnResolver resolver = FromCompiler.getResolverForProjectedTable(projectedTable, context.getConnection(), new HashMap<String,UDFParseNode>(1));
         TableRef tableRef = resolver.getTables().get(0);
         StatementContext subCtx = new StatementContext(statement, resolver, ScanUtil.newScan(originalScan), new SequenceManager(statement));
         subCtx.setCurrentTable(tableRef);
@@ -422,7 +424,7 @@ public class QueryCompiler {
         context.setResolver(resolver);
         TableNode from = NODE_FACTORY.namedTable(tableRef.getTableAlias(), NODE_FACTORY.table(tableRef.getTable().getSchemaName().getString(), tableRef.getTable().getTableName().getString()));
         ParseNode where = joinTable.getPostFiltersCombined();
-        SelectStatement select = asSubquery ? NODE_FACTORY.select(from, joinTable.getStatement().getHint(), false, Collections.<AliasedNode> emptyList(), where, null, null, orderBy, null, 0, false, joinTable.getStatement().hasSequence(), Collections.<SelectStatement>emptyList())
+        SelectStatement select = asSubquery ? NODE_FACTORY.select(from, joinTable.getStatement().getHint(), false, Collections.<AliasedNode> emptyList(), where, null, null, orderBy, null, 0, false, joinTable.getStatement().hasSequence(), Collections.<SelectStatement>emptyList(), joinTable.getStatement().getUdfParseNodes())
                 : NODE_FACTORY.select(joinTable.getStatement(), from, where);
         
         return compileSingleFlatQuery(context, select, binds, asSubquery, false, innerPlan, null, isInRowKeyOrder);
@@ -505,7 +507,7 @@ public class QueryCompiler {
         if (this.projectTuples) {
             projectedTable = TupleProjectionCompiler.createProjectedTable(select, context);
             if (projectedTable != null) {
-                context.setResolver(FromCompiler.getResolverForProjectedTable(projectedTable));
+                context.setResolver(FromCompiler.getResolverForProjectedTable(projectedTable, context.getConnection(), select.getUdfParseNodes()));
             }
         }
         

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/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
index 1b35e92..c60933e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/RowProjector.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/RowProjector.java
@@ -52,9 +52,10 @@ public class RowProjector {
     private final int estimatedSize;
     private final boolean isProjectEmptyKeyValue;
     private final boolean cloneRequired;
+    private final boolean hasUDFs;
     
     public RowProjector(RowProjector projector, boolean isProjectEmptyKeyValue) {
-        this(projector.getColumnProjectors(), projector.getEstimatedRowByteSize(), isProjectEmptyKeyValue);
+        this(projector.getColumnProjectors(), projector.getEstimatedRowByteSize(), isProjectEmptyKeyValue, projector.hasUDFs);
     }
     /**
      * Construct RowProjector based on a list of ColumnProjectors.
@@ -64,6 +65,18 @@ public class RowProjector {
      * @param estimatedRowSize 
      */
     public RowProjector(List<? extends ColumnProjector> columnProjectors, int estimatedRowSize, boolean isProjectEmptyKeyValue) {
+        this(columnProjectors, estimatedRowSize, isProjectEmptyKeyValue, false);
+    }
+    /**
+     * 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 
+     * @param isProjectEmptyKeyValue
+     * @param hasUDFs
+     */
+    public RowProjector(List<? extends ColumnProjector> columnProjectors, int estimatedRowSize, boolean isProjectEmptyKeyValue, boolean hasUDFs) {
         this.columnProjectors = Collections.unmodifiableList(columnProjectors);
         int position = columnProjectors.size();
         reverseIndex = ArrayListMultimap.<String, Integer>create();
@@ -82,15 +95,18 @@ public class RowProjector {
         this.someCaseSensitive = someCaseSensitive;
         this.estimatedSize = estimatedRowSize;
         this.isProjectEmptyKeyValue = isProjectEmptyKeyValue;
+        this.hasUDFs = hasUDFs;
         boolean hasPerInvocationExpression = false;
-        for (int i = 0; i < this.columnProjectors.size(); i++) {
-            Expression expression = this.columnProjectors.get(i).getExpression();
-            if (expression.getDeterminism() == Determinism.PER_INVOCATION) {
-                hasPerInvocationExpression = true;
-                break;
+        if (!hasUDFs) {
+            for (int i = 0; i < this.columnProjectors.size(); i++) {
+                Expression expression = this.columnProjectors.get(i).getExpression();
+                if (expression.getDeterminism() == Determinism.PER_INVOCATION) {
+                    hasPerInvocationExpression = true;
+                    break;
+                }
             }
         }
-        this.cloneRequired = hasPerInvocationExpression;
+        this.cloneRequired = hasPerInvocationExpression || hasUDFs;
     }
 
     public RowProjector cloneIfNecessary() {
@@ -114,7 +130,7 @@ public class RowProjector {
         }
         return new RowProjector(clonedColProjectors, 
                 this.getEstimatedRowByteSize(),
-                this.isProjectEmptyKeyValue());
+                this.isProjectEmptyKeyValue(), this.hasUDFs);
     }
 
     public boolean isProjectEmptyKeyValue() {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/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
index b9897b1..9b54c86 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/StatementNormalizer.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/StatementNormalizer.java
@@ -30,6 +30,7 @@ import org.apache.phoenix.parse.ComparisonParseNode;
 import org.apache.phoenix.parse.DerivedTableNode;
 import org.apache.phoenix.parse.FamilyWildcardParseNode;
 import org.apache.phoenix.parse.JoinTableNode;
+import org.apache.phoenix.parse.NamedNode;
 import org.apache.phoenix.parse.JoinTableNode.JoinType;
 import org.apache.phoenix.parse.LessThanOrEqualParseNode;
 import org.apache.phoenix.parse.NamedTableNode;
@@ -99,7 +100,7 @@ public class StatementNormalizer extends ParseNodeRewriter {
             if (selectNodes != normSelectNodes) {
                 statement = NODE_FACTORY.select(statement.getFrom(), statement.getHint(), statement.isDistinct(),
                         normSelectNodes, statement.getWhere(), statement.getGroupBy(), statement.getHaving(), statement.getOrderBy(),
-                        statement.getLimit(), statement.getBindCount(), statement.isAggregate(), statement.hasSequence(), statement.getSelects());
+                        statement.getLimit(), statement.getBindCount(), statement.isAggregate(), statement.hasSequence(), statement.getSelects(), statement.getUdfParseNodes());
             }
         }
         
@@ -151,7 +152,7 @@ public class StatementNormalizer extends ParseNodeRewriter {
          }
          return super.visitLeave(node, nodes);
     }
-    
+
     @Override
     public ParseNode visitLeave(final BetweenParseNode node, List<ParseNode> nodes) throws SQLException {
        

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/phoenix-core/src/main/java/org/apache/phoenix/compile/SubqueryRewriter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/SubqueryRewriter.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/SubqueryRewriter.java
index 1746d8a..123cb6a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/SubqueryRewriter.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/SubqueryRewriter.java
@@ -341,7 +341,7 @@ public class SubqueryRewriter extends ParseNodeRewriter {
                 groupbyNodes.set(i - 1, aliasedNode.getNode());
             }
             SelectStatement derivedTableStmt = NODE_FACTORY.select(subquery, subquery.isDistinct(), derivedTableSelect, where, derivedTableGroupBy, true);
-            subquery = NODE_FACTORY.select(NODE_FACTORY.derivedTable(derivedTableAlias, derivedTableStmt), subquery.getHint(), false, selectNodes, null, groupbyNodes, null, Collections.<OrderByNode> emptyList(), null, subquery.getBindCount(), true, false, Collections.<SelectStatement>emptyList());
+            subquery = NODE_FACTORY.select(NODE_FACTORY.derivedTable(derivedTableAlias, derivedTableStmt), subquery.getHint(), false, selectNodes, null, groupbyNodes, null, Collections.<OrderByNode> emptyList(), null, subquery.getBindCount(), true, false, Collections.<SelectStatement>emptyList(), subquery.getUdfParseNodes());
         }
         
         ParseNode onNode = conditionExtractor.getJoinCondition();
@@ -364,7 +364,7 @@ public class SubqueryRewriter extends ParseNodeRewriter {
             return select;
         
         // Wrap as a derived table.
-        return NODE_FACTORY.select(NODE_FACTORY.derivedTable(ParseNodeFactory.createTempAlias(), select), HintNode.EMPTY_HINT_NODE, false, select.getSelect(), null, null, null, null, null, select.getBindCount(), false, false, Collections.<SelectStatement> emptyList());
+        return NODE_FACTORY.select(NODE_FACTORY.derivedTable(ParseNodeFactory.createTempAlias(), select), HintNode.EMPTY_HINT_NODE, false, select.getSelect(), null, null, null, null, null, select.getBindCount(), false, false, Collections.<SelectStatement> emptyList(), select.getUdfParseNodes());
     }
     
     private List<AliasedNode> fixAliasedNodes(List<AliasedNode> nodes, boolean addSelectOne) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/phoenix-core/src/main/java/org/apache/phoenix/compile/SubselectRewriter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/SubselectRewriter.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/SubselectRewriter.java
index 6862802..5a91a17 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/SubselectRewriter.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/SubselectRewriter.java
@@ -205,7 +205,7 @@ public class SubselectRewriter extends ParseNodeRewriter {
         }
         
         return NODE_FACTORY.select(subselect.getFrom(), hintRewrite, isDistinctRewrite, selectNodesRewrite, whereRewrite, groupByRewrite, 
-            havingRewrite, orderByRewrite, limitRewrite, select.getBindCount(), isAggregateRewrite, select.hasSequence(), select.getSelects());
+            havingRewrite, orderByRewrite, limitRewrite, select.getBindCount(), isAggregateRewrite, select.hasSequence(), select.getSelects(), select.getUdfParseNodes());
     }
     
     private SelectStatement applyPostFilters(SelectStatement statement, List<ParseNode> postFilters) throws SQLException {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
index fab1ad0..cd10007 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
@@ -51,6 +51,17 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_CONSTANT_BYTE
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_INDEX_ID_BYTES;
 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.jdbc.PhoenixDatabaseMetaData.FUNCTION_NAME_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.NUM_ARGS_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_ARRAY_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_CONSTANT_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DEFAULT_VALUE_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.MIN_VALUE_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.MAX_VALUE_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CLASS_NAME_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.JAR_PATH_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TYPE_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.RETURN_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;
@@ -61,6 +72,8 @@ import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
+import java.util.Comparator;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Set;
 
@@ -91,22 +104,29 @@ import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegion.RowLock;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.util.EnvironmentEdgeManager;
 import org.apache.phoenix.cache.GlobalCache;
+import org.apache.phoenix.cache.GlobalCache.FunctionBytesPtr;
+import org.apache.phoenix.compile.ScanRanges;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.AddColumnRequest;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheRequest;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheResponse;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearTableFromCacheRequest;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearTableFromCacheResponse;
+import org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateFunctionRequest;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.DropColumnRequest;
+import org.apache.phoenix.coprocessor.generated.MetaDataProtos.DropFunctionRequest;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.DropTableRequest;
+import org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetFunctionsRequest;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetTableRequest;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetVersionRequest;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetVersionResponse;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.MetaDataResponse;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.UpdateIndexStateRequest;
+import org.apache.phoenix.expression.LiteralExpression;
 import org.apache.phoenix.hbase.index.covered.update.ColumnReference;
 import org.apache.phoenix.hbase.index.util.GenericKeyValueBuilder;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
@@ -115,7 +135,10 @@ import org.apache.phoenix.index.IndexMaintainer;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.metrics.Metrics;
+import org.apache.phoenix.parse.PFunction;
+import org.apache.phoenix.parse.PFunction.FunctionArgument;
 import org.apache.phoenix.protobuf.ProtobufUtil;
+import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.AmbiguousColumnException;
 import org.apache.phoenix.schema.ColumnFamilyNotFoundException;
@@ -123,6 +146,7 @@ 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.PMetaDataEntity;
 import org.apache.phoenix.schema.PIndexState;
 import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PNameFactory;
@@ -149,6 +173,7 @@ import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.KeyValueUtil;
 import org.apache.phoenix.util.MetaDataUtil;
 import org.apache.phoenix.util.QueryUtil;
+import org.apache.phoenix.util.ScanUtil;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.ServerUtil;
 import org.slf4j.Logger;
@@ -156,6 +181,7 @@ import org.slf4j.LoggerFactory;
 
 import com.google.common.cache.Cache;
 import com.google.common.collect.Lists;
+import com.google.protobuf.ByteString;
 import com.google.protobuf.RpcCallback;
 import com.google.protobuf.RpcController;
 import com.google.protobuf.Service;
@@ -220,6 +246,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     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);
@@ -277,6 +304,52 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     
     private static final int LINK_TYPE_INDEX = 0;
 
+    private static final KeyValue CLASS_NAME_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, CLASS_NAME_BYTES);
+    private static final KeyValue JAR_PATH_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, JAR_PATH_BYTES);
+    private static final KeyValue RETURN_TYPE_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, RETURN_TYPE_BYTES);
+    private static final KeyValue NUM_ARGS_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, NUM_ARGS_BYTES);
+    private static final KeyValue TYPE_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, TYPE_BYTES);
+    private static final KeyValue IS_CONSTANT_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, IS_CONSTANT_BYTES);
+    private static final KeyValue DEFAULT_VALUE_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, DEFAULT_VALUE_BYTES);
+    private static final KeyValue MIN_VALUE_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, MIN_VALUE_BYTES);
+    private static final KeyValue MAX_VALUE_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, MAX_VALUE_BYTES);
+    private static final KeyValue IS_ARRAY_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, IS_ARRAY_BYTES);
+
+    private static final List<KeyValue> FUNCTION_KV_COLUMNS = Arrays.<KeyValue>asList(
+        EMPTY_KEYVALUE_KV,
+        CLASS_NAME_KV,
+        JAR_PATH_KV,
+        RETURN_TYPE_KV,
+        NUM_ARGS_KV
+        );
+    static {
+        Collections.sort(FUNCTION_KV_COLUMNS, KeyValue.COMPARATOR);
+    }
+    
+    private static final int CLASS_NAME_INDEX = FUNCTION_KV_COLUMNS.indexOf(CLASS_NAME_KV);
+    private static final int JAR_PATH_INDEX = FUNCTION_KV_COLUMNS.indexOf(JAR_PATH_KV);
+    private static final int RETURN_TYPE_INDEX = FUNCTION_KV_COLUMNS.indexOf(RETURN_TYPE_KV);
+    private static final int NUM_ARGS_INDEX = FUNCTION_KV_COLUMNS.indexOf(NUM_ARGS_KV);
+
+    private static final List<KeyValue> FUNCTION_ARG_KV_COLUMNS = Arrays.<KeyValue>asList(
+        TYPE_KV,
+        IS_ARRAY_KV,
+        IS_CONSTANT_KV,
+        DEFAULT_VALUE_KV,
+        MIN_VALUE_KV,
+        MAX_VALUE_KV
+        );
+    static {
+        Collections.sort(FUNCTION_ARG_KV_COLUMNS, KeyValue.COMPARATOR);
+    }
+    
+    private static final int TYPE_INDEX = FUNCTION_ARG_KV_COLUMNS.indexOf(TYPE_KV);
+    private static final int IS_ARRAY_INDEX = FUNCTION_ARG_KV_COLUMNS.indexOf(IS_ARRAY_KV);
+    private static final int IS_CONSTANT_INDEX = FUNCTION_ARG_KV_COLUMNS.indexOf(IS_CONSTANT_KV);
+    private static final int DEFAULT_VALUE_INDEX = FUNCTION_ARG_KV_COLUMNS.indexOf(DEFAULT_VALUE_KV);
+    private static final int MIN_VALUE_INDEX = FUNCTION_ARG_KV_COLUMNS.indexOf(MIN_VALUE_KV);
+    private static final int MAX_VALUE_INDEX = FUNCTION_ARG_KV_COLUMNS.indexOf(MAX_VALUE_KV);
+    
     private static PName newPName(byte[] keyBuffer, int keyOffset, int keyLength) {
         if (keyLength <= 0) {
             return null;
@@ -368,9 +441,9 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         Scan scan = MetaDataUtil.newTableRowsScan(key, MIN_TABLE_TIMESTAMP, clientTimeStamp);
         RegionScanner scanner = region.getScanner(scan);
 
-        Cache<ImmutableBytesPtr,PTable> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
+        Cache<ImmutableBytesPtr,PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
         try {
-            PTable oldTable = metaDataCache.getIfPresent(cacheKey);
+            PTable oldTable = (PTable)metaDataCache.getIfPresent(cacheKey);
             long tableTimeStamp = oldTable == null ? MIN_TABLE_TIMESTAMP-1 : oldTable.getTimeStamp();
             PTable newTable;
             newTable = getTable(scanner, clientTimeStamp, tableTimeStamp);
@@ -393,6 +466,48 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         }
     }
 
+    private List<PFunction> buildFunctions(List<byte[]> keys, HRegion region,
+            long clientTimeStamp) throws IOException, SQLException {
+        List<KeyRange> keyRanges = Lists.newArrayListWithExpectedSize(keys.size());
+        for (byte[] key : keys) {
+            byte[] stopKey = ByteUtil.concat(key, QueryConstants.SEPARATOR_BYTE_ARRAY);
+            ByteUtil.nextKey(stopKey, stopKey.length);
+            keyRanges.add(PVarbinary.INSTANCE.getKeyRange(key, true, stopKey, false));
+        }
+        Scan scan = new Scan();
+        scan.setTimeRange(MIN_TABLE_TIMESTAMP, clientTimeStamp);
+        ScanRanges scanRanges =
+                ScanRanges.create(SchemaUtil.VAR_BINARY_SCHEMA,
+                    Collections.singletonList(keyRanges), ScanUtil.SINGLE_COLUMN_SLOT_SPAN);
+        scanRanges.initializeScan(scan);
+        scan.setFilter(scanRanges.getSkipScanFilter());
+
+        RegionScanner scanner = region.getScanner(scan);
+
+        Cache<ImmutableBytesPtr,PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
+        List<PFunction> functions = new ArrayList<PFunction>();
+        PFunction function = null;
+        try {
+            for(int i = 0; i< keys.size(); i++) {
+                function = null;
+                function = getFunction(scanner);
+                if (function == null) {
+                    return null;
+                }
+                byte[] functionKey =
+                        SchemaUtil.getFunctionKey(
+                            function.getTenantId() == null ? ByteUtil.EMPTY_BYTE_ARRAY : function
+                                    .getTenantId().getBytes(), Bytes.toBytes(function
+                                    .getFunctionName()));
+                metaDataCache.put(new FunctionBytesPtr(functionKey), function);
+                functions.add(function);
+            }
+            return functions;
+        } 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);
@@ -473,6 +588,61 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         PColumn column = new PColumnImpl(colName, famName, dataType, maxLength, scale, isNullable, position-1, sortOrder, arraySize, viewConstant, isViewReferenced, expressionStr);
         columns.add(column);
     }
+    
+    private void addArgumentToFunction(List<Cell> results, PName functionName, PName type,
+        Cell[] functionKeyValues, List<FunctionArgument> arguments, short argPosition) {
+        int i = 0;
+        int j = 0;
+        while (i < results.size() && j < FUNCTION_ARG_KV_COLUMNS.size()) {
+            Cell kv = results.get(i);
+            Cell searchKv = FUNCTION_ARG_KV_COLUMNS.get(j);
+            int cmp =
+                    Bytes.compareTo(kv.getQualifierArray(), kv.getQualifierOffset(),
+                        kv.getQualifierLength(), searchKv.getQualifierArray(),
+                        searchKv.getQualifierOffset(), searchKv.getQualifierLength());
+            if (cmp == 0) {
+                functionKeyValues[j++] = kv;
+                i++;
+            } else if (cmp > 0) {
+                functionKeyValues[j++] = null;
+            } else {
+                i++; // shouldn't happen - means unexpected KV in system table column row
+            }
+        }
+
+        Cell isArrayKv = functionKeyValues[IS_ARRAY_INDEX];
+        boolean isArrayType =
+                isArrayKv == null ? false : Boolean.TRUE.equals(PBoolean.INSTANCE.toObject(
+                    isArrayKv.getValueArray(), isArrayKv.getValueOffset(),
+                    isArrayKv.getValueLength()));
+        Cell isConstantKv = functionKeyValues[IS_CONSTANT_INDEX];
+        boolean isConstant =
+                isConstantKv == null ? false : Boolean.TRUE.equals(PBoolean.INSTANCE.toObject(
+                    isConstantKv.getValueArray(), isConstantKv.getValueOffset(),
+                    isConstantKv.getValueLength()));
+        Cell defaultValueKv = functionKeyValues[DEFAULT_VALUE_INDEX];
+        String defaultValue =
+                defaultValueKv == null ? null : (String) PVarchar.INSTANCE.toObject(
+                    defaultValueKv.getValueArray(), defaultValueKv.getValueOffset(),
+                    defaultValueKv.getValueLength());
+        Cell minValueKv = functionKeyValues[MIN_VALUE_INDEX];
+        String minValue =
+                minValueKv == null ? null : (String) PVarchar.INSTANCE.toObject(
+                    minValueKv.getValueArray(), minValueKv.getValueOffset(),
+                    minValueKv.getValueLength());
+        Cell maxValueKv = functionKeyValues[MAX_VALUE_INDEX];
+        String maxValue =
+                maxValueKv == null ? null : (String) PVarchar.INSTANCE.toObject(
+                    maxValueKv.getValueArray(), maxValueKv.getValueOffset(),
+                    maxValueKv.getValueLength());
+        FunctionArgument arg =
+                new FunctionArgument(type.getString(), isArrayType, isConstant,
+                        defaultValue == null ? null : LiteralExpression.newConstant(defaultValue),
+                        minValue == null ? null : LiteralExpression.newConstant(minValue),
+                        maxValue == null ? null : LiteralExpression.newConstant(maxValue),
+                        argPosition);
+        arguments.add(arg);
+    }
 
     private PTable getTable(RegionScanner scanner, long clientTimeStamp, long tableTimeStamp)
         throws IOException, SQLException {
@@ -646,6 +816,106 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             disableWAL, multiTenant, storeNulls, viewType, viewIndexId, indexType, stats);
     }
 
+    private PFunction getFunction(RegionScanner scanner)
+            throws IOException, SQLException {
+        List<Cell> results = Lists.newArrayList();
+        scanner.next(results);
+        if (results.isEmpty()) {
+            return null;
+        }
+        Cell[] functionKeyValues = new Cell[FUNCTION_KV_COLUMNS.size()];
+        Cell[] functionArgKeyValues = new Cell[FUNCTION_ARG_KV_COLUMNS.size()];
+
+        // Create PFunction based on KeyValues from scan
+        Cell keyValue = results.get(0);
+        byte[] keyBuffer = keyValue.getRowArray();
+        int keyLength = keyValue.getRowLength();
+        int keyOffset = keyValue.getRowOffset();
+        PName tenantId = newPName(keyBuffer, keyOffset, keyLength);
+        int tenantIdLength = (tenantId == null) ? 0 : tenantId.getBytes().length;
+        if (tenantIdLength == 0) {
+            tenantId = null;
+        }
+        PName functionName =
+                newPName(keyBuffer, keyOffset + tenantIdLength + 1, keyLength - tenantIdLength - 1);
+        int functionNameLength = functionName.getBytes().length+1;
+        int offset = tenantIdLength + functionNameLength + 1;
+
+        long timeStamp = keyValue.getTimestamp();
+
+        int i = 0;
+        int j = 0;
+        while (i < results.size() && j < FUNCTION_KV_COLUMNS.size()) {
+            Cell kv = results.get(i);
+            Cell searchKv = FUNCTION_KV_COLUMNS.get(j);
+            int cmp =
+                    Bytes.compareTo(kv.getQualifierArray(), kv.getQualifierOffset(),
+                        kv.getQualifierLength(), searchKv.getQualifierArray(),
+                        searchKv.getQualifierOffset(), searchKv.getQualifierLength());
+            if (cmp == 0) {
+                timeStamp = Math.max(timeStamp, kv.getTimestamp()); // Find max timestamp of table
+                                                                    // header row
+                functionKeyValues[j++] = kv;
+                i++;
+            } else if (cmp > 0) {
+                timeStamp = Math.max(timeStamp, kv.getTimestamp());
+                functionKeyValues[j++] = null;
+            } else {
+                i++; // shouldn't happen - means unexpected KV in system table header row
+            }
+        }
+        // CLASS_NAME,NUM_ARGS and JAR_PATH are required.
+        if (functionKeyValues[CLASS_NAME_INDEX] == null || functionKeyValues[NUM_ARGS_INDEX] == null) {
+            throw new IllegalStateException(
+                    "Didn't find expected key values for function row in metadata row");
+        }
+
+        Cell classNameKv = functionKeyValues[CLASS_NAME_INDEX];
+        PName className = newPName(classNameKv.getValueArray(), classNameKv.getValueOffset(),
+            classNameKv.getValueLength());
+        Cell jarPathKv = functionKeyValues[JAR_PATH_INDEX];
+        PName jarPath = null;
+        if(jarPathKv != null) {
+            jarPath = newPName(jarPathKv.getValueArray(), jarPathKv.getValueOffset(),
+                jarPathKv.getValueLength());
+        }
+        Cell numArgsKv = functionKeyValues[NUM_ARGS_INDEX];
+        int numArgs =
+                PInteger.INSTANCE.getCodec().decodeInt(numArgsKv.getValueArray(),
+                    numArgsKv.getValueOffset(), SortOrder.getDefault());
+        Cell returnTypeKv = functionKeyValues[RETURN_TYPE_INDEX];
+        PName returnType =
+                returnTypeKv == null ? null : newPName(returnTypeKv.getValueArray(),
+                    returnTypeKv.getValueOffset(), returnTypeKv.getValueLength());
+
+        List<FunctionArgument> arguments = Lists.newArrayListWithExpectedSize(numArgs);
+        for (int k = 0; k < numArgs; k++) {
+            results.clear();
+            scanner.next(results);
+            if (results.isEmpty()) {
+                break;
+            }
+            Cell typeKv = results.get(0);
+            int typeKeyLength = typeKv.getRowLength();
+            PName typeName =
+                    newPName(typeKv.getRowArray(), typeKv.getRowOffset() + offset, typeKeyLength
+                            - offset - 3);
+            
+            int argPositionOffset =  offset + typeName.getBytes().length + 1;
+            short argPosition = Bytes.toShort(typeKv.getRowArray(), typeKv.getRowOffset() + argPositionOffset, typeKeyLength
+                - argPositionOffset);
+            addArgumentToFunction(results, functionName, typeName, functionArgKeyValues, arguments, argPosition);
+        }
+        Collections.sort(arguments, new Comparator<FunctionArgument>() {
+            @Override
+            public int compare(FunctionArgument o1, FunctionArgument o2) {
+                return o1.getArgPosition() - o2.getArgPosition();
+            }
+        });
+        return new PFunction(tenantId, functionName.getString(), arguments, returnType.getString(),
+                className.getString(), jarPath == null ? null : jarPath.getString(), timeStamp);
+    }
+    
     private PTable buildDeletedTable(byte[] key, ImmutableBytesPtr cacheKey, HRegion region,
         long clientTimeStamp) throws IOException {
         if (clientTimeStamp == HConstants.LATEST_TIMESTAMP) {
@@ -663,7 +933,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         if (!results.isEmpty() && results.get(0).getTimestamp() > clientTimeStamp) {
             Cell kv = results.get(0);
             if (kv.getTypeByte() == Type.Delete.getCode()) {
-                Cache<ImmutableBytesPtr, PTable> metaDataCache =
+                Cache<ImmutableBytesPtr, PMetaDataEntity> metaDataCache =
                         GlobalCache.getInstance(this.env).getMetaDataCache();
                 PTable table = newDeletedTableMarker(kv.getTimestamp());
                 metaDataCache.put(cacheKey, table);
@@ -673,20 +943,57 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         return null;
     }
 
+    
+    private PFunction buildDeletedFunction(byte[] key, ImmutableBytesPtr cacheKey, HRegion region,
+        long clientTimeStamp) throws IOException {
+        if (clientTimeStamp == HConstants.LATEST_TIMESTAMP) {
+            return null;
+        }
+
+        Scan scan = MetaDataUtil.newTableRowsScan(key, clientTimeStamp, HConstants.LATEST_TIMESTAMP);
+        scan.setFilter(new FirstKeyOnlyFilter());
+        scan.setRaw(true);
+        List<Cell> results = Lists.<Cell> newArrayList();
+        try (RegionScanner scanner = region.getScanner(scan);) {
+          scanner.next(results);
+        }
+        // HBase ignores the time range on a raw scan (HBASE-7362)
+        if (!results.isEmpty() && results.get(0).getTimestamp() > clientTimeStamp) {
+            Cell kv = results.get(0);
+            if (kv.getTypeByte() == Type.Delete.getCode()) {
+                Cache<ImmutableBytesPtr, PMetaDataEntity> metaDataCache =
+                        GlobalCache.getInstance(this.env).getMetaDataCache();
+                PFunction function = newDeletedFunctionMarker(kv.getTimestamp());
+                metaDataCache.put(cacheKey, function);
+                return function;
+            }
+        }
+        return null;
+    }
+
+
     private static PTable newDeletedTableMarker(long timestamp) {
         return new PTableImpl(timestamp);
     }
 
+    private static PFunction newDeletedFunctionMarker(long timestamp) {
+        return new PFunction(timestamp);
+    }
+
     private static boolean isTableDeleted(PTable table) {
         return table.getName() == null;
     }
 
+    private static boolean isFunctionDeleted(PFunction function) {
+        return function.getFunctionName() == null;
+    }
+
     private PTable loadTable(RegionCoprocessorEnvironment env, byte[] key,
         ImmutableBytesPtr cacheKey, long clientTimeStamp, long asOfTimeStamp)
         throws IOException, SQLException {
         HRegion region = env.getRegion();
-        Cache<ImmutableBytesPtr,PTable> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
-        PTable table = metaDataCache.getIfPresent(cacheKey);
+        Cache<ImmutableBytesPtr,PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
+        PTable table = (PTable)metaDataCache.getIfPresent(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;
@@ -700,6 +1007,29 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         return null;
     }
 
+    private PFunction loadFunction(RegionCoprocessorEnvironment env, byte[] key,
+            ImmutableBytesPtr cacheKey, long clientTimeStamp, long asOfTimeStamp)
+            throws IOException, SQLException {
+            HRegion region = env.getRegion();
+            Cache<ImmutableBytesPtr,PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
+            PFunction function = (PFunction)metaDataCache.getIfPresent(cacheKey);
+            // We always cache the latest version - fault in if not in cache
+            if (function != null) {
+                return function;
+            }
+            ArrayList<byte[]> arrayList = new ArrayList<byte[]>(1);
+            arrayList.add(key);
+            List<PFunction> functions = buildFunctions(arrayList, region, asOfTimeStamp);
+            if(functions != null) return functions.get(0);
+            // if not found then check if newer table already exists and add delete marker for timestamp
+            // found
+            if (function == null
+                    && (function = buildDeletedFunction(key, cacheKey, region, clientTimeStamp)) != null) {
+                return function;
+            }
+            return null;
+        }
+
 
     @Override
     public void createTable(RpcController controller, CreateTableRequest request,
@@ -801,7 +1131,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
 
                 // Invalidate the cache - the next getTable call will add it
                 // TODO: consider loading the table that was just created here, patching up the parent table, and updating the cache
-                Cache<ImmutableBytesPtr,PTable> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
+                Cache<ImmutableBytesPtr,PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
                 if (parentCacheKey != null) {
                     metaDataCache.invalidate(parentCacheKey);
                 }
@@ -950,7 +1280,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     done.run(MetaDataMutationResult.toProto(result));
                     return;
                 }
-                Cache<ImmutableBytesPtr,PTable> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
+                Cache<ImmutableBytesPtr,PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
                 // Commit the list of deletion.
                 region.mutateRowsWithLocks(tableMetadata, Collections.<byte[]> emptySet());
                 long currentTime = MetaDataUtil.getClientTimeStamp(tableMetadata);
@@ -984,8 +1314,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         HRegion region = env.getRegion();
         ImmutableBytesPtr cacheKey = new ImmutableBytesPtr(key);
 
-        Cache<ImmutableBytesPtr,PTable> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
-        PTable table = metaDataCache.getIfPresent(cacheKey);
+        Cache<ImmutableBytesPtr,PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
+        PTable table = (PTable)metaDataCache.getIfPresent(cacheKey);
 
         // We always cache the latest version - fault in if not in cache
         if (table != null
@@ -1132,8 +1462,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 ImmutableBytesPtr cacheKey = new ImmutableBytesPtr(key);
                 List<ImmutableBytesPtr> invalidateList = new ArrayList<ImmutableBytesPtr>();
                 invalidateList.add(cacheKey);
-                Cache<ImmutableBytesPtr,PTable> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
-                PTable table = metaDataCache.getIfPresent(cacheKey);
+                Cache<ImmutableBytesPtr,PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
+                PTable table = (PTable)metaDataCache.getIfPresent(cacheKey);
                 if (logger.isDebugEnabled()) {
                     if (table == null) {
                         logger.debug("Table " + Bytes.toStringBinary(key)
@@ -1299,9 +1629,9 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
 
     private PTable doGetTable(byte[] key, long clientTimeStamp, RowLock rowLock) throws IOException, SQLException {
         ImmutableBytesPtr cacheKey = new ImmutableBytesPtr(key);
-        Cache<ImmutableBytesPtr, PTable> metaDataCache =
+        Cache<ImmutableBytesPtr, PMetaDataEntity> metaDataCache =
                 GlobalCache.getInstance(this.env).getMetaDataCache();
-        PTable table = metaDataCache.getIfPresent(cacheKey);
+        PTable table = (PTable)metaDataCache.getIfPresent(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
@@ -1333,7 +1663,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         }
         try {
             // Try cache again in case we were waiting on a lock
-            table = metaDataCache.getIfPresent(cacheKey);
+            table = (PTable)metaDataCache.getIfPresent(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
@@ -1358,6 +1688,64 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         }
     }
 
+    private List<PFunction> doGetFunctions(List<byte[]> keys, long clientTimeStamp) throws IOException, SQLException {
+        Cache<ImmutableBytesPtr, PMetaDataEntity> metaDataCache =
+                GlobalCache.getInstance(this.env).getMetaDataCache();
+        HRegion region = env.getRegion();
+        Collections.sort(keys, new Comparator<byte[]>() {
+            @Override
+            public int compare(byte[] o1, byte[] o2) {
+                return Bytes.compareTo(o1, o2);
+            }
+        });
+        /*
+         * Lock directly on key, though it may be an index table. This will just prevent a table
+         * from getting rebuilt too often.
+         */
+        List<RowLock> rowLocks = new ArrayList<HRegion.RowLock>(keys.size());;
+        try {
+            rowLocks = new ArrayList<HRegion.RowLock>(keys.size());
+            for (int i = 0; i < keys.size(); i++) {
+                HRegion.RowLock rowLock = region.getRowLock(keys.get(i));
+                if (rowLock == null) {
+                    throw new IOException("Failed to acquire lock on "
+                            + Bytes.toStringBinary(keys.get(i)));
+                }
+                rowLocks.add(rowLock);
+            }
+
+            List<PFunction> functionsAvailable = new ArrayList<PFunction>(keys.size());
+            int numFunctions = keys.size(); 
+            Iterator<byte[]> iterator = keys.iterator();
+            while(iterator.hasNext()) {
+                byte[] key = iterator.next();
+                PFunction function = (PFunction)metaDataCache.getIfPresent(new FunctionBytesPtr(key));
+                if (function != null && function.getTimeStamp() < clientTimeStamp) {
+                    if (isFunctionDeleted(function)) {
+                        return null;
+                    }
+                    functionsAvailable.add(function);
+                    iterator.remove();
+                }
+            }
+            if(functionsAvailable.size() == numFunctions) return functionsAvailable;
+
+            // Query for the latest table first, since it's not cached
+            List<PFunction> buildFunctions = buildFunctions(keys, region, clientTimeStamp);
+            if(buildFunctions == null || buildFunctions.isEmpty()) {
+                return null;
+            }
+            functionsAvailable.addAll(buildFunctions);
+            if(functionsAvailable.size() == numFunctions) return functionsAvailable;
+            return null;
+        } finally {
+            for (HRegion.RowLock lock : rowLocks) {
+                lock.release();
+            }
+            rowLocks.clear();
+        }
+    }
+
     @Override
     public void dropColumn(RpcController controller, DropColumnRequest request,
             RpcCallback<MetaDataResponse> done) {
@@ -1478,7 +1866,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     public void clearCache(RpcController controller, ClearCacheRequest request,
             RpcCallback<ClearCacheResponse> done) {
         GlobalCache cache = GlobalCache.getInstance(this.env);
-        Cache<ImmutableBytesPtr, PTable> metaDataCache =
+        Cache<ImmutableBytesPtr, PMetaDataEntity> metaDataCache =
                 GlobalCache.getInstance(this.env).getMetaDataCache();
         metaDataCache.invalidateAll();
         cache.clearTenantCache();
@@ -1635,7 +2023,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     }
                     region.mutateRowsWithLocks(tableMetadata, Collections.<byte[]> emptySet());
                     // Invalidate from cache
-                    Cache<ImmutableBytesPtr,PTable> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
+                    Cache<ImmutableBytesPtr,PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
                     metaDataCache.invalidate(cacheKey);
                     if(dataTableKey != null) {
                         metaDataCache.invalidate(new ImmutableBytesPtr(dataTableKey));
@@ -1670,6 +2058,18 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 EnvironmentEdgeManager.currentTimeMillis(), null);
     }
 
+    private static MetaDataMutationResult checkFunctionKeyInRegion(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.FUNCTION_NOT_IN_REGION,
+                EnvironmentEdgeManager.currentTimeMillis(), null);
+    }
+
     /**
      * Certain operations, such as DROP TABLE are not allowed if there a table has child views. This class wraps the
      * Results of a scanning the Phoenix Metadata for child views for a specific table and stores an additional flag for
@@ -1720,7 +2120,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             byte[] tenantId = request.getTenantId().toByteArray();
             byte[] key = SchemaUtil.getTableKey(tenantId, schemaName, tableName);
             ImmutableBytesPtr cacheKey = new ImmutableBytesPtr(key);
-            Cache<ImmutableBytesPtr, PTable> metaDataCache =
+            Cache<ImmutableBytesPtr, PMetaDataEntity> metaDataCache =
                     GlobalCache.getInstance(this.env).getMetaDataCache();
             metaDataCache.invalidate(cacheKey);
         } catch (Throwable t) {
@@ -1729,5 +2129,222 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 ServerUtil.createIOException(SchemaUtil.getTableName(schemaName, tableName), t));
         }
     }
+    @Override
+    public void getFunctions(RpcController controller, GetFunctionsRequest request,
+            RpcCallback<MetaDataResponse> done) {
+        MetaDataResponse.Builder builder = MetaDataResponse.newBuilder();
+        byte[] tenantId = request.getTenantId().toByteArray();
+        List<String> functionNames = new ArrayList<>(request.getFunctionNamesCount());
+        try {
+            HRegion region = env.getRegion();
+            List<ByteString> functionNamesList = request.getFunctionNamesList();
+            List<Long> functionTimestampsList = request.getFunctionTimestampsList();
+            List<byte[]> keys = new ArrayList<byte[]>(request.getFunctionNamesCount());
+            List<Pair<byte[], Long>> functions = new ArrayList<Pair<byte[], Long>>(request.getFunctionNamesCount());
+            for(int i = 0; i< functionNamesList.size();i++) {
+                byte[] functionName = functionNamesList.get(i).toByteArray();
+                functionNames.add(Bytes.toString(functionName));
+                byte[] key = SchemaUtil.getFunctionKey(tenantId, functionName);
+                MetaDataMutationResult result = checkFunctionKeyInRegion(key, region);
+                if (result != null) {
+                    done.run(MetaDataMutationResult.toProto(result));
+                    return;
+                }
+                functions.add(new Pair<byte[], Long>(functionName,functionTimestampsList.get(i)));
+                keys.add(key);
+            }
+
+            long currentTime = EnvironmentEdgeManager.currentTimeMillis();
+            List<PFunction> functionsAvailable = doGetFunctions(keys, request.getClientTimestamp());
+            if (functionsAvailable == null) {
+                builder.setReturnCode(MetaDataProtos.MutationCode.FUNCTION_NOT_FOUND);
+                builder.setMutationTime(currentTime);
+                done.run(builder.build());
+                return;
+            }
+            builder.setReturnCode(MetaDataProtos.MutationCode.FUNCTION_ALREADY_EXISTS);
+            builder.setMutationTime(currentTime);
+
+            for (PFunction function : functionsAvailable) {
+                builder.addFunction(PFunction.toProto(function));
+            }
+            done.run(builder.build());
+            return;
+        } catch (Throwable t) {
+            logger.error("getFunctions failed", t);
+            ProtobufUtil.setControllerException(controller,
+                ServerUtil.createIOException(functionNames.toString(), t));
+        }
+    }
 
+    @Override
+    public void createFunction(RpcController controller, CreateFunctionRequest request,
+            RpcCallback<MetaDataResponse> done) {
+        MetaDataResponse.Builder builder = MetaDataResponse.newBuilder();
+        byte[][] rowKeyMetaData = new byte[2][];
+        byte[] functionName = null;
+        try {
+            List<Mutation> functionMetaData = ProtobufUtil.getMutations(request);
+            boolean temporaryFunction = request.getTemporary();
+            MetaDataUtil.getTenantIdAndFunctionName(functionMetaData, rowKeyMetaData);
+            byte[] tenantIdBytes = rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
+            functionName = rowKeyMetaData[PhoenixDatabaseMetaData.FUNTION_NAME_INDEX];
+            byte[] lockKey = SchemaUtil.getFunctionKey(tenantIdBytes, functionName);
+            HRegion region = env.getRegion();
+            MetaDataMutationResult result = checkFunctionKeyInRegion(lockKey, region);
+            if (result != null) {
+                done.run(MetaDataMutationResult.toProto(result));
+                return;
+            }
+            List<RowLock> locks = Lists.newArrayList();
+            long clientTimeStamp = MetaDataUtil.getClientTimeStamp(functionMetaData);
+            try {
+                acquireLock(region, lockKey, locks);
+                // Get as of latest timestamp so we can detect if we have a newer function that already
+                // exists without making an additional query
+                ImmutableBytesPtr cacheKey = new FunctionBytesPtr(lockKey);
+                PFunction function =
+                        loadFunction(env, lockKey, cacheKey, clientTimeStamp, clientTimeStamp);
+                if (function != null) {
+                    if (function.getTimeStamp() < clientTimeStamp) {
+                        // If the function is older than the client time stamp and it's deleted,
+                        // continue
+                        if (!isFunctionDeleted(function)) {
+                            builder.setReturnCode(MetaDataProtos.MutationCode.FUNCTION_ALREADY_EXISTS);
+                            builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
+                            builder.addFunction(PFunction.toProto(function));
+                            done.run(builder.build());
+                            return;
+                        }
+                    } else {
+                        builder.setReturnCode(MetaDataProtos.MutationCode.NEWER_FUNCTION_FOUND);
+                        builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
+                        builder.addFunction(PFunction.toProto(function));
+                        done.run(builder.build());
+                        return;
+                    }
+                }
+                // Don't store function info for temporary functions.
+                if(!temporaryFunction) {
+                    region.mutateRowsWithLocks(functionMetaData, Collections.<byte[]> emptySet());
+                }
+
+                // Invalidate the cache - the next getFunction call will add it
+                // TODO: consider loading the function that was just created here, patching up the parent function, and updating the cache
+                Cache<ImmutableBytesPtr,PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
+                metaDataCache.invalidate(cacheKey);
+                // Get timeStamp from mutations - the above method sets it if it's unset
+                long currentTimeStamp = MetaDataUtil.getClientTimeStamp(functionMetaData);
+                builder.setReturnCode(MetaDataProtos.MutationCode.FUNCTION_NOT_FOUND);
+                builder.setMutationTime(currentTimeStamp);
+                done.run(builder.build());
+                return;
+            } finally {
+                region.releaseRowLocks(locks);
+            }
+        } catch (Throwable t) {
+          logger.error("createFunction failed", t);
+            ProtobufUtil.setControllerException(controller,
+                ServerUtil.createIOException(Bytes.toString(functionName), t));
+        }         
+    }
+
+    @Override
+    public void dropFunction(RpcController controller, DropFunctionRequest request,
+            RpcCallback<MetaDataResponse> done) {
+        byte[][] rowKeyMetaData = new byte[2][];
+        byte[] functionName = null;
+        try {
+            List<Mutation> functionMetaData = ProtobufUtil.getMutations(request);
+            MetaDataUtil.getTenantIdAndFunctionName(functionMetaData, rowKeyMetaData);
+            byte[] tenantIdBytes = rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
+            functionName = rowKeyMetaData[PhoenixDatabaseMetaData.FUNTION_NAME_INDEX];
+            byte[] lockKey = SchemaUtil.getFunctionKey(tenantIdBytes, functionName);
+            HRegion region = env.getRegion();
+            MetaDataMutationResult result = checkFunctionKeyInRegion(lockKey, region);
+            if (result != null) {
+                done.run(MetaDataMutationResult.toProto(result));
+                return;
+            }
+            List<RowLock> locks = Lists.newArrayList();
+            long clientTimeStamp = MetaDataUtil.getClientTimeStamp(functionMetaData);
+            try {
+                acquireLock(region, lockKey, locks);
+                ImmutableBytesPtr cacheKey = new FunctionBytesPtr(lockKey);
+                List<byte[]> keys = new ArrayList<byte[]>(1);
+                keys.add(lockKey);
+                List<ImmutableBytesPtr> invalidateList = new ArrayList<ImmutableBytesPtr>();
+
+                result = doDropFunction(clientTimeStamp, keys, functionMetaData, invalidateList);
+                if (result.getMutationCode() != MutationCode.FUNCTION_ALREADY_EXISTS) {
+                    done.run(MetaDataMutationResult.toProto(result));
+                    return;
+                }
+                region.mutateRowsWithLocks(functionMetaData, Collections.<byte[]> emptySet());
+
+                Cache<ImmutableBytesPtr,PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
+                long currentTime = MetaDataUtil.getClientTimeStamp(functionMetaData);
+                for(ImmutableBytesPtr ptr: invalidateList) {
+                    metaDataCache.invalidate(ptr);
+                    metaDataCache.put(ptr, newDeletedFunctionMarker(currentTime));
+                    
+                }
+
+                done.run(MetaDataMutationResult.toProto(result));
+                return;
+            } finally {
+                region.releaseRowLocks(locks);
+            }
+        } catch (Throwable t) {
+          logger.error("dropFunction failed", t);
+            ProtobufUtil.setControllerException(controller,
+                ServerUtil.createIOException(Bytes.toString(functionName), t));
+        }         
+    }
+
+    private MetaDataMutationResult doDropFunction(long clientTimeStamp, List<byte[]> keys, List<Mutation> functionMetaData, List<ImmutableBytesPtr> invalidateList)
+            throws IOException, SQLException {
+        List<byte[]> keysClone = new ArrayList<byte[]>(keys);
+        List<PFunction> functions = doGetFunctions(keysClone, clientTimeStamp);
+        // We didn't find a table at the latest timestamp, so either there is no table or
+        // there was a table, but it's been deleted. In either case we want to return.
+        if (functions == null || functions.isEmpty()) {
+            if (buildDeletedFunction(keys.get(0), new FunctionBytesPtr(keys.get(0)), env.getRegion(), clientTimeStamp) != null) {
+                return new MetaDataMutationResult(MutationCode.FUNCTION_ALREADY_EXISTS, EnvironmentEdgeManager.currentTimeMillis(), null);
+            }
+            return new MetaDataMutationResult(MutationCode.FUNCTION_NOT_FOUND, EnvironmentEdgeManager.currentTimeMillis(), null);
+        }
+
+        if (functions != null && !functions.isEmpty()) {
+            if (functions.get(0).getTimeStamp() < clientTimeStamp) {
+                // If the function is older than the client time stamp and it's deleted,
+                // continue
+                if (isFunctionDeleted(functions.get(0))) {
+                    return new MetaDataMutationResult(MutationCode.FUNCTION_NOT_FOUND,
+                            EnvironmentEdgeManager.currentTimeMillis(), null);
+                }
+                invalidateList.add(new FunctionBytesPtr(keys.get(0)));
+                HRegion region = env.getRegion();
+                Scan scan = MetaDataUtil.newTableRowsScan(keys.get(0), MIN_TABLE_TIMESTAMP, clientTimeStamp);
+                List<Cell> results = Lists.newArrayList();
+                try (RegionScanner scanner = region.getScanner(scan);) {
+                  scanner.next(results);
+                  if (results.isEmpty()) { // Should not be possible
+                    return new MetaDataMutationResult(MutationCode.FUNCTION_NOT_FOUND, EnvironmentEdgeManager.currentTimeMillis(), null);
+                  }
+                  do {
+                    Cell kv = results.get(0);
+                    Delete delete = new Delete(kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(), clientTimeStamp);
+                    functionMetaData.add(delete);
+                    results.clear();
+                    scanner.next(results);
+                  } while (!results.isEmpty());
+                }
+                return new MetaDataMutationResult(MutationCode.FUNCTION_ALREADY_EXISTS,
+                        EnvironmentEdgeManager.currentTimeMillis(), functions, true);
+            }
+        }
+        return new MetaDataMutationResult(MutationCode.FUNCTION_NOT_FOUND,
+                EnvironmentEdgeManager.currentTimeMillis(), null);
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
index 3ef6e80..2cca4bc 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
@@ -17,13 +17,16 @@
  */
 package org.apache.phoenix.coprocessor;
 
+import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos;
+import org.apache.phoenix.coprocessor.generated.PFunctionProtos;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.MetaDataResponse;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.MetaDataService;
 import org.apache.phoenix.hbase.index.util.VersionUtil;
+import org.apache.phoenix.parse.PFunction;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableImpl;
@@ -87,6 +90,10 @@ public abstract class MetaDataProtocol extends MetaDataService {
         UNALLOWED_TABLE_MUTATION,
         NO_PK_COLUMNS,
         PARENT_TABLE_NOT_FOUND,
+        FUNCTION_ALREADY_EXISTS,
+        FUNCTION_NOT_FOUND,
+        NEWER_FUNCTION_FOUND,
+        FUNCTION_NOT_IN_REGION,
         NO_OP
     };
 
@@ -98,6 +105,7 @@ public abstract class MetaDataProtocol extends MetaDataService {
         private byte[] columnName;
         private byte[] familyName;
         private boolean wasUpdated;
+        private List<PFunction> functions = new ArrayList<PFunction>(1);
 
         public MetaDataMutationResult() {
         }
@@ -114,12 +122,19 @@ public abstract class MetaDataProtocol extends MetaDataService {
            this(returnCode, currentTime, table, Collections.<byte[]> emptyList());
         }
 
+        public MetaDataMutationResult(MutationCode returnCode, long currentTime, List<PFunction> functions, boolean wasUpdated) {
+            this.returnCode = returnCode;
+            this.mutationTime = currentTime;
+            this.functions = functions;
+            this.wasUpdated = wasUpdated;
+         }
+
         // For testing, so that connectionless can set wasUpdated so ColumnResolver doesn't complain
         public MetaDataMutationResult(MutationCode returnCode, long currentTime, PTable table, boolean wasUpdated) {
             this(returnCode, currentTime, table, Collections.<byte[]> emptyList());
             this.wasUpdated = wasUpdated;
          }
-
+        
         public MetaDataMutationResult(MutationCode returnCode, long currentTime, PTable table, List<byte[]> tableNamesToDelete) {
             this.returnCode = returnCode;
             this.mutationTime = currentTime;
@@ -146,6 +161,10 @@ public abstract class MetaDataProtocol extends MetaDataService {
         public void setTable(PTable table) {
             this.table = table;
         }
+        
+        public void setFunction(PFunction function) {
+            this.functions.add(function);
+        }
 
         public List<byte[]> getTableNamesToDelete() {
             return tableNamesToDelete;
@@ -159,6 +178,10 @@ public abstract class MetaDataProtocol extends MetaDataService {
             return familyName;
         }
 
+        public List<PFunction> getFunctions() {
+            return functions;
+        }
+
         public static MetaDataMutationResult constructFromProto(MetaDataResponse proto) {
           MetaDataMutationResult result = new MetaDataMutationResult();
           result.returnCode = MutationCode.values()[proto.getReturnCode().ordinal()];
@@ -167,6 +190,11 @@ public abstract class MetaDataProtocol extends MetaDataService {
             result.wasUpdated = true;
             result.table = PTableImpl.createFromProto(proto.getTable());
           }
+          if (proto.getFunctionCount() > 0) {
+              result.wasUpdated = true;
+              for(PFunctionProtos.PFunction function: proto.getFunctionList())
+              result.functions.add(PFunction.createFromProto(function));
+          }
           if (proto.getTablesToDeleteCount() > 0) {
             result.tableNamesToDelete =
                 Lists.newArrayListWithExpectedSize(proto.getTablesToDeleteCount());


[2/7] phoenix git commit: PHOENIX-538 Support UDFs(Rajeshbabu Chintaguntla)

Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/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
index 5eb641e..291c84c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
@@ -22,6 +22,7 @@ import java.math.BigDecimal;
 import java.sql.SQLException;
 import java.util.Arrays;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -87,11 +88,11 @@ public class ParseNodeFactory {
      * 
      * @since 0.1
      */
-    private static class BuiltInFunctionKey {
+    public static class BuiltInFunctionKey {
         private final String upperName;
         private final int argCount;
 
-        private BuiltInFunctionKey(String lowerName, int argCount) {
+        public BuiltInFunctionKey(String lowerName, int argCount) {
             this.upperName = lowerName;
             this.argCount = argCount;
         }
@@ -180,9 +181,6 @@ public class ParseNodeFactory {
     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;
     }
 
@@ -288,8 +286,8 @@ public class ParseNodeFactory {
         return new CreateTableStatement(tableName, props, columns, pkConstraint, splits, tableType, ifNotExists, baseTableName, tableTypeIdNode, bindCount);
     }
 
-    public CreateIndexStatement createIndex(NamedNode indexName, NamedTableNode dataTable, IndexKeyConstraint ikConstraint, List<ColumnName> includeColumns, List<ParseNode> splits, ListMultimap<String,Pair<String,Object>> props, boolean ifNotExists, IndexType indexType,boolean async, int bindCount) {
-        return new CreateIndexStatement(indexName, dataTable, ikConstraint, includeColumns, splits, props, ifNotExists, indexType, async, bindCount);
+    public CreateIndexStatement createIndex(NamedNode indexName, NamedTableNode dataTable, IndexKeyConstraint ikConstraint, List<ColumnName> includeColumns, List<ParseNode> splits, ListMultimap<String,Pair<String,Object>> props, boolean ifNotExists, IndexType indexType,boolean async, int bindCount, Map<String, UDFParseNode> udfParseNodes) {
+        return new CreateIndexStatement(indexName, dataTable, ikConstraint, includeColumns, splits, props, ifNotExists, indexType, async, bindCount, udfParseNodes);
     }
 
     public CreateSequenceStatement createSequence(TableName tableName, ParseNode startsWith,
@@ -299,6 +297,14 @@ public class ParseNodeFactory {
                 maxValue, cycle, ifNotExits, bindCount);
     }
 
+    public CreateFunctionStatement createFunction(PFunction functionInfo, boolean temporary) {
+        return new CreateFunctionStatement(functionInfo, temporary);
+    }
+
+    public DropFunctionStatement dropFunction(String functionName, boolean ifExists) {
+        return new DropFunctionStatement(functionName, ifExists);
+    }
+
     public DropSequenceStatement dropSequence(TableName tableName, boolean ifExits, int bindCount){
         return new DropSequenceStatement(tableName, ifExits, bindCount);
     }
@@ -388,6 +394,9 @@ public class ParseNodeFactory {
 
     public FunctionParseNode function(String name, List<ParseNode> args) {
         BuiltInFunctionInfo info = getInfo(name, args);
+        if (info == null) {
+            return new UDFParseNode(name, args, info);
+        }
         Constructor<? extends FunctionParseNode> ctor = info.getNodeCtor();
         if (ctor == null) {
             return info.isAggregate()
@@ -411,6 +420,9 @@ public class ParseNodeFactory {
         args.addAll(valueNodes);
 
         BuiltInFunctionInfo info = getInfo(name, args);
+        if(info==null) {
+            return new UDFParseNode(name,args,info);
+        }
         Constructor<? extends FunctionParseNode> ctor = info.getNodeCtor();
         if (ctor == null) {
             return new AggregateFunctionWithinGroupParseNode(name, args, info);
@@ -657,100 +669,100 @@ public class ParseNodeFactory {
 
     public SelectStatement select(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, 
-            boolean hasSequence, List<SelectStatement> selects) {
+            boolean hasSequence, List<SelectStatement> selects, Map<String, UDFParseNode> udfParseNodes) {
 
         return new SelectStatement(from, hint, isDistinct, select, where, groupBy == null ? Collections.<ParseNode>emptyList() : groupBy, having,
-                orderBy == null ? Collections.<OrderByNode>emptyList() : orderBy, limit, bindCount, isAggregate, hasSequence, selects == null ? Collections.<SelectStatement>emptyList() : selects);
+                orderBy == null ? Collections.<OrderByNode>emptyList() : orderBy, limit, bindCount, isAggregate, hasSequence, selects == null ? Collections.<SelectStatement>emptyList() : selects, udfParseNodes);
     } 
     
-    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 UpsertStatement upsert(NamedTableNode table, HintNode hint, List<ColumnName> columns, List<ParseNode> values, SelectStatement select, int bindCount, Map<String, UDFParseNode> udfParseNodes) {
+        return new UpsertStatement(table, hint, columns, values, select, bindCount, udfParseNodes);
     }
 
-    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 DeleteStatement delete(NamedTableNode table, HintNode hint, ParseNode node, List<OrderByNode> orderBy, LimitNode limit, int bindCount, Map<String, UDFParseNode> udfParseNodes) {
+        return new DeleteStatement(table, hint, node, orderBy, limit, bindCount, udfParseNodes);
     }
 
     public SelectStatement select(SelectStatement statement, ParseNode where) {
         return select(statement.getFrom(), statement.getHint(), statement.isDistinct(), statement.getSelect(), where, statement.getGroupBy(), statement.getHaving(),
-                statement.getOrderBy(), statement.getLimit(), statement.getBindCount(), statement.isAggregate(), statement.hasSequence(), statement.getSelects());
+                statement.getOrderBy(), statement.getLimit(), statement.getBindCount(), statement.isAggregate(), statement.hasSequence(), statement.getSelects(), statement.getUdfParseNodes());
     }
 
     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(), statement.hasSequence(), statement.getSelects());
+                statement.getOrderBy(), statement.getLimit(), statement.getBindCount(), statement.isAggregate(), statement.hasSequence(), statement.getSelects(), statement.getUdfParseNodes());
     }
     
     public SelectStatement select(SelectStatement statement, List<AliasedNode> select, ParseNode where, List<ParseNode> groupBy, ParseNode having, List<OrderByNode> orderBy) {
         return select(statement.getFrom(), statement.getHint(), statement.isDistinct(), 
-                select, where, groupBy, having, orderBy, statement.getLimit(), statement.getBindCount(), statement.isAggregate(), statement.hasSequence(), statement.getSelects());
+                select, where, groupBy, having, orderBy, statement.getLimit(), statement.getBindCount(), statement.isAggregate(), statement.hasSequence(), statement.getSelects(), statement.getUdfParseNodes());
     }
     
     public SelectStatement select(SelectStatement statement, TableNode table) {
         return select(table, statement.getHint(), statement.isDistinct(), statement.getSelect(), statement.getWhere(), statement.getGroupBy(),
                 statement.getHaving(), statement.getOrderBy(), statement.getLimit(), statement.getBindCount(), statement.isAggregate(),
-                statement.hasSequence(), statement.getSelects());
+                statement.hasSequence(), statement.getSelects(), statement.getUdfParseNodes());
     }
 
     public SelectStatement select(SelectStatement statement, TableNode table, ParseNode where) {
         return select(table, statement.getHint(), statement.isDistinct(), statement.getSelect(), where, statement.getGroupBy(),
                 statement.getHaving(), statement.getOrderBy(), statement.getLimit(), statement.getBindCount(), statement.isAggregate(),
-                statement.hasSequence(), statement.getSelects());
+                statement.hasSequence(), statement.getSelects(), statement.getUdfParseNodes());
     }
 
     public SelectStatement select(SelectStatement statement, boolean isDistinct, List<AliasedNode> select) {
         return select(statement.getFrom(), statement.getHint(), isDistinct, select, statement.getWhere(), statement.getGroupBy(),
                 statement.getHaving(), statement.getOrderBy(), statement.getLimit(), statement.getBindCount(), statement.isAggregate(),
-                statement.hasSequence(), statement.getSelects());
+                statement.hasSequence(), statement.getSelects(), statement.getUdfParseNodes());
     }
 
     public SelectStatement select(SelectStatement statement, boolean isDistinct, List<AliasedNode> select, ParseNode where) {
         return select(statement.getFrom(), statement.getHint(), isDistinct, select, where, statement.getGroupBy(),
                 statement.getHaving(), statement.getOrderBy(), statement.getLimit(), statement.getBindCount(), statement.isAggregate(),
-                statement.hasSequence(), statement.getSelects());
+                statement.hasSequence(), statement.getSelects(), statement.getUdfParseNodes());
     }
 
     public SelectStatement select(SelectStatement statement, boolean isDistinct, List<AliasedNode> select, ParseNode where, List<ParseNode> groupBy, boolean isAggregate) {
         return select(statement.getFrom(), statement.getHint(), isDistinct, select, where, groupBy,
                 statement.getHaving(), statement.getOrderBy(), statement.getLimit(), statement.getBindCount(), isAggregate,
-                statement.hasSequence(), statement.getSelects());
+                statement.hasSequence(), statement.getSelects(), statement.getUdfParseNodes());
     }
 
     public SelectStatement select(SelectStatement statement, List<OrderByNode> orderBy) {
         return select(statement.getFrom(), statement.getHint(), statement.isDistinct(), statement.getSelect(),
                 statement.getWhere(), statement.getGroupBy(), statement.getHaving(), orderBy, statement.getLimit(),
-                statement.getBindCount(), statement.isAggregate(), statement.hasSequence(), statement.getSelects());
+                statement.getBindCount(), statement.isAggregate(), statement.hasSequence(), statement.getSelects(), statement.getUdfParseNodes());
     }
 
     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(), statement.hasSequence(), statement.getSelects());
+                statement.getBindCount(), statement.isAggregate(), statement.hasSequence(), statement.getSelects(), statement.getUdfParseNodes());
     }
 
     public SelectStatement select(SelectStatement statement, HintNode hint, ParseNode where) {
         return select(statement.getFrom(), hint, statement.isDistinct(), statement.getSelect(), where, statement.getGroupBy(),
                 statement.getHaving(), statement.getOrderBy(), statement.getLimit(), statement.getBindCount(), statement.isAggregate(),
-                statement.hasSequence(), statement.getSelects());
+                statement.hasSequence(), statement.getSelects(), statement.getUdfParseNodes());
     }
 
     public SelectStatement select(SelectStatement statement, List<OrderByNode> orderBy, LimitNode limit, int bindCount, boolean isAggregate) {
         return select(statement.getFrom(), statement.getHint(), statement.isDistinct(), statement.getSelect(),
             statement.getWhere(), statement.getGroupBy(), statement.getHaving(), orderBy, limit,
-            bindCount, isAggregate || statement.isAggregate(), statement.hasSequence(), statement.getSelects());
+            bindCount, isAggregate || statement.isAggregate(), statement.hasSequence(), statement.getSelects(), statement.getUdfParseNodes());
 
     }
 
     public SelectStatement select(SelectStatement statement, LimitNode limit) {
         return select(statement.getFrom(), statement.getHint(), statement.isDistinct(), statement.getSelect(),
             statement.getWhere(), statement.getGroupBy(), statement.getHaving(), statement.getOrderBy(), limit,
-            statement.getBindCount(), statement.isAggregate(), statement.hasSequence(), statement.getSelects());
+            statement.getBindCount(), statement.isAggregate(), statement.hasSequence(), statement.getSelects(), statement.getUdfParseNodes());
     }
 
     public SelectStatement select(SelectStatement statement, List<OrderByNode> orderBy, LimitNode limit) {
         return select(statement.getFrom(), statement.getHint(), statement.isDistinct(), statement.getSelect(),
             statement.getWhere(), statement.getGroupBy(), statement.getHaving(), orderBy, limit,
-            statement.getBindCount(), statement.isAggregate(), statement.hasSequence(), statement.getSelects());
+            statement.getBindCount(), statement.isAggregate(), statement.hasSequence(), statement.getSelects(), statement.getUdfParseNodes());
     }
 
     public SelectStatement select(List<SelectStatement> statements, List<OrderByNode> orderBy, LimitNode limit, int bindCount, boolean isAggregate) {
@@ -762,8 +774,10 @@ public class ParseNodeFactory {
         // it will be done later at compile stage. Empty or different aliases 
         // are ignored, since they cannot be referred by outer queries.
         List<String> aliases = Lists.<String> newArrayList();
+        Map<String, UDFParseNode> udfParseNodes = new HashMap<String, UDFParseNode>(1);
         for (int i = 0; i < statements.size() && aliases.isEmpty(); i++) {
             SelectStatement subselect = statements.get(i);
+            udfParseNodes.putAll(subselect.getUdfParseNodes());
             if (!subselect.hasWildcard()) {
                 for (AliasedNode aliasedNode : subselect.getSelect()) {
                     String alias = aliasedNode.getAlias();
@@ -786,7 +800,7 @@ public class ParseNodeFactory {
         }
         
         return select(null, HintNode.EMPTY_HINT_NODE, false, aliasedNodes, 
-                null, null, null, orderBy, limit, bindCount, false, false, statements);
+                null, null, null, orderBy, limit, bindCount, false, false, statements, udfParseNodes);
     }
 
     public SubqueryParseNode subquery(SelectStatement select, boolean expectSingleRow) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/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
index 4ce893d..e48967b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeRewriter.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeRewriter.java
@@ -151,7 +151,7 @@ public class ParseNodeRewriter extends TraverseAllParseNodeVisitor<ParseNode> {
         return NODE_FACTORY.select(normFrom, statement.getHint(), statement.isDistinct(),
                 normSelectNodes, normWhere, normGroupByNodes, normHaving, normOrderByNodes,
                 statement.getLimit(), statement.getBindCount(), statement.isAggregate(), statement.hasSequence(),
-                statement.getSelects());
+                statement.getSelects(), statement.getUdfParseNodes());
     }
 
     private Map<String, ParseNode> getAliasMap() {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/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
index 44b24af..362e98d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/SelectStatement.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/SelectStatement.java
@@ -20,7 +20,9 @@ package org.apache.phoenix.parse;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.phoenix.compile.ColumnResolver;
 import org.apache.phoenix.expression.function.CountAggregateFunction;
@@ -42,7 +44,7 @@ public class SelectStatement implements FilterableStatement {
                     Collections.<AliasedNode>singletonList(new AliasedNode(null, LiteralParseNode.ONE)),
                     null, Collections.<ParseNode>emptyList(),
                     null, Collections.<OrderByNode>emptyList(),
-                    null, 0, false, false, Collections.<SelectStatement>emptyList());
+                    null, 0, false, false, Collections.<SelectStatement>emptyList(), new HashMap<String, UDFParseNode>(1));
     public static final SelectStatement COUNT_ONE =
             new SelectStatement(
                     null, null, false,
@@ -54,14 +56,14 @@ public class SelectStatement implements FilterableStatement {
                                 new BuiltInFunctionInfo(CountAggregateFunction.class, CountAggregateFunction.class.getAnnotation(BuiltInFunction.class))))),
                     null, Collections.<ParseNode>emptyList(), 
                     null, Collections.<OrderByNode>emptyList(), 
-                    null, 0, true, false, Collections.<SelectStatement>emptyList());
+                    null, 0, true, false, Collections.<SelectStatement>emptyList(), new HashMap<String, UDFParseNode>(1));
     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(), select.hasSequence(), select.getSelects());
+                select.getOrderBy(), select.getLimit(), select.getBindCount(), select.isAggregate(), select.hasSequence(), select.getSelects(), select.getUdfParseNodes());
     }
     
     public SelectStatement combine(ParseNode where) {
@@ -73,13 +75,13 @@ public class SelectStatement implements FilterableStatement {
         }
         return new SelectStatement(this.getFrom(), this.getHint(), this.isDistinct(), 
                 this.getSelect(), where, this.getGroupBy(), this.getHaving(), 
-                this.getOrderBy(), this.getLimit(), this.getBindCount(), this.isAggregate(), this.hasSequence(), this.selects);
+                this.getOrderBy(), this.getLimit(), this.getBindCount(), this.isAggregate(), this.hasSequence(), this.selects, this.udfParseNodes);
     }
     
     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(), select.hasSequence(), select.getSelects());
+                select.getOrderBy(), select.getLimit(), select.getBindCount(), select.isAggregate(), select.hasSequence(), select.getSelects(), select.getUdfParseNodes());
     }
     
     // Copy constructor for sub select statements in a union
@@ -87,7 +89,7 @@ public class SelectStatement implements FilterableStatement {
             List<OrderByNode> orderBy, LimitNode limit, boolean isAggregate) {
         return new SelectStatement(select.getFrom(), select.getHint(), select.isDistinct(), 
                 select.getSelect(), select.getWhere(), select.getGroupBy(), select.getHaving(), 
-                orderBy, limit, select.getBindCount(), isAggregate, select.hasSequence(), select.getSelects());
+                orderBy, limit, select.getBindCount(), isAggregate, select.hasSequence(), select.getSelects(), select.getUdfParseNodes());
     }
 
     private final TableNode fromTable;
@@ -104,6 +106,7 @@ public class SelectStatement implements FilterableStatement {
     private final boolean hasSequence;
     private final boolean hasWildcard;
     private final List<SelectStatement> selects = new ArrayList<SelectStatement>();
+    private final Map<String, UDFParseNode> udfParseNodes;
     
     @Override
     public final String toString() {
@@ -216,7 +219,7 @@ public class SelectStatement implements FilterableStatement {
     
     protected SelectStatement(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, boolean hasSequence, List<SelectStatement> selects) {
+            int bindCount, boolean isAggregate, boolean hasSequence, List<SelectStatement> selects, Map<String, UDFParseNode> udfParseNodes) {
         this.fromTable = from;
         this.hint = hint == null ? HintNode.EMPTY_HINT_NODE : hint;
         this.isDistinct = isDistinct;
@@ -241,6 +244,7 @@ public class SelectStatement implements FilterableStatement {
         if (!selects.isEmpty()) {
             this.selects.addAll(selects);
         }
+        this.udfParseNodes = udfParseNodes;
     }
     
     @Override
@@ -333,4 +337,8 @@ public class SelectStatement implements FilterableStatement {
     public boolean hasWildcard() {
         return hasWildcard;
     }
+
+    public Map<String, UDFParseNode> getUdfParseNodes() {
+        return udfParseNodes;
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/phoenix-core/src/main/java/org/apache/phoenix/parse/UDFParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/UDFParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/UDFParseNode.java
new file mode 100644
index 0000000..c0b972f
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/UDFParseNode.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 UDFParseNode extends FunctionParseNode {
+
+    public UDFParseNode(String name, List<ParseNode> children, BuiltInFunctionInfo info) {
+        super(name, children, info);
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/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
index fc299d1..48698bd 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/UpsertStatement.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/UpsertStatement.java
@@ -19,15 +19,18 @@ package org.apache.phoenix.parse;
 
 import java.util.Collections;
 import java.util.List;
+import java.util.Map;
 
-public class UpsertStatement extends DMLStatement { 
+public class UpsertStatement extends DMLStatement {
     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);
+    public UpsertStatement(NamedTableNode table, HintNode hint, List<ColumnName> columns,
+            List<ParseNode> values, SelectStatement select, int bindCount,
+            Map<String, UDFParseNode> udfParseNodes) {
+        super(table, bindCount, udfParseNodes);
         this.columns = columns == null ? Collections.<ColumnName>emptyList() : columns;
         this.values = values;
         this.select = select;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/phoenix-core/src/main/java/org/apache/phoenix/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/protobuf/ProtobufUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/protobuf/ProtobufUtil.java
index c12c64d..f4a60bc 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/protobuf/ProtobufUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/protobuf/ProtobufUtil.java
@@ -71,6 +71,16 @@ public class ProtobufUtil {
         return getMutations(request.getTableMetadataMutationsList());
     }
 
+    public static List<Mutation> getMutations(MetaDataProtos.DropFunctionRequest request)
+            throws IOException {
+        return getMutations(request.getTableMetadataMutationsList());
+    }
+
+    public static List<Mutation> getMutations(MetaDataProtos.CreateFunctionRequest request)
+            throws IOException {
+        return getMutations(request.getTableMetadataMutationsList());
+    }
+
     public static List<Mutation> getMutations(MetaDataProtos.DropTableRequest request)
             throws IOException {
         return getMutations(request.getTableMetadataMutationsList());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/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
index 09705c6..dc51b10 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java
@@ -35,6 +35,7 @@ import org.apache.phoenix.coprocessor.MetaDataProtocol.MetaDataMutationResult;
 import org.apache.phoenix.execute.MutationState;
 import org.apache.phoenix.hbase.index.util.KeyValueBuilder;
 import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.parse.PFunction;
 import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableType;
@@ -70,8 +71,10 @@ public interface ConnectionQueryServices extends QueryServices, MetaDataMutated
     public PhoenixConnection connect(String url, Properties info) throws SQLException;
 
     public MetaDataMutationResult getTable(PName tenantId, byte[] schemaName, byte[] tableName, long tableTimestamp, long clientTimetamp) throws SQLException;
+    public MetaDataMutationResult getFunctions(PName tenantId, List<Pair<byte[], Long>> functionNameAndTimeStampPairs, long clientTimestamp) 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, boolean cascade) throws SQLException;
+    public MetaDataMutationResult dropFunction(List<Mutation> tableMetadata, boolean ifExists) throws SQLException;
     public MetaDataMutationResult addColumn(List<Mutation> tableMetaData, PTable table, Map<String, List<Pair<String,Object>>> properties, Set<String> colFamiliesForPColumnsToBeAdded) throws SQLException;
     public MetaDataMutationResult dropColumn(List<Mutation> tableMetadata, PTableType tableType) throws SQLException;
     public MetaDataMutationResult updateIndexState(List<Mutation> tableMetadata, String parentTableName) throws SQLException;
@@ -93,6 +96,7 @@ public interface ConnectionQueryServices extends QueryServices, MetaDataMutated
     long currentSequenceValue(SequenceKey sequenceKey, long timestamp) throws SQLException;
     void returnSequences(List<SequenceKey> sequenceKeys, long timestamp, SQLException[] exceptions) throws SQLException;
 
+    MetaDataMutationResult createFunction(List<Mutation> functionData, PFunction function, boolean temporary) throws SQLException;
     void addConnection(PhoenixConnection connection) throws SQLException;
     void removeConnection(PhoenixConnection connection) throws SQLException;
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index 49c946a..30b43d5 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -87,9 +87,12 @@ import org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheRequest
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheResponse;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearTableFromCacheRequest;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearTableFromCacheResponse;
+import org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateFunctionRequest;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.DropColumnRequest;
+import org.apache.phoenix.coprocessor.generated.MetaDataProtos.DropFunctionRequest;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.DropTableRequest;
+import org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetFunctionsRequest;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetTableRequest;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetVersionRequest;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetVersionResponse;
@@ -110,9 +113,11 @@ 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.parse.PFunction;
 import org.apache.phoenix.protobuf.ProtobufUtil;
 import org.apache.phoenix.schema.ColumnFamilyNotFoundException;
 import org.apache.phoenix.schema.EmptySequenceCacheException;
+import org.apache.phoenix.schema.FunctionNotFoundException;
 import org.apache.phoenix.schema.MetaDataSplitPolicy;
 import org.apache.phoenix.schema.NewerTableAlreadyExistsException;
 import org.apache.phoenix.schema.PColumn;
@@ -220,6 +225,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 QueryServicesOptions.DEFAULT_MAX_CLIENT_METADATA_CACHE_SIZE);
         return new PMetaDataImpl(INITIAL_META_DATA_TABLE_CAPACITY, maxSizeBytes);
     }
+    
     /**
      * Construct a ConnectionQueryServicesImpl that represents a connection to an HBase
      * cluster.
@@ -562,6 +568,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         if (metadata == null) {
             throwConnectionClosedException();
         }
+
         return new PhoenixConnection(this, url, info, metadata);
     }
 
@@ -691,12 +698,14 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
 
             // 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)) {
+            if (SchemaUtil.isMetaTable(tableName) || SchemaUtil.isFunctionTable(tableName)) {
                 if (!descriptor.hasCoprocessor(MetaDataEndpointImpl.class.getName())) {
                     descriptor.addCoprocessor(MetaDataEndpointImpl.class.getName(), null, priority, null);
                 }
-                if (!descriptor.hasCoprocessor(MetaDataRegionObserver.class.getName())) {
-                    descriptor.addCoprocessor(MetaDataRegionObserver.class.getName(), null, priority + 1, null);
+                if(SchemaUtil.isMetaTable(tableName) ) {
+                    if (!descriptor.hasCoprocessor(MetaDataRegionObserver.class.getName())) {
+                        descriptor.addCoprocessor(MetaDataRegionObserver.class.getName(), null, priority + 1, null);
+                    }
                 }
             } else if (SchemaUtil.isSequenceTable(tableName)) {
                 if (!descriptor.hasCoprocessor(SequenceRegionObserver.class.getName())) {
@@ -992,23 +1001,33 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
      */
     private MetaDataMutationResult metaDataCoprocessorExec(byte[] tableKey,
             Batch.Call<MetaDataService, MetaDataResponse> callable) throws SQLException {
+        return metaDataCoprocessorExec(tableKey, callable, PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
+    }
+        /**
+         * 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<MetaDataService, MetaDataResponse> callable, byte[] tableName) throws SQLException {
+
         try {
             boolean retried = false;
             while (true) {
                 if (retried) {
                     connection.relocateRegion(
-                        TableName.valueOf(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES),
+                        TableName.valueOf(tableName),
                         tableKey);
                 }
 
-                HTableInterface ht = this.getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
+                HTableInterface ht = this.getTable(tableName);
                 try {
                     final Map<byte[], MetaDataResponse> results =
                             ht.coprocessorService(MetaDataService.class, tableKey, tableKey, callable);
 
                     assert(results.size() == 1);
                     MetaDataResponse result = results.values().iterator().next();
-                    if (result.getReturnCode() == MetaDataProtos.MutationCode.TABLE_NOT_IN_REGION) {
+                    if (result.getReturnCode() == MetaDataProtos.MutationCode.TABLE_NOT_IN_REGION
+                            || result.getReturnCode() == MetaDataProtos.MutationCode.FUNCTION_NOT_IN_REGION) {
                         if (retried) return MetaDataMutationResult.constructFromProto(result);
                         retried = true;
                         continue;
@@ -1331,6 +1350,37 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
           return result;
     }
 
+    @Override
+    public MetaDataMutationResult dropFunction(final List<Mutation> functionData, final boolean ifExists) throws SQLException {
+        byte[][] rowKeyMetadata = new byte[2][];
+        byte[] key = functionData.get(0).getRow();
+        SchemaUtil.getVarChars(key, rowKeyMetadata);
+        byte[] tenantIdBytes = rowKeyMetadata[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
+        byte[] functionBytes = rowKeyMetadata[PhoenixDatabaseMetaData.FUNTION_NAME_INDEX];
+        byte[] functionKey = SchemaUtil.getFunctionKey(tenantIdBytes, functionBytes);
+        
+        final MetaDataMutationResult result =  metaDataCoprocessorExec(functionKey,
+                new Batch.Call<MetaDataService, MetaDataResponse>() {
+                    @Override
+                    public MetaDataResponse call(MetaDataService instance) throws IOException {
+                        ServerRpcController controller = new ServerRpcController();
+                        BlockingRpcCallback<MetaDataResponse> rpcCallback =
+                                new BlockingRpcCallback<MetaDataResponse>();
+                        DropFunctionRequest.Builder builder = DropFunctionRequest.newBuilder();
+                        for (Mutation m : functionData) {
+                            MutationProto mp = ProtobufUtil.toProto(m);
+                            builder.addTableMetadataMutations(mp.toByteString());
+                        }
+                        builder.setIfExists(ifExists);
+                        instance.dropFunction(controller, builder.build(), rpcCallback);
+                        if(controller.getFailedOn() != null) {
+                            throw controller.getFailedOn();
+                        }
+                        return rpcCallback.get();
+                    }
+                }, PhoenixDatabaseMetaData.SYSTEM_FUNCTION_NAME_BYTES);
+        return result;
+    }
     private void invalidateTables(final List<byte[]> tableNamesToDelete) {
         if (tableNamesToDelete != null) {
             for ( byte[] tableName : tableNamesToDelete ) {
@@ -1944,6 +1994,13 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                         PhoenixDatabaseMetaData.GUIDE_POSTS_ROW_COUNT + " "
                                                 + PLong.INSTANCE.getSqlTypeName());
                             }
+                            try {
+                                metaConnection.createStatement().executeUpdate(
+                                    QueryConstants.CREATE_FUNCTION_METADATA);
+                            } catch (NewerTableAlreadyExistsException e) {
+                            } catch (TableAlreadyExistsException e) {
+                            }
+
                         } catch (Exception e) {
                             if (e instanceof SQLException) {
                                 initializationException = (SQLException)e;
@@ -2540,4 +2597,96 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     public int getSequenceSaltBuckets() {
         return nSequenceSaltBuckets;
     }
+
+    @Override
+    public PMetaData addFunction(PFunction function) throws SQLException {
+        synchronized (latestMetaDataLock) {
+            try {
+                throwConnectionClosedIfNullMetaData();
+                // 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
+                PFunction existingFunction = latestMetaData.getFunction(new PTableKey(function.getTenantId(), function.getFunctionName()));
+                if (existingFunction.getTimeStamp() >= function.getTimeStamp()) {
+                    return latestMetaData;
+                }
+            } catch (FunctionNotFoundException e) {}
+            latestMetaData = latestMetaData.addFunction(function);
+            latestMetaDataLock.notifyAll();
+            return latestMetaData;
+        }
+    }
+
+    @Override
+    public PMetaData removeFunction(PName tenantId, String function, long functionTimeStamp)
+            throws SQLException {
+        synchronized (latestMetaDataLock) {
+            throwConnectionClosedIfNullMetaData();
+            latestMetaData = latestMetaData.removeFunction(tenantId, function, functionTimeStamp);
+            latestMetaDataLock.notifyAll();
+            return latestMetaData;
+        }
+    }
+
+    @Override
+    public MetaDataMutationResult getFunctions(PName tenantId, final List<Pair<byte[], Long>> functions,
+            final long clientTimestamp) throws SQLException {
+        final byte[] tenantIdBytes = tenantId == null ? ByteUtil.EMPTY_BYTE_ARRAY : tenantId.getBytes();
+        return metaDataCoprocessorExec(tenantIdBytes,
+            new Batch.Call<MetaDataService, MetaDataResponse>() {
+                @Override
+                public MetaDataResponse call(MetaDataService instance) throws IOException {
+                    ServerRpcController controller = new ServerRpcController();
+                    BlockingRpcCallback<MetaDataResponse> rpcCallback =
+                            new BlockingRpcCallback<MetaDataResponse>();
+                    GetFunctionsRequest.Builder builder = GetFunctionsRequest.newBuilder();
+                    builder.setTenantId(HBaseZeroCopyByteString.wrap(tenantIdBytes));
+                    for(Pair<byte[], Long> function: functions) {
+                        builder.addFunctionNames(HBaseZeroCopyByteString.wrap(function.getFirst()));
+                        builder.addFunctionTimestamps(function.getSecond().longValue());
+                    }
+                    builder.setClientTimestamp(clientTimestamp);
+
+                   instance.getFunctions(controller, builder.build(), rpcCallback);
+                   if(controller.getFailedOn() != null) {
+                       throw controller.getFailedOn();
+                   }
+                   return rpcCallback.get();
+                }
+            }, PhoenixDatabaseMetaData.SYSTEM_FUNCTION_NAME_BYTES);
+
+    }
+
+    // TODO the mutations should be added to System functions table.
+    @Override
+    public MetaDataMutationResult createFunction(final List<Mutation> functionData,
+            final PFunction function, final boolean temporary) throws SQLException {
+        byte[][] rowKeyMetadata = new byte[2][];
+        Mutation m = MetaDataUtil.getPutOnlyTableHeaderRow(functionData);
+        byte[] key = m.getRow();
+        SchemaUtil.getVarChars(key, rowKeyMetadata);
+        byte[] tenantIdBytes = rowKeyMetadata[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
+        byte[] functionBytes = rowKeyMetadata[PhoenixDatabaseMetaData.FUNTION_NAME_INDEX];
+        byte[] functionKey = SchemaUtil.getFunctionKey(tenantIdBytes, functionBytes);
+        MetaDataMutationResult result = metaDataCoprocessorExec(functionKey,
+            new Batch.Call<MetaDataService, MetaDataResponse>() {
+                    @Override
+                    public MetaDataResponse call(MetaDataService instance) throws IOException {
+                        ServerRpcController controller = new ServerRpcController();
+                        BlockingRpcCallback<MetaDataResponse> rpcCallback =
+                                new BlockingRpcCallback<MetaDataResponse>();
+                        CreateFunctionRequest.Builder builder = CreateFunctionRequest.newBuilder();
+                        for (Mutation m : functionData) {
+                            MutationProto mp = ProtobufUtil.toProto(m);
+                            builder.addTableMetadataMutations(mp.toByteString());
+                        }
+                        builder.setTemporary(temporary);
+                        instance.createFunction(controller, builder.build(), rpcCallback);
+                        if(controller.getFailedOn() != null) {
+                            throw controller.getFailedOn();
+                        }
+                        return rpcCallback.get();
+                    }
+        }, PhoenixDatabaseMetaData.SYSTEM_FUNCTION_NAME_BYTES);
+        return result;
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
index 742c38e..4d582be 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
@@ -20,6 +20,7 @@ package org.apache.phoenix.query;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_STATE_BYTES;
 
 import java.sql.SQLException;
+import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
@@ -51,6 +52,8 @@ import org.apache.phoenix.hbase.index.util.KeyValueBuilder;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver.ConnectionInfo;
+import org.apache.phoenix.parse.PFunction;
+import org.apache.phoenix.schema.FunctionNotFoundException;
 import org.apache.phoenix.schema.NewerTableAlreadyExistsException;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PIndexState;
@@ -104,6 +107,7 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
         super(queryServices);
         userName = connInfo.getPrincipal();
         metaData = newEmptyMetaData();
+        
         // Use KeyValueBuilder that builds real KeyValues, as our test utils require this
         this.kvBuilder = GenericKeyValueBuilder.INSTANCE;
     }
@@ -280,6 +284,11 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
                     // A TableAlreadyExistsException is not thrown, since the table only exists *after* this
                     // fixed timestamp.
                 }
+                
+                try {
+                   metaConnection.createStatement().executeUpdate(QueryConstants.CREATE_FUNCTION_METADATA);
+                } catch (NewerTableAlreadyExistsException ignore) {
+                }
             } catch (SQLException e) {
                 sqlE = e;
             } finally {
@@ -479,5 +488,46 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
         return getProps().getInt(QueryServices.SEQUENCE_SALT_BUCKETS_ATTRIB,
                 QueryServicesOptions.DEFAULT_SEQUENCE_TABLE_SALT_BUCKETS);
     }
- 
+
+    @Override
+    public MetaDataMutationResult createFunction(List<Mutation> functionData, PFunction function, boolean temporary)
+            throws SQLException {
+        return new MetaDataMutationResult(MutationCode.FUNCTION_NOT_FOUND, 0l, null);
+    }
+
+    @Override
+    public PMetaData addFunction(PFunction function) throws SQLException {
+        return metaData = this.metaData.addFunction(function);
+    }
+
+    @Override
+    public PMetaData removeFunction(PName tenantId, String function, long functionTimeStamp)
+            throws SQLException {
+        return metaData = this.metaData.removeFunction(tenantId, function, functionTimeStamp);
+    }
+
+    @Override
+    public MetaDataMutationResult getFunctions(PName tenantId,
+            List<Pair<byte[], Long>> functionNameAndTimeStampPairs, long clientTimestamp)
+            throws SQLException {
+        List<PFunction> functions = new ArrayList<PFunction>(functionNameAndTimeStampPairs.size());
+        for(Pair<byte[], Long> functionInfo: functionNameAndTimeStampPairs) {
+            try {
+                PFunction function2 = metaData.getFunction(new PTableKey(tenantId, Bytes.toString(functionInfo.getFirst())));
+                functions.add(function2);
+            } catch (FunctionNotFoundException e) {
+                return new MetaDataMutationResult(MutationCode.FUNCTION_NOT_FOUND, 0, null);
+            }
+        }
+        if(functions.isEmpty()) {
+            return null;
+        }
+        return new MetaDataMutationResult(MutationCode.FUNCTION_ALREADY_EXISTS, 0, functions, true);
+    }
+
+    @Override
+    public MetaDataMutationResult dropFunction(List<Mutation> tableMetadata, boolean ifExists)
+            throws SQLException {
+        return new MetaDataMutationResult(MutationCode.FUNCTION_ALREADY_EXISTS, 0, null);
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/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
index e2c9544..2a98cd5 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
@@ -35,6 +35,7 @@ import org.apache.phoenix.coprocessor.MetaDataProtocol.MetaDataMutationResult;
 import org.apache.phoenix.execute.MutationState;
 import org.apache.phoenix.hbase.index.util.KeyValueBuilder;
 import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.parse.PFunction;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PMetaData;
 import org.apache.phoenix.schema.PName;
@@ -250,4 +251,34 @@ public class DelegateConnectionQueryServices extends DelegateQueryServices imple
     public int getSequenceSaltBuckets() {
         return getDelegate().getSequenceSaltBuckets();
     }
+
+    @Override
+    public MetaDataMutationResult createFunction(List<Mutation> functionData, PFunction function, boolean temporary)
+            throws SQLException {
+        return getDelegate().createFunction(functionData, function, temporary);
+    }
+
+    @Override
+    public PMetaData addFunction(PFunction function) throws SQLException {
+        return getDelegate().addFunction(function);
+    }
+
+    @Override
+    public PMetaData removeFunction(PName tenantId, String function, long functionTimeStamp)
+            throws SQLException {
+        return getDelegate().removeFunction(tenantId, function, functionTimeStamp);
+    }
+
+    @Override
+    public MetaDataMutationResult getFunctions(PName tenantId,
+            List<Pair<byte[], Long>> functionNameAndTimeStampPairs, long clientTimestamp)
+            throws SQLException {
+        return getDelegate().getFunctions(tenantId, functionNameAndTimeStampPairs, clientTimestamp);
+    }
+
+    @Override
+    public MetaDataMutationResult dropFunction(List<Mutation> tableMetadata, boolean ifExists)
+            throws SQLException {
+        return getDelegate().dropFunction(tableMetadata, ifExists);
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/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
index ae37ac6..76e7593 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/MetaDataMutated.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/MetaDataMutated.java
@@ -20,6 +20,7 @@ package org.apache.phoenix.query;
 import java.sql.SQLException;
 import java.util.List;
 
+import org.apache.phoenix.parse.PFunction;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PMetaData;
 import org.apache.phoenix.schema.PName;
@@ -38,4 +39,6 @@ public interface MetaDataMutated {
     PMetaData removeTable(PName tenantId, String tableName, String parentTableName, long tableTimeStamp) throws SQLException;
     PMetaData addColumn(PName tenantId, String tableName, List<PColumn> columns, long tableTimeStamp, long tableSeqNum, boolean isImmutableRows, boolean isWalDisabled, boolean isMultitenant, boolean storeNulls) throws SQLException;
     PMetaData removeColumn(PName tenantId, String tableName, List<PColumn> columnsToRemove, long tableTimeStamp, long tableSeqNum) throws SQLException;
+    PMetaData addFunction(PFunction function) throws SQLException;
+    PMetaData removeFunction(PName tenantId, String function, long functionTimeStamp) throws SQLException;
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
index 6470b72..73d1123 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
@@ -86,6 +86,17 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_TYPE;
 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_SEQUENCE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CLASS_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_FUNCTION_TABLE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ARG_POSITION;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.FUNCTION_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.JAR_PATH;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.NUM_ARGS;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_ARRAY;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_CONSTANT;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DEFAULT_VALUE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TYPE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.RETURN_TYPE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_CONSTANT;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_INDEX_ID;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_STATEMENT;
@@ -276,4 +287,29 @@ public interface QueryConstants {
             " CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (" + TENANT_ID + "," + SEQUENCE_SCHEMA + "," + SEQUENCE_NAME + "))\n" +
             HConstants.VERSIONS + "=" + MetaDataProtocol.DEFAULT_MAX_META_DATA_VERSIONS + ",\n" +
             HColumnDescriptor.KEEP_DELETED_CELLS + "="  + MetaDataProtocol.DEFAULT_META_DATA_KEEP_DELETED_CELLS + "\n";
+
+    public static final String CREATE_FUNCTION_METADATA =
+            "CREATE TABLE " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_FUNCTION_TABLE + "\"(\n" +
+             // Pk columns
+            TENANT_ID + " VARCHAR NULL," +
+            FUNCTION_NAME + " VARCHAR NOT NULL, \n" +
+            NUM_ARGS + " INTEGER, \n" +
+            // Function metadata (will be null for argument row)
+            CLASS_NAME +  " VARCHAR, \n" +
+            JAR_PATH + "  VARCHAR, \n" +
+            RETURN_TYPE + " VARCHAR, \n" +
+            // Argument metadata (will be null for function row)
+            TYPE + " VARCHAR, \n" +
+            ARG_POSITION + " VARBINARY, \n" +
+            IS_ARRAY + " BOOLEAN, \n" +
+            IS_CONSTANT + " BOOLEAN, \n" +
+            DEFAULT_VALUE + " VARCHAR, \n" +
+            MIN_VALUE + " VARCHAR, \n" +
+            MAX_VALUE + " VARCHAR, \n" +
+            " CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (" + TENANT_ID + ", " + FUNCTION_NAME + ", " + TYPE + ", " + ARG_POSITION + "))\n" +
+            HConstants.VERSIONS + "=" + MetaDataProtocol.DEFAULT_MAX_META_DATA_VERSIONS + ",\n" +
+            HColumnDescriptor.KEEP_DELETED_CELLS + "="  + MetaDataProtocol.DEFAULT_META_DATA_KEEP_DELETED_CELLS + ",\n"+
+            // Install split policy to prevent a tenant's metadata from being split across regions.
+            HTableDescriptor.SPLIT_POLICY + "='" + MetaDataSplitPolicy.class.getName() + "'\n";
+
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
index 4b793d1..9183a70 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
@@ -103,6 +103,7 @@ public interface QueryServices extends SQLCloseable {
     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 DYNAMIC_JARS_DIR_KEY = "hbase.dynamic.jars.dir";
     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";
@@ -165,6 +166,7 @@ public interface QueryServices extends SQLCloseable {
     public static final String METADATA_HANDLER_COUNT_ATTRIB = "phoenix.rpc.metadata.handler.count";
     
     public static final String FORCE_ROW_KEY_ORDER_ATTRIB = "phoenix.query.force.rowkeyorder";
+    public static final String ALLOW_USER_DEFINED_FUNCTIONS_ATTRIB = "phoenix.functions.allowUserDefinedFunctions";
     
 
     /**

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
index b98c9ee..972bf26 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
@@ -18,6 +18,7 @@
 package org.apache.phoenix.query;
 
 import static org.apache.phoenix.query.QueryServices.ALLOW_ONLINE_TABLE_SCHEMA_UPDATE;
+import static org.apache.phoenix.query.QueryServices.ALLOW_USER_DEFINED_FUNCTIONS_ATTRIB;
 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;
@@ -195,6 +196,7 @@ public class QueryServicesOptions {
 
     public static final boolean DEFAULT_USE_BYTE_BASED_REGEX = false;
     public static final boolean DEFAULT_FORCE_ROW_KEY_ORDER = false;
+    public static final boolean DEFAULT_ALLOW_USER_DEFINED_FUNCTIONS = false;
 
     private final Configuration config;
 
@@ -250,7 +252,7 @@ public class QueryServicesOptions {
             .setIfUnset(METRICS_ENABLED, DEFAULT_IS_METRICS_ENABLED)
             .setIfUnset(RpcControllerFactory.CUSTOM_CONTROLLER_CONF_KEY, DEFAULT_CLIENT_RPC_CONTROLLER_FACTORY)
             .setIfUnset(USE_BYTE_BASED_REGEX_ATTRIB, DEFAULT_USE_BYTE_BASED_REGEX)
-            .setIfUnset(FORCE_ROW_KEY_ORDER_ATTRIB, DEFAULT_FORCE_ROW_KEY_ORDER)
+            .setIfUnset(FORCE_ROW_KEY_ORDER_ATTRIB, DEFAULT_FORCE_ROW_KEY_ORDER);
             ;
         // 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

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/phoenix-core/src/main/java/org/apache/phoenix/schema/FunctionAlreadyExistsException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/FunctionAlreadyExistsException.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/FunctionAlreadyExistsException.java
new file mode 100644
index 0000000..91b9d07
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/FunctionAlreadyExistsException.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+import org.apache.phoenix.parse.PFunction;
+
+public class FunctionAlreadyExistsException extends SQLException {
+    private static final long serialVersionUID = 1L;
+    private static SQLExceptionCode code = SQLExceptionCode.FUNCTION_ALREADY_EXIST;
+    private final PFunction function;
+    private final String functionName;
+
+    public FunctionAlreadyExistsException(String functionName) {
+        this(functionName, null, null);
+    }
+
+    public FunctionAlreadyExistsException(String functionName, String msg) {
+        this(functionName, msg, null);
+    }
+
+    public FunctionAlreadyExistsException(String functionName, PFunction function) {
+        this(functionName, null, function);
+    }
+
+    public FunctionAlreadyExistsException(String functionName, String msg, PFunction function) {
+        super(new SQLExceptionInfo.Builder(code).setFunctionName(functionName).setMessage(msg).build().toString(),
+                code.getSQLState(), code.getErrorCode());
+        this.functionName = functionName;
+        this.function = function;
+    }
+
+    public String getFunctionName() {
+        return functionName;
+    }
+    
+    public PFunction getFunction() {
+        return function;
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/phoenix-core/src/main/java/org/apache/phoenix/schema/FunctionNotFoundException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/FunctionNotFoundException.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/FunctionNotFoundException.java
new file mode 100644
index 0000000..73e23be
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/FunctionNotFoundException.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+
+public class FunctionNotFoundException extends MetaDataEntityNotFoundException {
+    private static final long serialVersionUID = 1L;
+    private static SQLExceptionCode code = SQLExceptionCode.FUNCTION_UNDEFINED;
+    private final String functionName;
+    private final long timestamp;
+
+    public FunctionNotFoundException(FunctionNotFoundException e, long timestamp) {
+        this(e.functionName, timestamp);
+    }
+
+    public FunctionNotFoundException(String functionName) {
+        this(functionName, HConstants.LATEST_TIMESTAMP);
+    }
+    
+    public FunctionNotFoundException(String functionName, long timestamp) {
+        super(new SQLExceptionInfo.Builder(code).setFunctionName(functionName).build().toString(),
+                code.getSQLState(), code.getErrorCode(), null);
+        this.functionName = functionName;
+        this.timestamp = timestamp;
+    }
+
+    public String getFunctionName() {
+        return functionName;
+    }
+
+    public long getTimeStamp() {
+        return timestamp;
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
index 22208f1..fcdb651 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
@@ -32,15 +32,20 @@ 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.DEFAULT_VALUE;
 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_DISABLE_TIMESTAMP;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_STATE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_TYPE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_ARRAY;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_CONSTANT;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_VIEW_REFERENCED;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.KEY_SEQ;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.LAST_STATS_UPDATE_TIME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.LINK_TYPE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.MAX_VALUE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.MIN_VALUE;
 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;
@@ -62,10 +67,20 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_CONSTANT;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_INDEX_ID;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_STATEMENT;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_TYPE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ARG_POSITION;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.NUM_ARGS;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_FUNCTION_TABLE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.FUNCTION_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CLASS_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.JAR_PATH;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TYPE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.RETURN_TYPE;
 import static org.apache.phoenix.query.QueryServices.DROP_METADATA_ATTRIB;
 import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_DROP_METADATA;
 
 import java.io.IOException;
+import java.lang.reflect.Constructor;
+import java.sql.Array;
 import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.ParameterMetaData;
@@ -75,6 +90,7 @@ import java.sql.ResultSetMetaData;
 import java.sql.SQLException;
 import java.sql.SQLFeatureNotSupportedException;
 import java.sql.Types;
+import java.util.ArrayList;
 import java.util.BitSet;
 import java.util.Collection;
 import java.util.Collections;
@@ -99,6 +115,7 @@ import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.io.TimeRange;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.DynamicClassLoader;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.compile.ColumnResolver;
 import org.apache.phoenix.compile.ExplainPlan;
@@ -120,6 +137,8 @@ import org.apache.phoenix.execute.MutationState;
 import org.apache.phoenix.expression.Determinism;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.RowKeyColumnExpression;
+import org.apache.phoenix.expression.function.FunctionExpression;
+import org.apache.phoenix.expression.function.ScalarFunction;
 import org.apache.phoenix.hbase.index.covered.update.ColumnReference;
 import org.apache.phoenix.index.IndexMaintainer;
 import org.apache.phoenix.jdbc.PhoenixConnection;
@@ -130,21 +149,27 @@ 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.CreateFunctionStatement;
 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.DropFunctionStatement;
 import org.apache.phoenix.parse.DropIndexStatement;
 import org.apache.phoenix.parse.DropSequenceStatement;
 import org.apache.phoenix.parse.DropTableStatement;
+import org.apache.phoenix.parse.FunctionParseNode;
 import org.apache.phoenix.parse.IndexKeyConstraint;
 import org.apache.phoenix.parse.NamedTableNode;
+import org.apache.phoenix.parse.PFunction;
 import org.apache.phoenix.parse.ParseNode;
 import org.apache.phoenix.parse.ParseNodeFactory;
 import org.apache.phoenix.parse.PrimaryKeyConstraint;
 import org.apache.phoenix.parse.TableName;
 import org.apache.phoenix.parse.UpdateStatisticsStatement;
+import org.apache.phoenix.parse.PFunction.FunctionArgument;
 import org.apache.phoenix.query.ConnectionQueryServices.Feature;
+import org.apache.phoenix.query.HBaseFactoryProvider;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
@@ -280,6 +305,28 @@ public class MetaDataClient {
         COLUMN_FAMILY + "," +
         ORDINAL_POSITION +
         ") VALUES (?, ?, ?, ?, ?, ?)";
+    private static final String CREATE_FUNCTION =
+            "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_FUNCTION_TABLE + "\" ( " +
+            TENANT_ID +","+
+            FUNCTION_NAME + "," +
+            NUM_ARGS + "," +
+            CLASS_NAME + "," +
+            JAR_PATH + "," +
+            RETURN_TYPE +
+            ") VALUES (?, ?, ?, ?, ?, ?)";
+    private static final String INSERT_FUNCTION_ARGUMENT =
+            "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_FUNCTION_TABLE + "\" ( " +
+            TENANT_ID +","+
+            FUNCTION_NAME + "," +
+            TYPE + "," +
+            ARG_POSITION +","+
+            IS_ARRAY + "," +
+            IS_CONSTANT  + "," +
+            DEFAULT_VALUE + "," +
+            MIN_VALUE + "," +
+            MAX_VALUE +
+            ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?)";
+
 
     private final PhoenixConnection connection;
 
@@ -315,6 +362,24 @@ public class MetaDataClient {
         return updateCache(tenantId, schemaName, tableName, false);
     }
 
+    /**
+     * Update the cache with the latest as of the connection scn.
+     * @param functioNames
+     * @return the timestamp from the server, negative if the function was added to the cache and positive otherwise
+     * @throws SQLException
+     */
+    public MetaDataMutationResult updateCache(List<String> functionNames) throws SQLException {
+        return updateCache(functionNames, false);
+    }
+
+    private MetaDataMutationResult updateCache(List<String> functionNames, boolean alwaysHitServer) throws SQLException {
+        return updateCache(connection.getTenantId(), functionNames, alwaysHitServer);
+    }
+
+    public MetaDataMutationResult updateCache(PName tenantId, List<String> functionNames) throws SQLException {
+        return updateCache(tenantId, functionNames, false);
+    }
+
     private long getClientTimeStamp() {
         Long scn = connection.getSCN();
         long clientTimeStamp = scn == null ? HConstants.LATEST_TIMESTAMP : scn;
@@ -394,6 +459,77 @@ public class MetaDataClient {
 
         return result;
     }
+    
+    private MetaDataMutationResult updateCache(PName tenantId, List<String> functionNames,
+            boolean alwaysHitServer) throws SQLException { // TODO: pass byte[] herez
+        long clientTimeStamp = getClientTimeStamp();
+        List<PFunction> functions = new ArrayList<PFunction>(functionNames.size());
+        List<Long> functionTimeStamps = new ArrayList<Long>(functionNames.size());
+        Iterator<String> iterator = functionNames.iterator();
+        while (iterator.hasNext()) {
+            PFunction function = null;
+            try {
+                String functionName = iterator.next();
+                function =
+                        connection.getMetaDataCache().getFunction(
+                            new PTableKey(tenantId, functionName));
+                if (function != null && !alwaysHitServer
+                        && function.getTimeStamp() == clientTimeStamp - 1) {
+                    functions.add(function);
+                    iterator.remove();
+                    continue;
+                }
+                if (function != null && function.getTimeStamp() != clientTimeStamp - 1) {
+                    functionTimeStamps.add(function.getTimeStamp());
+                } else {
+                    functionTimeStamps.add(HConstants.LATEST_TIMESTAMP);
+                }
+            } catch (FunctionNotFoundException e) {
+                functionTimeStamps.add(HConstants.LATEST_TIMESTAMP);
+            }
+        }
+        // Don't bother with server call: we can't possibly find a newer function
+        if (functionNames.isEmpty()) {
+            return new MetaDataMutationResult(MutationCode.FUNCTION_ALREADY_EXISTS,QueryConstants.UNSET_TIMESTAMP,functions, true);
+        }
+
+        int maxTryCount = tenantId == null ? 1 : 2;
+        int tryCount = 0;
+        MetaDataMutationResult result;
+
+        do {
+            List<Pair<byte[], Long>> functionsToFecth = new ArrayList<Pair<byte[], Long>>(functionNames.size()); 
+            for(int i = 0; i< functionNames.size(); i++) {
+                functionsToFecth.add(new Pair<byte[], Long>(PVarchar.INSTANCE.toBytes(functionNames.get(i)), functionTimeStamps.get(i)));
+            }
+            result = connection.getQueryServices().getFunctions(tenantId, functionsToFecth, clientTimeStamp);
+
+            MutationCode code = result.getMutationCode();
+            // We found an updated table, so update our cache
+            if (result.getFunctions() != null && !result.getFunctions().isEmpty()) {
+                result.getFunctions().addAll(functions);
+                addFunctionToCache(result);
+                return result;
+            } else {
+                if (code == MutationCode.FUNCTION_ALREADY_EXISTS) {
+                    result.getFunctions().addAll(functions);
+                    addFunctionToCache(result);
+                    return result;
+                }
+                if (code == MutationCode.FUNCTION_NOT_FOUND && tryCount + 1 == maxTryCount) {
+                    for (Pair<byte[], Long> f : functionsToFecth) {
+                        connection.removeFunction(tenantId, Bytes.toString(f.getFirst()),
+                            f.getSecond());
+                    }
+                    // TODO removeFunctions all together from cache when 
+                    throw new FunctionNotFoundException(functionNames.toString() + " not found");
+                }
+            }
+            tenantId = null; // Try again with global tenantId
+        } while (++tryCount < maxTryCount);
+
+        return result;
+    }
 
     /**
      * Fault in the physical table to the cache and add any indexes it has to the indexes
@@ -540,6 +676,20 @@ public class MetaDataClient {
         colUpsert.execute();
     }
 
+    private void addFunctionArgMutation(String functionName, FunctionArgument arg, PreparedStatement argUpsert, int position) throws SQLException {
+        argUpsert.setString(1, connection.getTenantId() == null ? null : connection.getTenantId().getString());
+        argUpsert.setString(2, functionName);
+        argUpsert.setString(3, arg.getArgumentType());
+        byte[] bytes = Bytes.toBytes((short)position);
+        argUpsert.setBytes(4, bytes);
+        argUpsert.setBoolean(5, arg.isArrayType());
+        argUpsert.setBoolean(6, arg.isConstant());
+        argUpsert.setString(7, arg.getDefaultValue() == null? null: (String)arg.getDefaultValue().getValue());
+        argUpsert.setString(8, arg.getMinValue() == null? null: (String)arg.getMinValue().getValue());
+        argUpsert.setString(9, arg.getMaxValue() == null? null: (String)arg.getMaxValue().getValue());
+        argUpsert.execute();
+    }
+
     private PColumn newColumn(int position, ColumnDef def, PrimaryKeyConstraint pkConstraint, String defaultColumnFamily, boolean addingToPK) throws SQLException {
         try {
             ColumnName columnDefName = def.getColumnDefName();
@@ -954,7 +1104,7 @@ public class MetaDataClient {
         }
         while (true) {
             try {
-                ColumnResolver resolver = FromCompiler.getResolver(statement, connection);
+                ColumnResolver resolver = FromCompiler.getResolver(statement, connection, statement.getUdfParseNodes());
                 tableRef = resolver.getTables().get(0);
                 PTable dataTable = tableRef.getTable();
                 boolean isTenantConnection = connection.getTenantId() != null;
@@ -1198,6 +1348,57 @@ public class MetaDataClient {
         return new MutationState(1, connection);
     }
 
+    public MutationState createFunction(CreateFunctionStatement stmt) throws SQLException {
+        boolean wasAutoCommit = connection.getAutoCommit();
+        connection.rollback();
+        try {
+            PFunction function = new PFunction(stmt.getFunctionInfo(), stmt.isTemporary());
+            connection.setAutoCommit(false);
+            String tenantIdStr = connection.getTenantId() == null ? null : connection.getTenantId().getString();
+            List<Mutation> functionData = Lists.newArrayListWithExpectedSize(function.getFunctionArguments().size() + 1);
+
+            List<FunctionArgument> args = function.getFunctionArguments();
+            PreparedStatement argUpsert = connection.prepareStatement(INSERT_FUNCTION_ARGUMENT);
+
+            for (int i = 0; i < args.size(); i++) {
+                FunctionArgument arg = args.get(i);
+                addFunctionArgMutation(function.getFunctionName(), arg, argUpsert, i);
+            }
+            functionData.addAll(connection.getMutationState().toMutations().next().getSecond());
+            connection.rollback();
+
+            PreparedStatement functionUpsert = connection.prepareStatement(CREATE_FUNCTION);
+            functionUpsert.setString(1, tenantIdStr);
+            functionUpsert.setString(2, function.getFunctionName());
+            functionUpsert.setInt(3, function.getFunctionArguments().size());
+            functionUpsert.setString(4, function.getClassName());
+            functionUpsert.setString(5, function.getJarPath());
+            functionUpsert.setString(6, function.getReturnType());
+            functionUpsert.execute();
+            functionData.addAll(connection.getMutationState().toMutations().next().getSecond());
+            connection.rollback();
+            MetaDataMutationResult result = connection.getQueryServices().createFunction(functionData, function, stmt.isTemporary());
+            MutationCode code = result.getMutationCode();
+            switch(code) {
+            case FUNCTION_ALREADY_EXISTS:
+                throw new FunctionAlreadyExistsException(function.getFunctionName(), result
+                        .getFunctions().get(0));
+            case NEWER_FUNCTION_FOUND:
+                // Add function to ConnectionQueryServices so it's cached, but don't add
+                // it to this connection as we can't see it.
+                throw new NewerFunctionAlreadyExistsException(function.getFunctionName(), result.getFunctions().get(0));
+            default:
+                List<PFunction> functions = new ArrayList<PFunction>(1);
+                functions.add(function);
+                result = new MetaDataMutationResult(code, result.getMutationTime(), functions, true);
+                addFunctionToCache(result);
+            }
+        } finally {
+            connection.setAutoCommit(wasAutoCommit);
+        }
+        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())) {
@@ -1807,6 +2008,10 @@ public class MetaDataClient {
         return dropTable(schemaName, tableName, null, statement.getTableType(), statement.ifExists(), statement.cascade());
     }
 
+    public MutationState dropFunction(DropFunctionStatement statement) throws SQLException {
+        return dropFunction(statement.getFunctionName(), statement.ifExists());
+    }
+
     public MutationState dropIndex(DropIndexStatement statement) throws SQLException {
         String schemaName = statement.getTableName().getSchemaName();
         String tableName = statement.getIndexName().getName();
@@ -1814,6 +2019,46 @@ public class MetaDataClient {
         return dropTable(schemaName, tableName, parentTableName, PTableType.INDEX, statement.ifExists(), false);
     }
 
+    private MutationState dropFunction(String functionName, 
+            boolean ifExists) throws SQLException {
+        connection.rollback();
+        boolean wasAutoCommit = connection.getAutoCommit();
+        try {
+            PName tenantId = connection.getTenantId();
+            byte[] key =
+                    SchemaUtil.getFunctionKey(tenantId == null ? ByteUtil.EMPTY_BYTE_ARRAY
+                            : tenantId.getBytes(), Bytes.toBytes(functionName));
+            Long scn = connection.getSCN();
+            long clientTimeStamp = scn == null ? HConstants.LATEST_TIMESTAMP : scn;
+            try {
+                PFunction function = connection.getMetaDataCache().getFunction(new PTableKey(tenantId, functionName));
+                if (function.isTemporaryFunction()) {
+                    connection.removeFunction(tenantId, functionName, clientTimeStamp);
+                    return new MutationState(0, connection);
+                }
+            } catch(FunctionNotFoundException e) {
+                
+            }
+            List<Mutation> functionMetaData = Lists.newArrayListWithExpectedSize(2);
+            Delete functionDelete = new Delete(key, clientTimeStamp);
+            functionMetaData.add(functionDelete);
+            MetaDataMutationResult result = connection.getQueryServices().dropFunction(functionMetaData, ifExists);
+            MutationCode code = result.getMutationCode();
+            switch (code) {
+            case FUNCTION_NOT_FOUND:
+                if (!ifExists) {
+                    throw new FunctionNotFoundException(functionName);
+                }
+                break;
+            default:
+                connection.removeFunction(tenantId, functionName, result.getMutationTime());
+                break;
+            }
+            return new MutationState(0, connection);
+        } finally {
+            connection.setAutoCommit(wasAutoCommit);
+        }
+    }
     private MutationState dropTable(String schemaName, String tableName, String parentTableName, PTableType tableType,
             boolean ifExists, boolean cascade) throws SQLException {
         connection.rollback();
@@ -2658,7 +2903,14 @@ public class MetaDataClient {
         connection.addTable(table);
         return table;
     }
-    
+
+    private List<PFunction> addFunctionToCache(MetaDataMutationResult result) throws SQLException {
+        for(PFunction function: result.getFunctions()) {
+            connection.addFunction(function);
+        }
+        return result.getFunctions();
+    }
+
     private void throwIfAlteringViewPK(ColumnDef col, PTable table) throws SQLException {
         if (col != null && col.isPK() && table.getType() == PTableType.VIEW) {
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_MODIFY_VIEW_PK)


[5/7] phoenix git commit: PHOENIX-538 Support UDFs(Rajeshbabu Chintaguntla)

Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/MetaDataProtos.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/MetaDataProtos.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/MetaDataProtos.java
index f24a292..acb32d2 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/MetaDataProtos.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/MetaDataProtos.java
@@ -53,6 +53,22 @@ public final class MetaDataProtos {
      * <code>PARENT_TABLE_NOT_FOUND = 9;</code>
      */
     PARENT_TABLE_NOT_FOUND(9, 9),
+    /**
+     * <code>FUNCTION_ALREADY_EXISTS = 10;</code>
+     */
+    FUNCTION_ALREADY_EXISTS(10, 10),
+    /**
+     * <code>FUNCTION_NOT_FOUND = 11;</code>
+     */
+    FUNCTION_NOT_FOUND(11, 11),
+    /**
+     * <code>NEWER_FUNCTION_FOUND = 12;</code>
+     */
+    NEWER_FUNCTION_FOUND(12, 12),
+    /**
+     * <code>FUNCTION_NOT_IN_REGION = 13;</code>
+     */
+    FUNCTION_NOT_IN_REGION(13, 13),
     ;
 
     /**
@@ -95,6 +111,22 @@ public final class MetaDataProtos {
      * <code>PARENT_TABLE_NOT_FOUND = 9;</code>
      */
     public static final int PARENT_TABLE_NOT_FOUND_VALUE = 9;
+    /**
+     * <code>FUNCTION_ALREADY_EXISTS = 10;</code>
+     */
+    public static final int FUNCTION_ALREADY_EXISTS_VALUE = 10;
+    /**
+     * <code>FUNCTION_NOT_FOUND = 11;</code>
+     */
+    public static final int FUNCTION_NOT_FOUND_VALUE = 11;
+    /**
+     * <code>NEWER_FUNCTION_FOUND = 12;</code>
+     */
+    public static final int NEWER_FUNCTION_FOUND_VALUE = 12;
+    /**
+     * <code>FUNCTION_NOT_IN_REGION = 13;</code>
+     */
+    public static final int FUNCTION_NOT_IN_REGION_VALUE = 13;
 
 
     public final int getNumber() { return value; }
@@ -111,6 +143,10 @@ public final class MetaDataProtos {
         case 7: return UNALLOWED_TABLE_MUTATION;
         case 8: return NO_PK_COLUMNS;
         case 9: return PARENT_TABLE_NOT_FOUND;
+        case 10: return FUNCTION_ALREADY_EXISTS;
+        case 11: return FUNCTION_NOT_FOUND;
+        case 12: return NEWER_FUNCTION_FOUND;
+        case 13: return FUNCTION_NOT_IN_REGION;
         default: return null;
       }
     }
@@ -232,6 +268,41 @@ public final class MetaDataProtos {
      * <code>optional bytes familyName = 6;</code>
      */
     com.google.protobuf.ByteString getFamilyName();
+
+    // optional bytes functionName = 7;
+    /**
+     * <code>optional bytes functionName = 7;</code>
+     */
+    boolean hasFunctionName();
+    /**
+     * <code>optional bytes functionName = 7;</code>
+     */
+    com.google.protobuf.ByteString getFunctionName();
+
+    // repeated .PFunction function = 8;
+    /**
+     * <code>repeated .PFunction function = 8;</code>
+     */
+    java.util.List<org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction> 
+        getFunctionList();
+    /**
+     * <code>repeated .PFunction function = 8;</code>
+     */
+    org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction getFunction(int index);
+    /**
+     * <code>repeated .PFunction function = 8;</code>
+     */
+    int getFunctionCount();
+    /**
+     * <code>repeated .PFunction function = 8;</code>
+     */
+    java.util.List<? extends org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionOrBuilder> 
+        getFunctionOrBuilderList();
+    /**
+     * <code>repeated .PFunction function = 8;</code>
+     */
+    org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionOrBuilder getFunctionOrBuilder(
+        int index);
   }
   /**
    * Protobuf type {@code MetaDataResponse}
@@ -331,6 +402,19 @@ public final class MetaDataProtos {
               familyName_ = input.readBytes();
               break;
             }
+            case 58: {
+              bitField0_ |= 0x00000020;
+              functionName_ = input.readBytes();
+              break;
+            }
+            case 66: {
+              if (!((mutable_bitField0_ & 0x00000080) == 0x00000080)) {
+                function_ = new java.util.ArrayList<org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction>();
+                mutable_bitField0_ |= 0x00000080;
+              }
+              function_.add(input.readMessage(org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction.PARSER, extensionRegistry));
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -342,6 +426,9 @@ public final class MetaDataProtos {
         if (((mutable_bitField0_ & 0x00000008) == 0x00000008)) {
           tablesToDelete_ = java.util.Collections.unmodifiableList(tablesToDelete_);
         }
+        if (((mutable_bitField0_ & 0x00000080) == 0x00000080)) {
+          function_ = java.util.Collections.unmodifiableList(function_);
+        }
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
@@ -483,6 +570,58 @@ public final class MetaDataProtos {
       return familyName_;
     }
 
+    // optional bytes functionName = 7;
+    public static final int FUNCTIONNAME_FIELD_NUMBER = 7;
+    private com.google.protobuf.ByteString functionName_;
+    /**
+     * <code>optional bytes functionName = 7;</code>
+     */
+    public boolean hasFunctionName() {
+      return ((bitField0_ & 0x00000020) == 0x00000020);
+    }
+    /**
+     * <code>optional bytes functionName = 7;</code>
+     */
+    public com.google.protobuf.ByteString getFunctionName() {
+      return functionName_;
+    }
+
+    // repeated .PFunction function = 8;
+    public static final int FUNCTION_FIELD_NUMBER = 8;
+    private java.util.List<org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction> function_;
+    /**
+     * <code>repeated .PFunction function = 8;</code>
+     */
+    public java.util.List<org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction> getFunctionList() {
+      return function_;
+    }
+    /**
+     * <code>repeated .PFunction function = 8;</code>
+     */
+    public java.util.List<? extends org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionOrBuilder> 
+        getFunctionOrBuilderList() {
+      return function_;
+    }
+    /**
+     * <code>repeated .PFunction function = 8;</code>
+     */
+    public int getFunctionCount() {
+      return function_.size();
+    }
+    /**
+     * <code>repeated .PFunction function = 8;</code>
+     */
+    public org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction getFunction(int index) {
+      return function_.get(index);
+    }
+    /**
+     * <code>repeated .PFunction function = 8;</code>
+     */
+    public org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionOrBuilder getFunctionOrBuilder(
+        int index) {
+      return function_.get(index);
+    }
+
     private void initFields() {
       returnCode_ = org.apache.phoenix.coprocessor.generated.MetaDataProtos.MutationCode.TABLE_ALREADY_EXISTS;
       mutationTime_ = 0L;
@@ -490,6 +629,8 @@ public final class MetaDataProtos {
       tablesToDelete_ = java.util.Collections.emptyList();
       columnName_ = com.google.protobuf.ByteString.EMPTY;
       familyName_ = com.google.protobuf.ByteString.EMPTY;
+      functionName_ = com.google.protobuf.ByteString.EMPTY;
+      function_ = java.util.Collections.emptyList();
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -502,6 +643,12 @@ public final class MetaDataProtos {
           return false;
         }
       }
+      for (int i = 0; i < getFunctionCount(); i++) {
+        if (!getFunction(i).isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
       memoizedIsInitialized = 1;
       return true;
     }
@@ -527,6 +674,12 @@ public final class MetaDataProtos {
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
         output.writeBytes(6, familyName_);
       }
+      if (((bitField0_ & 0x00000020) == 0x00000020)) {
+        output.writeBytes(7, functionName_);
+      }
+      for (int i = 0; i < function_.size(); i++) {
+        output.writeMessage(8, function_.get(i));
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -565,6 +718,14 @@ public final class MetaDataProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeBytesSize(6, familyName_);
       }
+      if (((bitField0_ & 0x00000020) == 0x00000020)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(7, functionName_);
+      }
+      for (int i = 0; i < function_.size(); i++) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(8, function_.get(i));
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -615,6 +776,13 @@ public final class MetaDataProtos {
         result = result && getFamilyName()
             .equals(other.getFamilyName());
       }
+      result = result && (hasFunctionName() == other.hasFunctionName());
+      if (hasFunctionName()) {
+        result = result && getFunctionName()
+            .equals(other.getFunctionName());
+      }
+      result = result && getFunctionList()
+          .equals(other.getFunctionList());
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -652,6 +820,14 @@ public final class MetaDataProtos {
         hash = (37 * hash) + FAMILYNAME_FIELD_NUMBER;
         hash = (53 * hash) + getFamilyName().hashCode();
       }
+      if (hasFunctionName()) {
+        hash = (37 * hash) + FUNCTIONNAME_FIELD_NUMBER;
+        hash = (53 * hash) + getFunctionName().hashCode();
+      }
+      if (getFunctionCount() > 0) {
+        hash = (37 * hash) + FUNCTION_FIELD_NUMBER;
+        hash = (53 * hash) + getFunctionList().hashCode();
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -754,6 +930,7 @@ public final class MetaDataProtos {
       private void maybeForceBuilderInitialization() {
         if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
           getTableFieldBuilder();
+          getFunctionFieldBuilder();
         }
       }
       private static Builder create() {
@@ -778,6 +955,14 @@ public final class MetaDataProtos {
         bitField0_ = (bitField0_ & ~0x00000010);
         familyName_ = com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000020);
+        functionName_ = com.google.protobuf.ByteString.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000040);
+        if (functionBuilder_ == null) {
+          function_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000080);
+        } else {
+          functionBuilder_.clear();
+        }
         return this;
       }
 
@@ -835,6 +1020,19 @@ public final class MetaDataProtos {
           to_bitField0_ |= 0x00000010;
         }
         result.familyName_ = familyName_;
+        if (((from_bitField0_ & 0x00000040) == 0x00000040)) {
+          to_bitField0_ |= 0x00000020;
+        }
+        result.functionName_ = functionName_;
+        if (functionBuilder_ == null) {
+          if (((bitField0_ & 0x00000080) == 0x00000080)) {
+            function_ = java.util.Collections.unmodifiableList(function_);
+            bitField0_ = (bitField0_ & ~0x00000080);
+          }
+          result.function_ = function_;
+        } else {
+          result.function_ = functionBuilder_.build();
+        }
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -876,6 +1074,35 @@ public final class MetaDataProtos {
         if (other.hasFamilyName()) {
           setFamilyName(other.getFamilyName());
         }
+        if (other.hasFunctionName()) {
+          setFunctionName(other.getFunctionName());
+        }
+        if (functionBuilder_ == null) {
+          if (!other.function_.isEmpty()) {
+            if (function_.isEmpty()) {
+              function_ = other.function_;
+              bitField0_ = (bitField0_ & ~0x00000080);
+            } else {
+              ensureFunctionIsMutable();
+              function_.addAll(other.function_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.function_.isEmpty()) {
+            if (functionBuilder_.isEmpty()) {
+              functionBuilder_.dispose();
+              functionBuilder_ = null;
+              function_ = other.function_;
+              bitField0_ = (bitField0_ & ~0x00000080);
+              functionBuilder_ = 
+                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
+                   getFunctionFieldBuilder() : null;
+            } else {
+              functionBuilder_.addAllMessages(other.function_);
+            }
+          }
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -887,6 +1114,12 @@ public final class MetaDataProtos {
             return false;
           }
         }
+        for (int i = 0; i < getFunctionCount(); i++) {
+          if (!getFunction(i).isInitialized()) {
+            
+            return false;
+          }
+        }
         return true;
       }
 
@@ -1239,6 +1472,282 @@ public final class MetaDataProtos {
         return this;
       }
 
+      // optional bytes functionName = 7;
+      private com.google.protobuf.ByteString functionName_ = com.google.protobuf.ByteString.EMPTY;
+      /**
+       * <code>optional bytes functionName = 7;</code>
+       */
+      public boolean hasFunctionName() {
+        return ((bitField0_ & 0x00000040) == 0x00000040);
+      }
+      /**
+       * <code>optional bytes functionName = 7;</code>
+       */
+      public com.google.protobuf.ByteString getFunctionName() {
+        return functionName_;
+      }
+      /**
+       * <code>optional bytes functionName = 7;</code>
+       */
+      public Builder setFunctionName(com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000040;
+        functionName_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bytes functionName = 7;</code>
+       */
+      public Builder clearFunctionName() {
+        bitField0_ = (bitField0_ & ~0x00000040);
+        functionName_ = getDefaultInstance().getFunctionName();
+        onChanged();
+        return this;
+      }
+
+      // repeated .PFunction function = 8;
+      private java.util.List<org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction> function_ =
+        java.util.Collections.emptyList();
+      private void ensureFunctionIsMutable() {
+        if (!((bitField0_ & 0x00000080) == 0x00000080)) {
+          function_ = new java.util.ArrayList<org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction>(function_);
+          bitField0_ |= 0x00000080;
+         }
+      }
+
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction, org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction.Builder, org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionOrBuilder> functionBuilder_;
+
+      /**
+       * <code>repeated .PFunction function = 8;</code>
+       */
+      public java.util.List<org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction> getFunctionList() {
+        if (functionBuilder_ == null) {
+          return java.util.Collections.unmodifiableList(function_);
+        } else {
+          return functionBuilder_.getMessageList();
+        }
+      }
+      /**
+       * <code>repeated .PFunction function = 8;</code>
+       */
+      public int getFunctionCount() {
+        if (functionBuilder_ == null) {
+          return function_.size();
+        } else {
+          return functionBuilder_.getCount();
+        }
+      }
+      /**
+       * <code>repeated .PFunction function = 8;</code>
+       */
+      public org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction getFunction(int index) {
+        if (functionBuilder_ == null) {
+          return function_.get(index);
+        } else {
+          return functionBuilder_.getMessage(index);
+        }
+      }
+      /**
+       * <code>repeated .PFunction function = 8;</code>
+       */
+      public Builder setFunction(
+          int index, org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction value) {
+        if (functionBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureFunctionIsMutable();
+          function_.set(index, value);
+          onChanged();
+        } else {
+          functionBuilder_.setMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .PFunction function = 8;</code>
+       */
+      public Builder setFunction(
+          int index, org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction.Builder builderForValue) {
+        if (functionBuilder_ == null) {
+          ensureFunctionIsMutable();
+          function_.set(index, builderForValue.build());
+          onChanged();
+        } else {
+          functionBuilder_.setMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .PFunction function = 8;</code>
+       */
+      public Builder addFunction(org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction value) {
+        if (functionBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureFunctionIsMutable();
+          function_.add(value);
+          onChanged();
+        } else {
+          functionBuilder_.addMessage(value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .PFunction function = 8;</code>
+       */
+      public Builder addFunction(
+          int index, org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction value) {
+        if (functionBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureFunctionIsMutable();
+          function_.add(index, value);
+          onChanged();
+        } else {
+          functionBuilder_.addMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .PFunction function = 8;</code>
+       */
+      public Builder addFunction(
+          org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction.Builder builderForValue) {
+        if (functionBuilder_ == null) {
+          ensureFunctionIsMutable();
+          function_.add(builderForValue.build());
+          onChanged();
+        } else {
+          functionBuilder_.addMessage(builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .PFunction function = 8;</code>
+       */
+      public Builder addFunction(
+          int index, org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction.Builder builderForValue) {
+        if (functionBuilder_ == null) {
+          ensureFunctionIsMutable();
+          function_.add(index, builderForValue.build());
+          onChanged();
+        } else {
+          functionBuilder_.addMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .PFunction function = 8;</code>
+       */
+      public Builder addAllFunction(
+          java.lang.Iterable<? extends org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction> values) {
+        if (functionBuilder_ == null) {
+          ensureFunctionIsMutable();
+          super.addAll(values, function_);
+          onChanged();
+        } else {
+          functionBuilder_.addAllMessages(values);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .PFunction function = 8;</code>
+       */
+      public Builder clearFunction() {
+        if (functionBuilder_ == null) {
+          function_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000080);
+          onChanged();
+        } else {
+          functionBuilder_.clear();
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .PFunction function = 8;</code>
+       */
+      public Builder removeFunction(int index) {
+        if (functionBuilder_ == null) {
+          ensureFunctionIsMutable();
+          function_.remove(index);
+          onChanged();
+        } else {
+          functionBuilder_.remove(index);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .PFunction function = 8;</code>
+       */
+      public org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction.Builder getFunctionBuilder(
+          int index) {
+        return getFunctionFieldBuilder().getBuilder(index);
+      }
+      /**
+       * <code>repeated .PFunction function = 8;</code>
+       */
+      public org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionOrBuilder getFunctionOrBuilder(
+          int index) {
+        if (functionBuilder_ == null) {
+          return function_.get(index);  } else {
+          return functionBuilder_.getMessageOrBuilder(index);
+        }
+      }
+      /**
+       * <code>repeated .PFunction function = 8;</code>
+       */
+      public java.util.List<? extends org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionOrBuilder> 
+           getFunctionOrBuilderList() {
+        if (functionBuilder_ != null) {
+          return functionBuilder_.getMessageOrBuilderList();
+        } else {
+          return java.util.Collections.unmodifiableList(function_);
+        }
+      }
+      /**
+       * <code>repeated .PFunction function = 8;</code>
+       */
+      public org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction.Builder addFunctionBuilder() {
+        return getFunctionFieldBuilder().addBuilder(
+            org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .PFunction function = 8;</code>
+       */
+      public org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction.Builder addFunctionBuilder(
+          int index) {
+        return getFunctionFieldBuilder().addBuilder(
+            index, org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .PFunction function = 8;</code>
+       */
+      public java.util.List<org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction.Builder> 
+           getFunctionBuilderList() {
+        return getFunctionFieldBuilder().getBuilderList();
+      }
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction, org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction.Builder, org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionOrBuilder> 
+          getFunctionFieldBuilder() {
+        if (functionBuilder_ == null) {
+          functionBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
+              org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction, org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction.Builder, org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionOrBuilder>(
+                  function_,
+                  ((bitField0_ & 0x00000080) == 0x00000080),
+                  getParentForChildren(),
+                  isClean());
+          function_ = null;
+        }
+        return functionBuilder_;
+      }
+
       // @@protoc_insertion_point(builder_scope:MetaDataResponse)
     }
 
@@ -2093,46 +2602,76 @@ public final class MetaDataProtos {
     // @@protoc_insertion_point(class_scope:GetTableRequest)
   }
 
-  public interface CreateTableRequestOrBuilder
+  public interface GetFunctionsRequestOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
 
-    // repeated bytes tableMetadataMutations = 1;
+    // required bytes tenantId = 1;
     /**
-     * <code>repeated bytes tableMetadataMutations = 1;</code>
+     * <code>required bytes tenantId = 1;</code>
      */
-    java.util.List<com.google.protobuf.ByteString> getTableMetadataMutationsList();
+    boolean hasTenantId();
     /**
-     * <code>repeated bytes tableMetadataMutations = 1;</code>
+     * <code>required bytes tenantId = 1;</code>
      */
-    int getTableMetadataMutationsCount();
+    com.google.protobuf.ByteString getTenantId();
+
+    // repeated bytes functionNames = 2;
     /**
-     * <code>repeated bytes tableMetadataMutations = 1;</code>
+     * <code>repeated bytes functionNames = 2;</code>
      */
-    com.google.protobuf.ByteString getTableMetadataMutations(int index);
-  }
-  /**
-   * Protobuf type {@code CreateTableRequest}
-   *
-   * <pre>
-   * each byte array represents a MutationProto instance
-   * </pre>
-   */
-  public static final class CreateTableRequest extends
-      com.google.protobuf.GeneratedMessage
-      implements CreateTableRequestOrBuilder {
-    // Use CreateTableRequest.newBuilder() to construct.
-    private CreateTableRequest(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
-      super(builder);
-      this.unknownFields = builder.getUnknownFields();
-    }
-    private CreateTableRequest(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+    java.util.List<com.google.protobuf.ByteString> getFunctionNamesList();
+    /**
+     * <code>repeated bytes functionNames = 2;</code>
+     */
+    int getFunctionNamesCount();
+    /**
+     * <code>repeated bytes functionNames = 2;</code>
+     */
+    com.google.protobuf.ByteString getFunctionNames(int index);
 
-    private static final CreateTableRequest defaultInstance;
-    public static CreateTableRequest getDefaultInstance() {
+    // repeated int64 functionTimestamps = 3;
+    /**
+     * <code>repeated int64 functionTimestamps = 3;</code>
+     */
+    java.util.List<java.lang.Long> getFunctionTimestampsList();
+    /**
+     * <code>repeated int64 functionTimestamps = 3;</code>
+     */
+    int getFunctionTimestampsCount();
+    /**
+     * <code>repeated int64 functionTimestamps = 3;</code>
+     */
+    long getFunctionTimestamps(int index);
+
+    // required int64 clientTimestamp = 4;
+    /**
+     * <code>required int64 clientTimestamp = 4;</code>
+     */
+    boolean hasClientTimestamp();
+    /**
+     * <code>required int64 clientTimestamp = 4;</code>
+     */
+    long getClientTimestamp();
+  }
+  /**
+   * Protobuf type {@code GetFunctionsRequest}
+   */
+  public static final class GetFunctionsRequest extends
+      com.google.protobuf.GeneratedMessage
+      implements GetFunctionsRequestOrBuilder {
+    // Use GetFunctionsRequest.newBuilder() to construct.
+    private GetFunctionsRequest(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private GetFunctionsRequest(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final GetFunctionsRequest defaultInstance;
+    public static GetFunctionsRequest getDefaultInstance() {
       return defaultInstance;
     }
 
-    public CreateTableRequest getDefaultInstanceForType() {
+    public GetFunctionsRequest getDefaultInstanceForType() {
       return defaultInstance;
     }
 
@@ -2142,7 +2681,7 @@ public final class MetaDataProtos {
         getUnknownFields() {
       return this.unknownFields;
     }
-    private CreateTableRequest(
+    private GetFunctionsRequest(
         com.google.protobuf.CodedInputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
@@ -2166,11 +2705,42 @@ public final class MetaDataProtos {
               break;
             }
             case 10: {
-              if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
-                tableMetadataMutations_ = new java.util.ArrayList<com.google.protobuf.ByteString>();
-                mutable_bitField0_ |= 0x00000001;
+              bitField0_ |= 0x00000001;
+              tenantId_ = input.readBytes();
+              break;
+            }
+            case 18: {
+              if (!((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
+                functionNames_ = new java.util.ArrayList<com.google.protobuf.ByteString>();
+                mutable_bitField0_ |= 0x00000002;
               }
-              tableMetadataMutations_.add(input.readBytes());
+              functionNames_.add(input.readBytes());
+              break;
+            }
+            case 24: {
+              if (!((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
+                functionTimestamps_ = new java.util.ArrayList<java.lang.Long>();
+                mutable_bitField0_ |= 0x00000004;
+              }
+              functionTimestamps_.add(input.readInt64());
+              break;
+            }
+            case 26: {
+              int length = input.readRawVarint32();
+              int limit = input.pushLimit(length);
+              if (!((mutable_bitField0_ & 0x00000004) == 0x00000004) && input.getBytesUntilLimit() > 0) {
+                functionTimestamps_ = new java.util.ArrayList<java.lang.Long>();
+                mutable_bitField0_ |= 0x00000004;
+              }
+              while (input.getBytesUntilLimit() > 0) {
+                functionTimestamps_.add(input.readInt64());
+              }
+              input.popLimit(limit);
+              break;
+            }
+            case 32: {
+              bitField0_ |= 0x00000002;
+              clientTimestamp_ = input.readInt64();
               break;
             }
           }
@@ -2181,8 +2751,11 @@ public final class MetaDataProtos {
         throw new com.google.protobuf.InvalidProtocolBufferException(
             e.getMessage()).setUnfinishedMessage(this);
       } finally {
-        if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
-          tableMetadataMutations_ = java.util.Collections.unmodifiableList(tableMetadataMutations_);
+        if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
+          functionNames_ = java.util.Collections.unmodifiableList(functionNames_);
+        }
+        if (((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
+          functionTimestamps_ = java.util.Collections.unmodifiableList(functionTimestamps_);
         }
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
@@ -2190,62 +2763,129 @@ public final class MetaDataProtos {
     }
     public static final com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
-      return org.apache.phoenix.coprocessor.generated.MetaDataProtos.internal_static_CreateTableRequest_descriptor;
+      return org.apache.phoenix.coprocessor.generated.MetaDataProtos.internal_static_GetFunctionsRequest_descriptor;
     }
 
     protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
         internalGetFieldAccessorTable() {
-      return org.apache.phoenix.coprocessor.generated.MetaDataProtos.internal_static_CreateTableRequest_fieldAccessorTable
+      return org.apache.phoenix.coprocessor.generated.MetaDataProtos.internal_static_GetFunctionsRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
-              org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest.class, org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest.Builder.class);
+              org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetFunctionsRequest.class, org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetFunctionsRequest.Builder.class);
     }
 
-    public static com.google.protobuf.Parser<CreateTableRequest> PARSER =
-        new com.google.protobuf.AbstractParser<CreateTableRequest>() {
-      public CreateTableRequest parsePartialFrom(
+    public static com.google.protobuf.Parser<GetFunctionsRequest> PARSER =
+        new com.google.protobuf.AbstractParser<GetFunctionsRequest>() {
+      public GetFunctionsRequest parsePartialFrom(
           com.google.protobuf.CodedInputStream input,
           com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws com.google.protobuf.InvalidProtocolBufferException {
-        return new CreateTableRequest(input, extensionRegistry);
+        return new GetFunctionsRequest(input, extensionRegistry);
       }
     };
 
     @java.lang.Override
-    public com.google.protobuf.Parser<CreateTableRequest> getParserForType() {
+    public com.google.protobuf.Parser<GetFunctionsRequest> getParserForType() {
       return PARSER;
     }
 
-    // repeated bytes tableMetadataMutations = 1;
-    public static final int TABLEMETADATAMUTATIONS_FIELD_NUMBER = 1;
-    private java.util.List<com.google.protobuf.ByteString> tableMetadataMutations_;
+    private int bitField0_;
+    // required bytes tenantId = 1;
+    public static final int TENANTID_FIELD_NUMBER = 1;
+    private com.google.protobuf.ByteString tenantId_;
     /**
-     * <code>repeated bytes tableMetadataMutations = 1;</code>
+     * <code>required bytes tenantId = 1;</code>
+     */
+    public boolean hasTenantId() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required bytes tenantId = 1;</code>
+     */
+    public com.google.protobuf.ByteString getTenantId() {
+      return tenantId_;
+    }
+
+    // repeated bytes functionNames = 2;
+    public static final int FUNCTIONNAMES_FIELD_NUMBER = 2;
+    private java.util.List<com.google.protobuf.ByteString> functionNames_;
+    /**
+     * <code>repeated bytes functionNames = 2;</code>
      */
     public java.util.List<com.google.protobuf.ByteString>
-        getTableMetadataMutationsList() {
-      return tableMetadataMutations_;
+        getFunctionNamesList() {
+      return functionNames_;
     }
     /**
-     * <code>repeated bytes tableMetadataMutations = 1;</code>
+     * <code>repeated bytes functionNames = 2;</code>
      */
-    public int getTableMetadataMutationsCount() {
-      return tableMetadataMutations_.size();
+    public int getFunctionNamesCount() {
+      return functionNames_.size();
     }
     /**
-     * <code>repeated bytes tableMetadataMutations = 1;</code>
+     * <code>repeated bytes functionNames = 2;</code>
      */
-    public com.google.protobuf.ByteString getTableMetadataMutations(int index) {
-      return tableMetadataMutations_.get(index);
+    public com.google.protobuf.ByteString getFunctionNames(int index) {
+      return functionNames_.get(index);
+    }
+
+    // repeated int64 functionTimestamps = 3;
+    public static final int FUNCTIONTIMESTAMPS_FIELD_NUMBER = 3;
+    private java.util.List<java.lang.Long> functionTimestamps_;
+    /**
+     * <code>repeated int64 functionTimestamps = 3;</code>
+     */
+    public java.util.List<java.lang.Long>
+        getFunctionTimestampsList() {
+      return functionTimestamps_;
+    }
+    /**
+     * <code>repeated int64 functionTimestamps = 3;</code>
+     */
+    public int getFunctionTimestampsCount() {
+      return functionTimestamps_.size();
+    }
+    /**
+     * <code>repeated int64 functionTimestamps = 3;</code>
+     */
+    public long getFunctionTimestamps(int index) {
+      return functionTimestamps_.get(index);
+    }
+
+    // required int64 clientTimestamp = 4;
+    public static final int CLIENTTIMESTAMP_FIELD_NUMBER = 4;
+    private long clientTimestamp_;
+    /**
+     * <code>required int64 clientTimestamp = 4;</code>
+     */
+    public boolean hasClientTimestamp() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>required int64 clientTimestamp = 4;</code>
+     */
+    public long getClientTimestamp() {
+      return clientTimestamp_;
     }
 
     private void initFields() {
-      tableMetadataMutations_ = java.util.Collections.emptyList();
+      tenantId_ = com.google.protobuf.ByteString.EMPTY;
+      functionNames_ = java.util.Collections.emptyList();
+      functionTimestamps_ = java.util.Collections.emptyList();
+      clientTimestamp_ = 0L;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
       byte isInitialized = memoizedIsInitialized;
       if (isInitialized != -1) return isInitialized == 1;
 
+      if (!hasTenantId()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasClientTimestamp()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
       memoizedIsInitialized = 1;
       return true;
     }
@@ -2253,8 +2893,17 @@ public final class MetaDataProtos {
     public void writeTo(com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       getSerializedSize();
-      for (int i = 0; i < tableMetadataMutations_.size(); i++) {
-        output.writeBytes(1, tableMetadataMutations_.get(i));
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeBytes(1, tenantId_);
+      }
+      for (int i = 0; i < functionNames_.size(); i++) {
+        output.writeBytes(2, functionNames_.get(i));
+      }
+      for (int i = 0; i < functionTimestamps_.size(); i++) {
+        output.writeInt64(3, functionTimestamps_.get(i));
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeInt64(4, clientTimestamp_);
       }
       getUnknownFields().writeTo(output);
     }
@@ -2265,14 +2914,31 @@ public final class MetaDataProtos {
       if (size != -1) return size;
 
       size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(1, tenantId_);
+      }
       {
         int dataSize = 0;
-        for (int i = 0; i < tableMetadataMutations_.size(); i++) {
+        for (int i = 0; i < functionNames_.size(); i++) {
           dataSize += com.google.protobuf.CodedOutputStream
-            .computeBytesSizeNoTag(tableMetadataMutations_.get(i));
+            .computeBytesSizeNoTag(functionNames_.get(i));
         }
         size += dataSize;
-        size += 1 * getTableMetadataMutationsList().size();
+        size += 1 * getFunctionNamesList().size();
+      }
+      {
+        int dataSize = 0;
+        for (int i = 0; i < functionTimestamps_.size(); i++) {
+          dataSize += com.google.protobuf.CodedOutputStream
+            .computeInt64SizeNoTag(functionTimestamps_.get(i));
+        }
+        size += dataSize;
+        size += 1 * getFunctionTimestampsList().size();
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt64Size(4, clientTimestamp_);
       }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
@@ -2291,14 +2957,26 @@ public final class MetaDataProtos {
       if (obj == this) {
        return true;
       }
-      if (!(obj instanceof org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest)) {
+      if (!(obj instanceof org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetFunctionsRequest)) {
         return super.equals(obj);
       }
-      org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest other = (org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest) obj;
+      org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetFunctionsRequest other = (org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetFunctionsRequest) obj;
 
       boolean result = true;
-      result = result && getTableMetadataMutationsList()
-          .equals(other.getTableMetadataMutationsList());
+      result = result && (hasTenantId() == other.hasTenantId());
+      if (hasTenantId()) {
+        result = result && getTenantId()
+            .equals(other.getTenantId());
+      }
+      result = result && getFunctionNamesList()
+          .equals(other.getFunctionNamesList());
+      result = result && getFunctionTimestampsList()
+          .equals(other.getFunctionTimestampsList());
+      result = result && (hasClientTimestamp() == other.hasClientTimestamp());
+      if (hasClientTimestamp()) {
+        result = result && (getClientTimestamp()
+            == other.getClientTimestamp());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -2312,62 +2990,74 @@ public final class MetaDataProtos {
       }
       int hash = 41;
       hash = (19 * hash) + getDescriptorForType().hashCode();
-      if (getTableMetadataMutationsCount() > 0) {
-        hash = (37 * hash) + TABLEMETADATAMUTATIONS_FIELD_NUMBER;
-        hash = (53 * hash) + getTableMetadataMutationsList().hashCode();
+      if (hasTenantId()) {
+        hash = (37 * hash) + TENANTID_FIELD_NUMBER;
+        hash = (53 * hash) + getTenantId().hashCode();
+      }
+      if (getFunctionNamesCount() > 0) {
+        hash = (37 * hash) + FUNCTIONNAMES_FIELD_NUMBER;
+        hash = (53 * hash) + getFunctionNamesList().hashCode();
+      }
+      if (getFunctionTimestampsCount() > 0) {
+        hash = (37 * hash) + FUNCTIONTIMESTAMPS_FIELD_NUMBER;
+        hash = (53 * hash) + getFunctionTimestampsList().hashCode();
+      }
+      if (hasClientTimestamp()) {
+        hash = (37 * hash) + CLIENTTIMESTAMP_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getClientTimestamp());
       }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
     }
 
-    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest parseFrom(
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetFunctionsRequest parseFrom(
         com.google.protobuf.ByteString data)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest parseFrom(
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetFunctionsRequest parseFrom(
         com.google.protobuf.ByteString data,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
-    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest parseFrom(byte[] data)
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetFunctionsRequest parseFrom(byte[] data)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest parseFrom(
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetFunctionsRequest parseFrom(
         byte[] data,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
-    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest parseFrom(java.io.InputStream input)
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetFunctionsRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
       return PARSER.parseFrom(input);
     }
-    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest parseFrom(
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetFunctionsRequest parseFrom(
         java.io.InputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
       return PARSER.parseFrom(input, extensionRegistry);
     }
-    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest parseDelimitedFrom(java.io.InputStream input)
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetFunctionsRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
       return PARSER.parseDelimitedFrom(input);
     }
-    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest parseDelimitedFrom(
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetFunctionsRequest parseDelimitedFrom(
         java.io.InputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
       return PARSER.parseDelimitedFrom(input, extensionRegistry);
     }
-    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest parseFrom(
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetFunctionsRequest parseFrom(
         com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
       return PARSER.parseFrom(input);
     }
-    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest parseFrom(
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetFunctionsRequest parseFrom(
         com.google.protobuf.CodedInputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
@@ -2376,7 +3066,7 @@ public final class MetaDataProtos {
 
     public static Builder newBuilder() { return Builder.create(); }
     public Builder newBuilderForType() { return newBuilder(); }
-    public static Builder newBuilder(org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest prototype) {
+    public static Builder newBuilder(org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetFunctionsRequest prototype) {
       return newBuilder().mergeFrom(prototype);
     }
     public Builder toBuilder() { return newBuilder(this); }
@@ -2388,28 +3078,24 @@ public final class MetaDataProtos {
       return builder;
     }
     /**
-     * Protobuf type {@code CreateTableRequest}
-     *
-     * <pre>
-     * each byte array represents a MutationProto instance
-     * </pre>
+     * Protobuf type {@code GetFunctionsRequest}
      */
     public static final class Builder extends
         com.google.protobuf.GeneratedMessage.Builder<Builder>
-       implements org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequestOrBuilder {
+       implements org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetFunctionsRequestOrBuilder {
       public static final com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
-        return org.apache.phoenix.coprocessor.generated.MetaDataProtos.internal_static_CreateTableRequest_descriptor;
+        return org.apache.phoenix.coprocessor.generated.MetaDataProtos.internal_static_GetFunctionsRequest_descriptor;
       }
 
       protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
           internalGetFieldAccessorTable() {
-        return org.apache.phoenix.coprocessor.generated.MetaDataProtos.internal_static_CreateTableRequest_fieldAccessorTable
+        return org.apache.phoenix.coprocessor.generated.MetaDataProtos.internal_static_GetFunctionsRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
-                org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest.class, org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest.Builder.class);
+                org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetFunctionsRequest.class, org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetFunctionsRequest.Builder.class);
       }
 
-      // Construct using org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest.newBuilder()
+      // Construct using org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetFunctionsRequest.newBuilder()
       private Builder() {
         maybeForceBuilderInitialization();
       }
@@ -2429,8 +3115,14 @@ public final class MetaDataProtos {
 
       public Builder clear() {
         super.clear();
-        tableMetadataMutations_ = java.util.Collections.emptyList();
+        tenantId_ = com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000001);
+        functionNames_ = java.util.Collections.emptyList();
+        bitField0_ = (bitField0_ & ~0x00000002);
+        functionTimestamps_ = java.util.Collections.emptyList();
+        bitField0_ = (bitField0_ & ~0x00000004);
+        clientTimestamp_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000008);
         return this;
       }
 
@@ -2440,71 +3132,2102 @@ public final class MetaDataProtos {
 
       public com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
-        return org.apache.phoenix.coprocessor.generated.MetaDataProtos.internal_static_CreateTableRequest_descriptor;
+        return org.apache.phoenix.coprocessor.generated.MetaDataProtos.internal_static_GetFunctionsRequest_descriptor;
       }
 
-      public org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest getDefaultInstanceForType() {
-        return org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest.getDefaultInstance();
+      public org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetFunctionsRequest getDefaultInstanceForType() {
+        return org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetFunctionsRequest.getDefaultInstance();
       }
 
-      public org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest build() {
-        org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest result = buildPartial();
+      public org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetFunctionsRequest build() {
+        org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetFunctionsRequest result = buildPartial();
         if (!result.isInitialized()) {
           throw newUninitializedMessageException(result);
         }
         return result;
       }
 
-      public org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest buildPartial() {
-        org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest result = new org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest(this);
+      public org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetFunctionsRequest buildPartial() {
+        org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetFunctionsRequest result = new org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetFunctionsRequest(this);
         int from_bitField0_ = bitField0_;
-        if (((bitField0_ & 0x00000001) == 0x00000001)) {
-          tableMetadataMutations_ = java.util.Collections.unmodifiableList(tableMetadataMutations_);
-          bitField0_ = (bitField0_ & ~0x00000001);
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
         }
-        result.tableMetadataMutations_ = tableMetadataMutations_;
+        result.tenantId_ = tenantId_;
+        if (((bitField0_ & 0x00000002) == 0x00000002)) {
+          functionNames_ = java.util.Collections.unmodifiableList(functionNames_);
+          bitField0_ = (bitField0_ & ~0x00000002);
+        }
+        result.functionNames_ = functionNames_;
+        if (((bitField0_ & 0x00000004) == 0x00000004)) {
+          functionTimestamps_ = java.util.Collections.unmodifiableList(functionTimestamps_);
+          bitField0_ = (bitField0_ & ~0x00000004);
+        }
+        result.functionTimestamps_ = functionTimestamps_;
+        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.clientTimestamp_ = clientTimestamp_;
+        result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
       }
 
       public Builder mergeFrom(com.google.protobuf.Message other) {
-        if (other instanceof org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest) {
-          return mergeFrom((org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest)other);
+        if (other instanceof org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetFunctionsRequest) {
+          return mergeFrom((org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetFunctionsRequest)other);
         } else {
           super.mergeFrom(other);
           return this;
         }
       }
 
-      public Builder mergeFrom(org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest other) {
-        if (other == org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest.getDefaultInstance()) return this;
-        if (!other.tableMetadataMutations_.isEmpty()) {
-          if (tableMetadataMutations_.isEmpty()) {
-            tableMetadataMutations_ = other.tableMetadataMutations_;
-            bitField0_ = (bitField0_ & ~0x00000001);
+      public Builder mergeFrom(org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetFunctionsRequest other) {
+        if (other == org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetFunctionsRequest.getDefaultInstance()) return this;
+        if (other.hasTenantId()) {
+          setTenantId(other.getTenantId());
+        }
+        if (!other.functionNames_.isEmpty()) {
+          if (functionNames_.isEmpty()) {
+            functionNames_ = other.functionNames_;
+            bitField0_ = (bitField0_ & ~0x00000002);
           } else {
-            ensureTableMetadataMutationsIsMutable();
-            tableMetadataMutations_.addAll(other.tableMetadataMutations_);
+            ensureFunctionNamesIsMutable();
+            functionNames_.addAll(other.functionNames_);
+          }
+          onChanged();
+        }
+        if (!other.functionTimestamps_.isEmpty()) {
+          if (functionTimestamps_.isEmpty()) {
+            functionTimestamps_ = other.functionTimestamps_;
+            bitField0_ = (bitField0_ & ~0x00000004);
+          } else {
+            ensureFunctionTimestampsIsMutable();
+            functionTimestamps_.addAll(other.functionTimestamps_);
           }
           onChanged();
         }
+        if (other.hasClientTimestamp()) {
+          setClientTimestamp(other.getClientTimestamp());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
 
       public final boolean isInitialized() {
-        return true;
-      }
-
+        if (!hasTenantId()) {
+          
+          return false;
+        }
+        if (!hasClientTimestamp()) {
+          
+          return false;
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetFunctionsRequest parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetFunctionsRequest) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // required bytes tenantId = 1;
+      private com.google.protobuf.ByteString tenantId_ = com.google.protobuf.ByteString.EMPTY;
+      /**
+       * <code>required bytes tenantId = 1;</code>
+       */
+      public boolean hasTenantId() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required bytes tenantId = 1;</code>
+       */
+      public com.google.protobuf.ByteString getTenantId() {
+        return tenantId_;
+      }
+      /**
+       * <code>required bytes tenantId = 1;</code>
+       */
+      public Builder setTenantId(com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        tenantId_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required bytes tenantId = 1;</code>
+       */
+      public Builder clearTenantId() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        tenantId_ = getDefaultInstance().getTenantId();
+        onChanged();
+        return this;
+      }
+
+      // repeated bytes functionNames = 2;
+      private java.util.List<com.google.protobuf.ByteString> functionNames_ = java.util.Collections.emptyList();
+      private void ensureFunctionNamesIsMutable() {
+        if (!((bitField0_ & 0x00000002) == 0x00000002)) {
+          functionNames_ = new java.util.ArrayList<com.google.protobuf.ByteString>(functionNames_);
+          bitField0_ |= 0x00000002;
+         }
+      }
+      /**
+       * <code>repeated bytes functionNames = 2;</code>
+       */
+      public java.util.List<com.google.protobuf.ByteString>
+          getFunctionNamesList() {
+        return java.util.Collections.unmodifiableList(functionNames_);
+      }
+      /**
+       * <code>repeated bytes functionNames = 2;</code>
+       */
+      public int getFunctionNamesCount() {
+        return functionNames_.size();
+      }
+      /**
+       * <code>repeated bytes functionNames = 2;</code>
+       */
+      public com.google.protobuf.ByteString getFunctionNames(int index) {
+        return functionNames_.get(index);
+      }
+      /**
+       * <code>repeated bytes functionNames = 2;</code>
+       */
+      public Builder setFunctionNames(
+          int index, com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  ensureFunctionNamesIsMutable();
+        functionNames_.set(index, value);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated bytes functionNames = 2;</code>
+       */
+      public Builder addFunctionNames(com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  ensureFunctionNamesIsMutable();
+        functionNames_.add(value);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated bytes functionNames = 2;</code>
+       */
+      public Builder addAllFunctionNames(
+          java.lang.Iterable<? extends com.google.protobuf.ByteString> values) {
+        ensureFunctionNamesIsMutable();
+        super.addAll(values, functionNames_);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated bytes functionNames = 2;</code>
+       */
+      public Builder clearFunctionNames() {
+        functionNames_ = java.util.Collections.emptyList();
+        bitField0_ = (bitField0_ & ~0x00000002);
+        onChanged();
+        return this;
+      }
+
+      // repeated int64 functionTimestamps = 3;
+      private java.util.List<java.lang.Long> functionTimestamps_ = java.util.Collections.emptyList();
+      private void ensureFunctionTimestampsIsMutable() {
+        if (!((bitField0_ & 0x00000004) == 0x00000004)) {
+          functionTimestamps_ = new java.util.ArrayList<java.lang.Long>(functionTimestamps_);
+          bitField0_ |= 0x00000004;
+         }
+      }
+      /**
+       * <code>repeated int64 functionTimestamps = 3;</code>
+       */
+      public java.util.List<java.lang.Long>
+          getFunctionTimestampsList() {
+        return java.util.Collections.unmodifiableList(functionTimestamps_);
+      }
+      /**
+       * <code>repeated int64 functionTimestamps = 3;</code>
+       */
+      public int getFunctionTimestampsCount() {
+        return functionTimestamps_.size();
+      }
+      /**
+       * <code>repeated int64 functionTimestamps = 3;</code>
+       */
+      public long getFunctionTimestamps(int index) {
+        return functionTimestamps_.get(index);
+      }
+      /**
+       * <code>repeated int64 functionTimestamps = 3;</code>
+       */
+      public Builder setFunctionTimestamps(
+          int index, long value) {
+        ensureFunctionTimestampsIsMutable();
+        functionTimestamps_.set(index, value);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated int64 functionTimestamps = 3;</code>
+       */
+      public Builder addFunctionTimestamps(long value) {
+        ensureFunctionTimestampsIsMutable();
+        functionTimestamps_.add(value);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated int64 functionTimestamps = 3;</code>
+       */
+      public Builder addAllFunctionTimestamps(
+          java.lang.Iterable<? extends java.lang.Long> values) {
+        ensureFunctionTimestampsIsMutable();
+        super.addAll(values, functionTimestamps_);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated int64 functionTimestamps = 3;</code>
+       */
+      public Builder clearFunctionTimestamps() {
+        functionTimestamps_ = java.util.Collections.emptyList();
+        bitField0_ = (bitField0_ & ~0x00000004);
+        onChanged();
+        return this;
+      }
+
+      // required int64 clientTimestamp = 4;
+      private long clientTimestamp_ ;
+      /**
+       * <code>required int64 clientTimestamp = 4;</code>
+       */
+      public boolean hasClientTimestamp() {
+        return ((bitField0_ & 0x00000008) == 0x00000008);
+      }
+      /**
+       * <code>required int64 clientTimestamp = 4;</code>
+       */
+      public long getClientTimestamp() {
+        return clientTimestamp_;
+      }
+      /**
+       * <code>required int64 clientTimestamp = 4;</code>
+       */
+      public Builder setClientTimestamp(long value) {
+        bitField0_ |= 0x00000008;
+        clientTimestamp_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required int64 clientTimestamp = 4;</code>
+       */
+      public Builder clearClientTimestamp() {
+        bitField0_ = (bitField0_ & ~0x00000008);
+        clientTimestamp_ = 0L;
+        onChanged();
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:GetFunctionsRequest)
+    }
+
+    static {
+      defaultInstance = new GetFunctionsRequest(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:GetFunctionsRequest)
+  }
+
+  public interface CreateTableRequestOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // repeated bytes tableMetadataMutations = 1;
+    /**
+     * <code>repeated bytes tableMetadataMutations = 1;</code>
+     */
+    java.util.List<com.google.protobuf.ByteString> getTableMetadataMutationsList();
+    /**
+     * <code>repeated bytes tableMetadataMutations = 1;</code>
+     */
+    int getTableMetadataMutationsCount();
+    /**
+     * <code>repeated bytes tableMetadataMutations = 1;</code>
+     */
+    com.google.protobuf.ByteString getTableMetadataMutations(int index);
+  }
+  /**
+   * Protobuf type {@code CreateTableRequest}
+   *
+   * <pre>
+   * each byte array represents a MutationProto instance
+   * </pre>
+   */
+  public static final class CreateTableRequest extends
+      com.google.protobuf.GeneratedMessage
+      implements CreateTableRequestOrBuilder {
+    // Use CreateTableRequest.newBuilder() to construct.
+    private CreateTableRequest(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private CreateTableRequest(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final CreateTableRequest defaultInstance;
+    public static CreateTableRequest getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public CreateTableRequest getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private CreateTableRequest(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+                tableMetadataMutations_ = new java.util.ArrayList<com.google.protobuf.ByteString>();
+                mutable_bitField0_ |= 0x00000001;
+              }
+              tableMetadataMutations_.add(input.readBytes());
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+          tableMetadataMutations_ = java.util.Collections.unmodifiableList(tableMetadataMutations_);
+        }
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.phoenix.coprocessor.generated.MetaDataProtos.internal_static_CreateTableRequest_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.phoenix.coprocessor.generated.MetaDataProtos.internal_static_CreateTableRequest_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest.class, org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<CreateTableRequest> PARSER =
+        new com.google.protobuf.AbstractParser<CreateTableRequest>() {
+      public CreateTableRequest parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new CreateTableRequest(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<CreateTableRequest> getParserForType() {
+      return PARSER;
+    }
+
+    // repeated bytes tableMetadataMutations = 1;
+    public static final int TABLEMETADATAMUTATIONS_FIELD_NUMBER = 1;
+    private java.util.List<com.google.protobuf.ByteString> tableMetadataMutations_;
+    /**
+     * <code>repeated bytes tableMetadataMutations = 1;</code>
+     */
+    public java.util.List<com.google.protobuf.ByteString>
+        getTableMetadataMutationsList() {
+      return tableMetadataMutations_;
+    }
+    /**
+     * <code>repeated bytes tableMetadataMutations = 1;</code>
+     */
+    public int getTableMetadataMutationsCount() {
+      return tableMetadataMutations_.size();
+    }
+    /**
+     * <code>repeated bytes tableMetadataMutations = 1;</code>
+     */
+    public com.google.protobuf.ByteString getTableMetadataMutations(int index) {
+      return tableMetadataMutations_.get(index);
+    }
+
+    private void initFields() {
+      tableMetadataMutations_ = java.util.Collections.emptyList();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      for (int i = 0; i < tableMetadataMutations_.size(); i++) {
+        output.writeBytes(1, tableMetadataMutations_.get(i));
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      {
+        int dataSize = 0;
+        for (int i = 0; i < tableMetadataMutations_.size(); i++) {
+          dataSize += com.google.protobuf.CodedOutputStream
+            .computeBytesSizeNoTag(tableMetadataMutations_.get(i));
+        }
+        size += dataSize;
+        size += 1 * getTableMetadataMutationsList().size();
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest)) {
+        return super.equals(obj);
+      }
+      org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest other = (org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest) obj;
+
+      boolean result = true;
+      result = result && getTableMetadataMutationsList()
+          .equals(other.getTableMetadataMutationsList());
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (getTableMetadataMutationsCount() > 0) {
+        hash = (37 * hash) + TABLEMETADATAMUTATIONS_FIELD_NUMBER;
+        hash = (53 * hash) + getTableMetadataMutationsList().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code CreateTableRequest}
+     *
+     * <pre>
+     * each byte array represents a MutationProto instance
+     * </pre>
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequestOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.phoenix.coprocessor.generated.MetaDataProtos.internal_static_CreateTableRequest_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.phoenix.coprocessor.generated.MetaDataProtos.internal_static_CreateTableRequest_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest.class, org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest.Builder.class);
+      }
+
+      // Construct using org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        tableMetadataMutations_ = java.util.Collections.emptyList();
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.phoenix.coprocessor.generated.MetaDataProtos.internal_static_CreateTableRequest_descriptor;
+      }
+
+      public org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest getDefaultInstanceForType() {
+        return org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest.getDefaultInstance();
+      }
+
+      public org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest build() {
+        org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest buildPartial() {
+        org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest result = new org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest(this);
+        int from_bitField0_ = bitField0_;
+        if (((bitField0_ & 0x00000001) == 0x00000001)) {
+          tableMetadataMutations_ = java.util.Collections.unmodifiableList(tableMetadataMutations_);
+          bitField0_ = (bitField0_ & ~0x00000001);
+        }
+        result.tableMetadataMutations_ = tableMetadataMutations_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest) {
+          return mergeFrom((org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest other) {
+        if (other == org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest.getDefaultInstance()) return this;
+        if (!other.tableMetadataMutations_.isEmpty()) {
+          if (tableMetadataMutations_.isEmpty()) {
+            tableMetadataMutations_ = other.tableMetadataMutations_;
+            bitField0_ = (bitField0_ & ~0x00000001);
+          } else {
+            ensureTableMetadataMutationsIsMutable();
+            tableMetadataMutations_.addAll(other.tableMetadataMutations_);
+          }
+          onChanged();
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // repeated bytes tableMetadataMutations = 1;
+      private java.util.List<com.google.protobuf.ByteString> tableMetadataMutations_ = java.util.Collections.emptyList();
+      private void ensureTableMetadataMutationsIsMutable() {
+        if (!((bitField0_ & 0x00000001) == 0x00000001)) {
+          tableMetadataMutations_ = new java.util.ArrayList<com.google.protobuf.ByteString>(tableMetadataMutations_);
+          bitField0_ |= 0x00000001;
+         }
+      }
+      /**
+       * <code>repeated bytes tableMetadataMutations = 1;</code>
+       */
+      public java.util.List<com.google.protobuf.ByteString>
+          getTableMetadataMutationsList() {
+        return java.util.Collections.unmodifiableList(tableMetadataMutations_);
+      }
+      /**
+       * <code>repeated bytes tableMetadataMutations = 1;</code>
+       */
+      public int getTableMetadataMutationsCount() {
+        return tableMetadataMutations_.size();
+      }
+      /**
+       * <code>repeated bytes tableMetadataMutations = 1;</code>
+       */
+      public com.google.protobuf.ByteString getTableMetadataMutations(int index) {
+        return tableMetadataMutations_.get(index);
+      }
+      /**
+       * <code>repeated bytes tableMetadataMutations = 1;</code>
+       */
+      public Builder setTableMetadataMutations(
+          int index, com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  ensureTableMetadataMutationsIsMutable();
+        tableMetadataMutations_.set(index, value);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated bytes tableMetadataMutations = 1;</code>
+       */
+      public Builder addTableMetadataMutations(com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  ensureTableMetadataMutationsIsMutable();
+        tableMetadataMutations_.add(value);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated bytes tableMetadataMutations = 1;</code>
+       */
+      public Builder addAllTableMetadataMutations(
+          java.lang.Iterable<? extends com.google.protobuf.ByteString> values) {
+        ensureTableMetadataMutationsIsMutable();
+        super.addAll(values, tableMetadataMutations_);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated bytes tableMetadataMutations = 1;</code>
+       */
+      public Builder clearTableMetadataMutations() {
+        tableMetadataMutations_ = java.util.Collections.emptyList();
+        bitField0_ = (bitField0_ & ~0x00000001);
+        onChanged();
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:CreateTableRequest)
+    }
+
+    static {
+      defaultInstance = new CreateTableRequest(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:CreateTableRequest)
+  }
+
+  public interface CreateFunctionRequestOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // repeated bytes tableMetadataMutations = 1;
+    /**
+     * <code>repeated bytes tableMetadataMutations = 1;</code>
+     */
+    java.util.List<com.google.protobuf.ByteString> getTableMetadataMutationsList();
+    /**
+     * <code>repeated bytes tableMetadataMutations = 1;</code>
+     */
+    int getTableMetadataMutationsCount();
+    /**
+     * <code>repeated bytes tableMetadataMutations = 1;</code>
+     */
+    com.google.protobuf.ByteString getTableMetadataMutations(int index);
+
+    // required bool temporary = 2;
+    /**
+     * <code>required bool temporary = 2;</code>
+     */
+    boolean hasTemporary();
+    /**
+     * <code>required bool temporary = 2;</code>
+     */
+    boolean getTemporary();
+
+    // optional bool replace = 3;
+    /**
+     * <code>optional bool replace = 3;</code>
+     */
+    boolean hasReplace();
+    /**
+     * <code>optional bool replace = 3;</code>
+     */
+    boolean getReplace();
+  }
+  /**
+   * Protobuf type {@code CreateFunctionRequest}
+   *
+   * <pre>
+   * each byte array represents a MutationProto instance
+   * </pre>
+   */
+  public static final class CreateFunctionRequest extends
+      com.google.protobuf.GeneratedMessage
+      implements CreateFunctionRequestOrBuilder {
+    // Use CreateFunctionRequest.newBuilder() to construct.
+    private CreateFunctionRequest(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private CreateFunctionRequest(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final CreateFunctionRequest defaultInstance;
+    public static CreateFunctionRequest getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public CreateFunctionRequest getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private CreateFunctionRequest(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+                tableMetadataMutations_ = new java.util.ArrayList<com.google.protobuf.ByteString>();
+                mutable_bitField0_ |= 0x00000001;
+              }
+              tableMetadataMutations_.add(input.readBytes());
+              break;
+            }
+            case 16: {
+              bitField0_ |= 0x00000001;
+              temporary_ = input.readBool();
+              break;
+            }
+            case 24: {
+              bitField0_ |= 0x00000002;
+              replace_ = input.readBool();
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+          tableMetadataMutations_ = java.util.Collections.unmodifiableList(tableMetadataMutations_);
+        }
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.phoenix.coprocessor.generated.MetaDataProtos.internal_static_CreateFunctionRequest_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.phoenix.coprocessor.generated.MetaDataProtos.internal_static_CreateFunctionRequest_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateFunctionRequest.class, org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateFunctionRequest.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<CreateFunctionRequest> PARSER =
+        new com.google.protobuf.AbstractParser<CreateFunctionRequest>() {
+      public CreateFunctionRequest parsePartialFrom(
+          com.go

<TRUNCATED>

[4/7] phoenix git commit: PHOENIX-538 Support UDFs(Rajeshbabu Chintaguntla)

Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PFunctionProtos.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PFunctionProtos.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PFunctionProtos.java
new file mode 100644
index 0000000..12927aa
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PFunctionProtos.java
@@ -0,0 +1,2942 @@
+// Generated by the protocol buffer compiler.  DO NOT EDIT!
+// source: PFunction.proto
+
+package org.apache.phoenix.coprocessor.generated;
+
+public final class PFunctionProtos {
+  private PFunctionProtos() {}
+  public static void registerAllExtensions(
+      com.google.protobuf.ExtensionRegistry registry) {
+  }
+  public interface PFunctionArgOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // required string argumentType = 1;
+    /**
+     * <code>required string argumentType = 1;</code>
+     */
+    boolean hasArgumentType();
+    /**
+     * <code>required string argumentType = 1;</code>
+     */
+    java.lang.String getArgumentType();
+    /**
+     * <code>required string argumentType = 1;</code>
+     */
+    com.google.protobuf.ByteString
+        getArgumentTypeBytes();
+
+    // optional bool isArrayType = 2;
+    /**
+     * <code>optional bool isArrayType = 2;</code>
+     */
+    boolean hasIsArrayType();
+    /**
+     * <code>optional bool isArrayType = 2;</code>
+     */
+    boolean getIsArrayType();
+
+    // optional bool isConstant = 3;
+    /**
+     * <code>optional bool isConstant = 3;</code>
+     */
+    boolean hasIsConstant();
+    /**
+     * <code>optional bool isConstant = 3;</code>
+     */
+    boolean getIsConstant();
+
+    // optional string defaultValue = 4;
+    /**
+     * <code>optional string defaultValue = 4;</code>
+     */
+    boolean hasDefaultValue();
+    /**
+     * <code>optional string defaultValue = 4;</code>
+     */
+    java.lang.String getDefaultValue();
+    /**
+     * <code>optional string defaultValue = 4;</code>
+     */
+    com.google.protobuf.ByteString
+        getDefaultValueBytes();
+
+    // optional string minValue = 5;
+    /**
+     * <code>optional string minValue = 5;</code>
+     */
+    boolean hasMinValue();
+    /**
+     * <code>optional string minValue = 5;</code>
+     */
+    java.lang.String getMinValue();
+    /**
+     * <code>optional string minValue = 5;</code>
+     */
+    com.google.protobuf.ByteString
+        getMinValueBytes();
+
+    // optional string maxValue = 6;
+    /**
+     * <code>optional string maxValue = 6;</code>
+     */
+    boolean hasMaxValue();
+    /**
+     * <code>optional string maxValue = 6;</code>
+     */
+    java.lang.String getMaxValue();
+    /**
+     * <code>optional string maxValue = 6;</code>
+     */
+    com.google.protobuf.ByteString
+        getMaxValueBytes();
+  }
+  /**
+   * Protobuf type {@code PFunctionArg}
+   */
+  public static final class PFunctionArg extends
+      com.google.protobuf.GeneratedMessage
+      implements PFunctionArgOrBuilder {
+    // Use PFunctionArg.newBuilder() to construct.
+    private PFunctionArg(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private PFunctionArg(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final PFunctionArg defaultInstance;
+    public static PFunctionArg getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public PFunctionArg getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private PFunctionArg(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              bitField0_ |= 0x00000001;
+              argumentType_ = input.readBytes();
+              break;
+            }
+            case 16: {
+              bitField0_ |= 0x00000002;
+              isArrayType_ = input.readBool();
+              break;
+            }
+            case 24: {
+              bitField0_ |= 0x00000004;
+              isConstant_ = input.readBool();
+              break;
+            }
+            case 34: {
+              bitField0_ |= 0x00000008;
+              defaultValue_ = input.readBytes();
+              break;
+            }
+            case 42: {
+              bitField0_ |= 0x00000010;
+              minValue_ = input.readBytes();
+              break;
+            }
+            case 50: {
+              bitField0_ |= 0x00000020;
+              maxValue_ = input.readBytes();
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.phoenix.coprocessor.generated.PFunctionProtos.internal_static_PFunctionArg_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.phoenix.coprocessor.generated.PFunctionProtos.internal_static_PFunctionArg_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg.class, org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<PFunctionArg> PARSER =
+        new com.google.protobuf.AbstractParser<PFunctionArg>() {
+      public PFunctionArg parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new PFunctionArg(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<PFunctionArg> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // required string argumentType = 1;
+    public static final int ARGUMENTTYPE_FIELD_NUMBER = 1;
+    private java.lang.Object argumentType_;
+    /**
+     * <code>required string argumentType = 1;</code>
+     */
+    public boolean hasArgumentType() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required string argumentType = 1;</code>
+     */
+    public java.lang.String getArgumentType() {
+      java.lang.Object ref = argumentType_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          argumentType_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>required string argumentType = 1;</code>
+     */
+    public com.google.protobuf.ByteString
+        getArgumentTypeBytes() {
+      java.lang.Object ref = argumentType_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        argumentType_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    // optional bool isArrayType = 2;
+    public static final int ISARRAYTYPE_FIELD_NUMBER = 2;
+    private boolean isArrayType_;
+    /**
+     * <code>optional bool isArrayType = 2;</code>
+     */
+    public boolean hasIsArrayType() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>optional bool isArrayType = 2;</code>
+     */
+    public boolean getIsArrayType() {
+      return isArrayType_;
+    }
+
+    // optional bool isConstant = 3;
+    public static final int ISCONSTANT_FIELD_NUMBER = 3;
+    private boolean isConstant_;
+    /**
+     * <code>optional bool isConstant = 3;</code>
+     */
+    public boolean hasIsConstant() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>optional bool isConstant = 3;</code>
+     */
+    public boolean getIsConstant() {
+      return isConstant_;
+    }
+
+    // optional string defaultValue = 4;
+    public static final int DEFAULTVALUE_FIELD_NUMBER = 4;
+    private java.lang.Object defaultValue_;
+    /**
+     * <code>optional string defaultValue = 4;</code>
+     */
+    public boolean hasDefaultValue() {
+      return ((bitField0_ & 0x00000008) == 0x00000008);
+    }
+    /**
+     * <code>optional string defaultValue = 4;</code>
+     */
+    public java.lang.String getDefaultValue() {
+      java.lang.Object ref = defaultValue_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          defaultValue_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>optional string defaultValue = 4;</code>
+     */
+    public com.google.protobuf.ByteString
+        getDefaultValueBytes() {
+      java.lang.Object ref = defaultValue_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        defaultValue_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    // optional string minValue = 5;
+    public static final int MINVALUE_FIELD_NUMBER = 5;
+    private java.lang.Object minValue_;
+    /**
+     * <code>optional string minValue = 5;</code>
+     */
+    public boolean hasMinValue() {
+      return ((bitField0_ & 0x00000010) == 0x00000010);
+    }
+    /**
+     * <code>optional string minValue = 5;</code>
+     */
+    public java.lang.String getMinValue() {
+      java.lang.Object ref = minValue_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          minValue_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>optional string minValue = 5;</code>
+     */
+    public com.google.protobuf.ByteString
+        getMinValueBytes() {
+      java.lang.Object ref = minValue_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        minValue_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    // optional string maxValue = 6;
+    public static final int MAXVALUE_FIELD_NUMBER = 6;
+    private java.lang.Object maxValue_;
+    /**
+     * <code>optional string maxValue = 6;</code>
+     */
+    public boolean hasMaxValue() {
+      return ((bitField0_ & 0x00000020) == 0x00000020);
+    }
+    /**
+     * <code>optional string maxValue = 6;</code>
+     */
+    public java.lang.String getMaxValue() {
+      java.lang.Object ref = maxValue_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          maxValue_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>optional string maxValue = 6;</code>
+     */
+    public com.google.protobuf.ByteString
+        getMaxValueBytes() {
+      java.lang.Object ref = maxValue_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        maxValue_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    private void initFields() {
+      argumentType_ = "";
+      isArrayType_ = false;
+      isConstant_ = false;
+      defaultValue_ = "";
+      minValue_ = "";
+      maxValue_ = "";
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      if (!hasArgumentType()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeBytes(1, getArgumentTypeBytes());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeBool(2, isArrayType_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeBool(3, isConstant_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        output.writeBytes(4, getDefaultValueBytes());
+      }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        output.writeBytes(5, getMinValueBytes());
+      }
+      if (((bitField0_ & 0x00000020) == 0x00000020)) {
+        output.writeBytes(6, getMaxValueBytes());
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(1, getArgumentTypeBytes());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(2, isArrayType_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(3, isConstant_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(4, getDefaultValueBytes());
+      }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(5, getMinValueBytes());
+      }
+      if (((bitField0_ & 0x00000020) == 0x00000020)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(6, getMaxValueBytes());
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg)) {
+        return super.equals(obj);
+      }
+      org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg other = (org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg) obj;
+
+      boolean result = true;
+      result = result && (hasArgumentType() == other.hasArgumentType());
+      if (hasArgumentType()) {
+        result = result && getArgumentType()
+            .equals(other.getArgumentType());
+      }
+      result = result && (hasIsArrayType() == other.hasIsArrayType());
+      if (hasIsArrayType()) {
+        result = result && (getIsArrayType()
+            == other.getIsArrayType());
+      }
+      result = result && (hasIsConstant() == other.hasIsConstant());
+      if (hasIsConstant()) {
+        result = result && (getIsConstant()
+            == other.getIsConstant());
+      }
+      result = result && (hasDefaultValue() == other.hasDefaultValue());
+      if (hasDefaultValue()) {
+        result = result && getDefaultValue()
+            .equals(other.getDefaultValue());
+      }
+      result = result && (hasMinValue() == other.hasMinValue());
+      if (hasMinValue()) {
+        result = result && getMinValue()
+            .equals(other.getMinValue());
+      }
+      result = result && (hasMaxValue() == other.hasMaxValue());
+      if (hasMaxValue()) {
+        result = result && getMaxValue()
+            .equals(other.getMaxValue());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasArgumentType()) {
+        hash = (37 * hash) + ARGUMENTTYPE_FIELD_NUMBER;
+        hash = (53 * hash) + getArgumentType().hashCode();
+      }
+      if (hasIsArrayType()) {
+        hash = (37 * hash) + ISARRAYTYPE_FIELD_NUMBER;
+        hash = (53 * hash) + hashBoolean(getIsArrayType());
+      }
+      if (hasIsConstant()) {
+        hash = (37 * hash) + ISCONSTANT_FIELD_NUMBER;
+        hash = (53 * hash) + hashBoolean(getIsConstant());
+      }
+      if (hasDefaultValue()) {
+        hash = (37 * hash) + DEFAULTVALUE_FIELD_NUMBER;
+        hash = (53 * hash) + getDefaultValue().hashCode();
+      }
+      if (hasMinValue()) {
+        hash = (37 * hash) + MINVALUE_FIELD_NUMBER;
+        hash = (53 * hash) + getMinValue().hashCode();
+      }
+      if (hasMaxValue()) {
+        hash = (37 * hash) + MAXVALUE_FIELD_NUMBER;
+        hash = (53 * hash) + getMaxValue().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code PFunctionArg}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArgOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.phoenix.coprocessor.generated.PFunctionProtos.internal_static_PFunctionArg_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.phoenix.coprocessor.generated.PFunctionProtos.internal_static_PFunctionArg_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg.class, org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg.Builder.class);
+      }
+
+      // Construct using org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        argumentType_ = "";
+        bitField0_ = (bitField0_ & ~0x00000001);
+        isArrayType_ = false;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        isConstant_ = false;
+        bitField0_ = (bitField0_ & ~0x00000004);
+        defaultValue_ = "";
+        bitField0_ = (bitField0_ & ~0x00000008);
+        minValue_ = "";
+        bitField0_ = (bitField0_ & ~0x00000010);
+        maxValue_ = "";
+        bitField0_ = (bitField0_ & ~0x00000020);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.phoenix.coprocessor.generated.PFunctionProtos.internal_static_PFunctionArg_descriptor;
+      }
+
+      public org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg getDefaultInstanceForType() {
+        return org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg.getDefaultInstance();
+      }
+
+      public org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg build() {
+        org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg buildPartial() {
+        org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg result = new org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.argumentType_ = argumentType_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.isArrayType_ = isArrayType_;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.isConstant_ = isConstant_;
+        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+          to_bitField0_ |= 0x00000008;
+        }
+        result.defaultValue_ = defaultValue_;
+        if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
+          to_bitField0_ |= 0x00000010;
+        }
+        result.minValue_ = minValue_;
+        if (((from_bitField0_ & 0x00000020) == 0x00000020)) {
+          to_bitField0_ |= 0x00000020;
+        }
+        result.maxValue_ = maxValue_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg) {
+          return mergeFrom((org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg other) {
+        if (other == org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg.getDefaultInstance()) return this;
+        if (other.hasArgumentType()) {
+          bitField0_ |= 0x00000001;
+          argumentType_ = other.argumentType_;
+          onChanged();
+        }
+        if (other.hasIsArrayType()) {
+          setIsArrayType(other.getIsArrayType());
+        }
+        if (other.hasIsConstant()) {
+          setIsConstant(other.getIsConstant());
+        }
+        if (other.hasDefaultValue()) {
+          bitField0_ |= 0x00000008;
+          defaultValue_ = other.defaultValue_;
+          onChanged();
+        }
+        if (other.hasMinValue()) {
+          bitField0_ |= 0x00000010;
+          minValue_ = other.minValue_;
+          onChanged();
+        }
+        if (other.hasMaxValue()) {
+          bitField0_ |= 0x00000020;
+          maxValue_ = other.maxValue_;
+          onChanged();
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (!hasArgumentType()) {
+          
+          return false;
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // required string argumentType = 1;
+      private java.lang.Object argumentType_ = "";
+      /**
+       * <code>required string argumentType = 1;</code>
+       */
+      public boolean hasArgumentType() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required string argumentType = 1;</code>
+       */
+      public java.lang.String getArgumentType() {
+        java.lang.Object ref = argumentType_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          argumentType_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>required string argumentType = 1;</code>
+       */
+      public com.google.protobuf.ByteString
+          getArgumentTypeBytes() {
+        java.lang.Object ref = argumentType_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          argumentType_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>required string argumentType = 1;</code>
+       */
+      public Builder setArgumentType(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        argumentType_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string argumentType = 1;</code>
+       */
+      public Builder clearArgumentType() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        argumentType_ = getDefaultInstance().getArgumentType();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string argumentType = 1;</code>
+       */
+      public Builder setArgumentTypeBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        argumentType_ = value;
+        onChanged();
+        return this;
+      }
+
+      // optional bool isArrayType = 2;
+      private boolean isArrayType_ ;
+      /**
+       * <code>optional bool isArrayType = 2;</code>
+       */
+      public boolean hasIsArrayType() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>optional bool isArrayType = 2;</code>
+       */
+      public boolean getIsArrayType() {
+        return isArrayType_;
+      }
+      /**
+       * <code>optional bool isArrayType = 2;</code>
+       */
+      public Builder setIsArrayType(boolean value) {
+        bitField0_ |= 0x00000002;
+        isArrayType_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bool isArrayType = 2;</code>
+       */
+      public Builder clearIsArrayType() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        isArrayType_ = false;
+        onChanged();
+        return this;
+      }
+
+      // optional bool isConstant = 3;
+      private boolean isConstant_ ;
+      /**
+       * <code>optional bool isConstant = 3;</code>
+       */
+      public boolean hasIsConstant() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      /**
+       * <code>optional bool isConstant = 3;</code>
+       */
+      public boolean getIsConstant() {
+        return isConstant_;
+      }
+      /**
+       * <code>optional bool isConstant = 3;</code>
+       */
+      public Builder setIsConstant(boolean value) {
+        bitField0_ |= 0x00000004;
+        isConstant_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bool isConstant = 3;</code>
+       */
+      public Builder clearIsConstant() {
+        bitField0_ = (bitField0_ & ~0x00000004);
+        isConstant_ = false;
+        onChanged();
+        return this;
+      }
+
+      // optional string defaultValue = 4;
+      private java.lang.Object defaultValue_ = "";
+      /**
+       * <code>optional string defaultValue = 4;</code>
+       */
+      public boolean hasDefaultValue() {
+        return ((bitField0_ & 0x00000008) == 0x00000008);
+      }
+      /**
+       * <code>optional string defaultValue = 4;</code>
+       */
+      public java.lang.String getDefaultValue() {
+        java.lang.Object ref = defaultValue_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          defaultValue_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>optional string defaultValue = 4;</code>
+       */
+      public com.google.protobuf.ByteString
+          getDefaultValueBytes() {
+        java.lang.Object ref = defaultValue_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          defaultValue_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>optional string defaultValue = 4;</code>
+       */
+      public Builder setDefaultValue(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000008;
+        defaultValue_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string defaultValue = 4;</code>
+       */
+      public Builder clearDefaultValue() {
+        bitField0_ = (bitField0_ & ~0x00000008);
+        defaultValue_ = getDefaultInstance().getDefaultValue();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string defaultValue = 4;</code>
+       */
+      public Builder setDefaultValueBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000008;
+        defaultValue_ = value;
+        onChanged();
+        return this;
+      }
+
+      // optional string minValue = 5;
+      private java.lang.Object minValue_ = "";
+      /**
+       * <code>optional string minValue = 5;</code>
+       */
+      public boolean hasMinValue() {
+        return ((bitField0_ & 0x00000010) == 0x00000010);
+      }
+      /**
+       * <code>optional string minValue = 5;</code>
+       */
+      public java.lang.String getMinValue() {
+        java.lang.Object ref = minValue_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          minValue_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>optional string minValue = 5;</code>
+       */
+      public com.google.protobuf.ByteString
+          getMinValueBytes() {
+        java.lang.Object ref = minValue_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          minValue_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>optional string minValue = 5;</code>
+       */
+      public Builder setMinValue(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000010;
+        minValue_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string minValue = 5;</code>
+       */
+      public Builder clearMinValue() {
+        bitField0_ = (bitField0_ & ~0x00000010);
+        minValue_ = getDefaultInstance().getMinValue();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string minValue = 5;</code>
+       */
+      public Builder setMinValueBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000010;
+        minValue_ = value;
+        onChanged();
+        return this;
+      }
+
+      // optional string maxValue = 6;
+      private java.lang.Object maxValue_ = "";
+      /**
+       * <code>optional string maxValue = 6;</code>
+       */
+      public boolean hasMaxValue() {
+        return ((bitField0_ & 0x00000020) == 0x00000020);
+      }
+      /**
+       * <code>optional string maxValue = 6;</code>
+       */
+      public java.lang.String getMaxValue() {
+        java.lang.Object ref = maxValue_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          maxValue_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>optional string maxValue = 6;</code>
+       */
+      public com.google.protobuf.ByteString
+          getMaxValueBytes() {
+        java.lang.Object ref = maxValue_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          maxValue_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>optional string maxValue = 6;</code>
+       */
+      public Builder setMaxValue(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000020;
+        maxValue_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string maxValue = 6;</code>
+       */
+      public Builder clearMaxValue() {
+        bitField0_ = (bitField0_ & ~0x00000020);
+        maxValue_ = getDefaultInstance().getMaxValue();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string maxValue = 6;</code>
+       */
+      public Builder setMaxValueBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000020;
+        maxValue_ = value;
+        onChanged();
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:PFunctionArg)
+    }
+
+    static {
+      defaultInstance = new PFunctionArg(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:PFunctionArg)
+  }
+
+  public interface PFunctionOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // required string functionName = 1;
+    /**
+     * <code>required string functionName = 1;</code>
+     */
+    boolean hasFunctionName();
+    /**
+     * <code>required string functionName = 1;</code>
+     */
+    java.lang.String getFunctionName();
+    /**
+     * <code>required string functionName = 1;</code>
+     */
+    com.google.protobuf.ByteString
+        getFunctionNameBytes();
+
+    // repeated .PFunctionArg arguments = 2;
+    /**
+     * <code>repeated .PFunctionArg arguments = 2;</code>
+     */
+    java.util.List<org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg> 
+        getArgumentsList();
+    /**
+     * <code>repeated .PFunctionArg arguments = 2;</code>
+     */
+    org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg getArguments(int index);
+    /**
+     * <code>repeated .PFunctionArg arguments = 2;</code>
+     */
+    int getArgumentsCount();
+    /**
+     * <code>repeated .PFunctionArg arguments = 2;</code>
+     */
+    java.util.List<? extends org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArgOrBuilder> 
+        getArgumentsOrBuilderList();
+    /**
+     * <code>repeated .PFunctionArg arguments = 2;</code>
+     */
+    org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArgOrBuilder getArgumentsOrBuilder(
+        int index);
+
+    // required string classname = 3;
+    /**
+     * <code>required string classname = 3;</code>
+     */
+    boolean hasClassname();
+    /**
+     * <code>required string classname = 3;</code>
+     */
+    java.lang.String getClassname();
+    /**
+     * <code>required string classname = 3;</code>
+     */
+    com.google.protobuf.ByteString
+        getClassnameBytes();
+
+    // required int64 timeStamp = 4;
+    /**
+     * <code>required int64 timeStamp = 4;</code>
+     */
+    boolean hasTimeStamp();
+    /**
+     * <code>required int64 timeStamp = 4;</code>
+     */
+    long getTimeStamp();
+
+    // optional string jarPath = 5;
+    /**
+     * <code>optional string jarPath = 5;</code>
+     */
+    boolean hasJarPath();
+    /**
+     * <code>optional string jarPath = 5;</code>
+     */
+    java.lang.String getJarPath();
+    /**
+     * <code>optional string jarPath = 5;</code>
+     */
+    com.google.protobuf.ByteString
+        getJarPathBytes();
+
+    // optional bytes tenantId = 6;
+    /**
+     * <code>optional bytes tenantId = 6;</code>
+     */
+    boolean hasTenantId();
+    /**
+     * <code>optional bytes tenantId = 6;</code>
+     */
+    com.google.protobuf.ByteString getTenantId();
+
+    // optional string returnType = 7;
+    /**
+     * <code>optional string returnType = 7;</code>
+     */
+    boolean hasReturnType();
+    /**
+     * <code>optional string returnType = 7;</code>
+     */
+    java.lang.String getReturnType();
+    /**
+     * <code>optional string returnType = 7;</code>
+     */
+    com.google.protobuf.ByteString
+        getReturnTypeBytes();
+
+    // optional bool isArrayReturnType = 8;
+    /**
+     * <code>optional bool isArrayReturnType = 8;</code>
+     */
+    boolean hasIsArrayReturnType();
+    /**
+     * <code>optional bool isArrayReturnType = 8;</code>
+     */
+    boolean getIsArrayReturnType();
+  }
+  /**
+   * Protobuf type {@code PFunction}
+   */
+  public static final class PFunction extends
+      com.google.protobuf.GeneratedMessage
+      implements PFunctionOrBuilder {
+    // Use PFunction.newBuilder() to construct.
+    private PFunction(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private PFunction(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final PFunction defaultInstance;
+    public static PFunction getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public PFunction getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private PFunction(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              bitField0_ |= 0x00000001;
+              functionName_ = input.readBytes();
+              break;
+            }
+            case 18: {
+              if (!((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
+                arguments_ = new java.util.ArrayList<org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg>();
+                mutable_bitField0_ |= 0x00000002;
+              }
+              arguments_.add(input.readMessage(org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg.PARSER, extensionRegistry));
+              break;
+            }
+            case 26: {
+              bitField0_ |= 0x00000002;
+              classname_ = input.readBytes();
+              break;
+            }
+            case 32: {
+              bitField0_ |= 0x00000004;
+              timeStamp_ = input.readInt64();
+              break;
+            }
+            case 42: {
+              bitField0_ |= 0x00000008;
+              jarPath_ = input.readBytes();
+              break;
+            }
+            case 50: {
+              bitField0_ |= 0x00000010;
+              tenantId_ = input.readBytes();
+              break;
+            }
+            case 58: {
+              bitField0_ |= 0x00000020;
+              returnType_ = input.readBytes();
+              break;
+            }
+            case 64: {
+              bitField0_ |= 0x00000040;
+              isArrayReturnType_ = input.readBool();
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
+          arguments_ = java.util.Collections.unmodifiableList(arguments_);
+        }
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.phoenix.coprocessor.generated.PFunctionProtos.internal_static_PFunction_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.phoenix.coprocessor.generated.PFunctionProtos.internal_static_PFunction_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction.class, org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<PFunction> PARSER =
+        new com.google.protobuf.AbstractParser<PFunction>() {
+      public PFunction parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new PFunction(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<PFunction> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // required string functionName = 1;
+    public static final int FUNCTIONNAME_FIELD_NUMBER = 1;
+    private java.lang.Object functionName_;
+    /**
+     * <code>required string functionName = 1;</code>
+     */
+    public boolean hasFunctionName() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required string functionName = 1;</code>
+     */
+    public java.lang.String getFunctionName() {
+      java.lang.Object ref = functionName_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          functionName_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>required string functionName = 1;</code>
+     */
+    public com.google.protobuf.ByteString
+        getFunctionNameBytes() {
+      java.lang.Object ref = functionName_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        functionName_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    // repeated .PFunctionArg arguments = 2;
+    public static final int ARGUMENTS_FIELD_NUMBER = 2;
+    private java.util.List<org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg> arguments_;
+    /**
+     * <code>repeated .PFunctionArg arguments = 2;</code>
+     */
+    public java.util.List<org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg> getArgumentsList() {
+      return arguments_;
+    }
+    /**
+     * <code>repeated .PFunctionArg arguments = 2;</code>
+     */
+    public java.util.List<? extends org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArgOrBuilder> 
+        getArgumentsOrBuilderList() {
+      return arguments_;
+    }
+    /**
+     * <code>repeated .PFunctionArg arguments = 2;</code>
+     */
+    public int getArgumentsCount() {
+      return arguments_.size();
+    }
+    /**
+     * <code>repeated .PFunctionArg arguments = 2;</code>
+     */
+    public org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg getArguments(int index) {
+      return arguments_.get(index);
+    }
+    /**
+     * <code>repeated .PFunctionArg arguments = 2;</code>
+     */
+    public org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArgOrBuilder getArgumentsOrBuilder(
+        int index) {
+      return arguments_.get(index);
+    }
+
+    // required string classname = 3;
+    public static final int CLASSNAME_FIELD_NUMBER = 3;
+    private java.lang.Object classname_;
+    /**
+     * <code>required string classname = 3;</code>
+     */
+    public boolean hasClassname() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>required string classname = 3;</code>
+     */
+    public java.lang.String getClassname() {
+      java.lang.Object ref = classname_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          classname_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>required string classname = 3;</code>
+     */
+    public com.google.protobuf.ByteString
+        getClassnameBytes() {
+      java.lang.Object ref = classname_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        classname_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    // required int64 timeStamp = 4;
+    public static final int TIMESTAMP_FIELD_NUMBER = 4;
+    private long timeStamp_;
+    /**
+     * <code>required int64 timeStamp = 4;</code>
+     */
+    public boolean hasTimeStamp() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>required int64 timeStamp = 4;</code>
+     */
+    public long getTimeStamp() {
+      return timeStamp_;
+    }
+
+    // optional string jarPath = 5;
+    public static final int JARPATH_FIELD_NUMBER = 5;
+    private java.lang.Object jarPath_;
+    /**
+     * <code>optional string jarPath = 5;</code>
+     */
+    public boolean hasJarPath() {
+      return ((bitField0_ & 0x00000008) == 0x00000008);
+    }
+    /**
+     * <code>optional string jarPath = 5;</code>
+     */
+    public java.lang.String getJarPath() {
+      java.lang.Object ref = jarPath_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          jarPath_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>optional string jarPath = 5;</code>
+     */
+    public com.google.protobuf.ByteString
+        getJarPathBytes() {
+      java.lang.Object ref = jarPath_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        jarPath_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    // optional bytes tenantId = 6;
+    public static final int TENANTID_FIELD_NUMBER = 6;
+    private com.google.protobuf.ByteString tenantId_;
+    /**
+     * <code>optional bytes tenantId = 6;</code>
+     */
+    public boolean hasTenantId() {
+      return ((bitField0_ & 0x00000010) == 0x00000010);
+    }
+    /**
+     * <code>optional bytes tenantId = 6;</code>
+     */
+    public com.google.protobuf.ByteString getTenantId() {
+      return tenantId_;
+    }
+
+    // optional string returnType = 7;
+    public static final int RETURNTYPE_FIELD_NUMBER = 7;
+    private java.lang.Object returnType_;
+    /**
+     * <code>optional string returnType = 7;</code>
+     */
+    public boolean hasReturnType() {
+      return ((bitField0_ & 0x00000020) == 0x00000020);
+    }
+    /**
+     * <code>optional string returnType = 7;</code>
+     */
+    public java.lang.String getReturnType() {
+      java.lang.Object ref = returnType_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          returnType_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>optional string returnType = 7;</code>
+     */
+    public com.google.protobuf.ByteString
+        getReturnTypeBytes() {
+      java.lang.Object ref = returnType_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        returnType_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    // optional bool isArrayReturnType = 8;
+    public static final int ISARRAYRETURNTYPE_FIELD_NUMBER = 8;
+    private boolean isArrayReturnType_;
+    /**
+     * <code>optional bool isArrayReturnType = 8;</code>
+     */
+    public boolean hasIsArrayReturnType() {
+      return ((bitField0_ & 0x00000040) == 0x00000040);
+    }
+    /**
+     * <code>optional bool isArrayReturnType = 8;</code>
+     */
+    public boolean getIsArrayReturnType() {
+      return isArrayReturnType_;
+    }
+
+    private void initFields() {
+      functionName_ = "";
+      arguments_ = java.util.Collections.emptyList();
+      classname_ = "";
+      timeStamp_ = 0L;
+      jarPath_ = "";
+      tenantId_ = com.google.protobuf.ByteString.EMPTY;
+      returnType_ = "";
+      isArrayReturnType_ = false;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      if (!hasFunctionName()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasClassname()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasTimeStamp()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      for (int i = 0; i < getArgumentsCount(); i++) {
+        if (!getArguments(i).isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeBytes(1, getFunctionNameBytes());
+      }
+      for (int i = 0; i < arguments_.size(); i++) {
+        output.writeMessage(2, arguments_.get(i));
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeBytes(3, getClassnameBytes());
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeInt64(4, timeStamp_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        output.writeBytes(5, getJarPathBytes());
+      }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        output.writeBytes(6, tenantId_);
+      }
+      if (((bitField0_ & 0x00000020) == 0x00000020)) {
+        output.writeBytes(7, getReturnTypeBytes());
+      }
+      if (((bitField0_ & 0x00000040) == 0x00000040)) {
+        output.writeBool(8, isArrayReturnType_);
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(1, getFunctionNameBytes());
+      }
+      for (int i = 0; i < arguments_.size(); i++) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(2, arguments_.get(i));
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(3, getClassnameBytes());
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt64Size(4, timeStamp_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(5, getJarPathBytes());
+      }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(6, tenantId_);
+      }
+      if (((bitField0_ & 0x00000020) == 0x00000020)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(7, getReturnTypeBytes());
+      }
+      if (((bitField0_ & 0x00000040) == 0x00000040)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(8, isArrayReturnType_);
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction)) {
+        return super.equals(obj);
+      }
+      org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction other = (org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction) obj;
+
+      boolean result = true;
+      result = result && (hasFunctionName() == other.hasFunctionName());
+      if (hasFunctionName()) {
+        result = result && getFunctionName()
+            .equals(other.getFunctionName());
+      }
+      result = result && getArgumentsList()
+          .equals(other.getArgumentsList());
+      result = result && (hasClassname() == other.hasClassname());
+      if (hasClassname()) {
+        result = result && getClassname()
+            .equals(other.getClassname());
+      }
+      result = result && (hasTimeStamp() == other.hasTimeStamp());
+      if (hasTimeStamp()) {
+        result = result && (getTimeStamp()
+            == other.getTimeStamp());
+      }
+      result = result && (hasJarPath() == other.hasJarPath());
+      if (hasJarPath()) {
+        result = result && getJarPath()
+            .equals(other.getJarPath());
+      }
+      result = result && (hasTenantId() == other.hasTenantId());
+      if (hasTenantId()) {
+        result = result && getTenantId()
+            .equals(other.getTenantId());
+      }
+      result = result && (hasReturnType() == other.hasReturnType());
+      if (hasReturnType()) {
+        result = result && getReturnType()
+            .equals(other.getReturnType());
+      }
+      result = result && (hasIsArrayReturnType() == other.hasIsArrayReturnType());
+      if (hasIsArrayReturnType()) {
+        result = result && (getIsArrayReturnType()
+            == other.getIsArrayReturnType());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasFunctionName()) {
+        hash = (37 * hash) + FUNCTIONNAME_FIELD_NUMBER;
+        hash = (53 * hash) + getFunctionName().hashCode();
+      }
+      if (getArgumentsCount() > 0) {
+        hash = (37 * hash) + ARGUMENTS_FIELD_NUMBER;
+        hash = (53 * hash) + getArgumentsList().hashCode();
+      }
+      if (hasClassname()) {
+        hash = (37 * hash) + CLASSNAME_FIELD_NUMBER;
+        hash = (53 * hash) + getClassname().hashCode();
+      }
+      if (hasTimeStamp()) {
+        hash = (37 * hash) + TIMESTAMP_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getTimeStamp());
+      }
+      if (hasJarPath()) {
+        hash = (37 * hash) + JARPATH_FIELD_NUMBER;
+        hash = (53 * hash) + getJarPath().hashCode();
+      }
+      if (hasTenantId()) {
+        hash = (37 * hash) + TENANTID_FIELD_NUMBER;
+        hash = (53 * hash) + getTenantId().hashCode();
+      }
+      if (hasReturnType()) {
+        hash = (37 * hash) + RETURNTYPE_FIELD_NUMBER;
+        hash = (53 * hash) + getReturnType().hashCode();
+      }
+      if (hasIsArrayReturnType()) {
+        hash = (37 * hash) + ISARRAYRETURNTYPE_FIELD_NUMBER;
+        hash = (53 * hash) + hashBoolean(getIsArrayReturnType());
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code PFunction}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.phoenix.coprocessor.generated.PFunctionProtos.internal_static_PFunction_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.phoenix.coprocessor.generated.PFunctionProtos.internal_static_PFunction_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction.class, org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction.Builder.class);
+      }
+
+      // Construct using org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getArgumentsFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        functionName_ = "";
+        bitField0_ = (bitField0_ & ~0x00000001);
+        if (argumentsBuilder_ == null) {
+          arguments_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000002);
+        } else {
+          argumentsBuilder_.clear();
+        }
+        classname_ = "";
+        bitField0_ = (bitField0_ & ~0x00000004);
+        timeStamp_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000008);
+        jarPath_ = "";
+        bitField0_ = (bitField0_ & ~0x00000010);
+        tenantId_ = com.google.protobuf.ByteString.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000020);
+        returnType_ = "";
+        bitField0_ = (bitField0_ & ~0x00000040);
+        isArrayReturnType_ = false;
+        bitField0_ = (bitField0_ & ~0x00000080);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.phoenix.coprocessor.generated.PFunctionProtos.internal_static_PFunction_descriptor;
+      }
+
+      public org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction getDefaultInstanceForType() {
+        return org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction.getDefaultInstance();
+      }
+
+      public org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction build() {
+        org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction buildPartial() {
+        org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction result = new org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.functionName_ = functionName_;
+        if (argumentsBuilder_ == null) {
+          if (((bitField0_ & 0x00000002) == 0x00000002)) {
+            arguments_ = java.util.Collections.unmodifiableList(arguments_);
+            bitField0_ = (bitField0_ & ~0x00000002);
+          }
+          result.arguments_ = arguments_;
+        } else {
+          result.arguments_ = argumentsBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.classname_ = classname_;
+        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.timeStamp_ = timeStamp_;
+        if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
+          to_bitField0_ |= 0x00000008;
+        }
+        result.jarPath_ = jarPath_;
+        if (((from_bitField0_ & 0x00000020) == 0x00000020)) {
+          to_bitField0_ |= 0x00000010;
+        }
+        result.tenantId_ = tenantId_;
+        if (((from_bitField0_ & 0x00000040) == 0x00000040)) {
+          to_bitField0_ |= 0x00000020;
+        }
+        result.returnType_ = returnType_;
+        if (((from_bitField0_ & 0x00000080) == 0x00000080)) {
+          to_bitField0_ |= 0x00000040;
+        }
+        result.isArrayReturnType_ = isArrayReturnType_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction) {
+          return mergeFrom((org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction other) {
+        if (other == org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction.getDefaultInstance()) return this;
+        if (other.hasFunctionName()) {
+          bitField0_ |= 0x00000001;
+          functionName_ = other.functionName_;
+          onChanged();
+        }
+        if (argumentsBuilder_ == null) {
+          if (!other.arguments_.isEmpty()) {
+            if (arguments_.isEmpty()) {
+              arguments_ = other.arguments_;
+              bitField0_ = (bitField0_ & ~0x00000002);
+            } else {
+              ensureArgumentsIsMutable();
+              arguments_.addAll(other.arguments_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.arguments_.isEmpty()) {
+            if (argumentsBuilder_.isEmpty()) {
+              argumentsBuilder_.dispose();
+              argumentsBuilder_ = null;
+              arguments_ = other.arguments_;
+              bitField0_ = (bitField0_ & ~0x00000002);
+              argumentsBuilder_ = 
+                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
+                   getArgumentsFieldBuilder() : null;
+            } else {
+              argumentsBuilder_.addAllMessages(other.arguments_);
+            }
+          }
+        }
+        if (other.hasClassname()) {
+          bitField0_ |= 0x00000004;
+          classname_ = other.classname_;
+          onChanged();
+        }
+        if (other.hasTimeStamp()) {
+          setTimeStamp(other.getTimeStamp());
+        }
+        if (other.hasJarPath()) {
+          bitField0_ |= 0x00000010;
+          jarPath_ = other.jarPath_;
+          onChanged();
+        }
+        if (other.hasTenantId()) {
+          setTenantId(other.getTenantId());
+        }
+        if (other.hasReturnType()) {
+          bitField0_ |= 0x00000040;
+          returnType_ = other.returnType_;
+          onChanged();
+        }
+        if (other.hasIsArrayReturnType()) {
+          setIsArrayReturnType(other.getIsArrayReturnType());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (!hasFunctionName()) {
+          
+          return false;
+        }
+        if (!hasClassname()) {
+          
+          return false;
+        }
+        if (!hasTimeStamp()) {
+          
+          return false;
+        }
+        for (int i = 0; i < getArgumentsCount(); i++) {
+          if (!getArguments(i).isInitialized()) {
+            
+            return false;
+          }
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // required string functionName = 1;
+      private java.lang.Object functionName_ = "";
+      /**
+       * <code>required string functionName = 1;</code>
+       */
+      public boolean hasFunctionName() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required string functionName = 1;</code>
+       */
+      public java.lang.String getFunctionName() {
+        java.lang.Object ref = functionName_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          functionName_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>required string functionName = 1;</code>
+       */
+      public com.google.protobuf.ByteString
+          getFunctionNameBytes() {
+        java.lang.Object ref = functionName_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          functionName_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>required string functionName = 1;</code>
+       */
+      public Builder setFunctionName(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        functionName_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string functionName = 1;</code>
+       */
+      public Builder clearFunctionName() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        functionName_ = getDefaultInstance().getFunctionName();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string functionName = 1;</code>
+       */
+      public Builder setFunctionNameBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        functionName_ = value;
+        onChanged();
+        return this;
+      }
+
+      // repeated .PFunctionArg arguments = 2;
+      private java.util.List<org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg> arguments_ =
+        java.util.Collections.emptyList();
+      private void ensureArgumentsIsMutable() {
+        if (!((bitField0_ & 0x00000002) == 0x00000002)) {
+          arguments_ = new java.util.ArrayList<org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg>(arguments_);
+          bitField0_ |= 0x00000002;
+         }
+      }
+
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg, org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg.Builder, org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArgOrBuilder> argumentsBuilder_;
+
+      /**
+       * <code>repeated .PFunctionArg arguments = 2;</code>
+       */
+      public java.util.List<org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg> getArgumentsList() {
+        if (argumentsBuilder_ == null) {
+          return java.util.Collections.unmodifiableList(arguments_);
+        } else {
+          return argumentsBuilder_.getMessageList();
+        }
+      }
+      /**
+       * <code>repeated .PFunctionArg arguments = 2;</code>
+       */
+      public int getArgumentsCount() {
+        if (argumentsBuilder_ == null) {
+          return arguments_.size();
+        } else {
+          return argumentsBuilder_.getCount();
+        }
+      }
+      /**
+       * <code>repeated .PFunctionArg arguments = 2;</code>
+       */
+      public org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg getArguments(int index) {
+        if (argumentsBuilder_ == null) {
+          return arguments_.get(index);
+        } else {
+          return argumentsBuilder_.getMessage(index);
+        }
+      }
+      /**
+       * <code>repeated .PFunctionArg arguments = 2;</code>
+       */
+      public Builder setArguments(
+          int index, org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg value) {
+        if (argumentsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureArgumentsIsMutable();
+          arguments_.set(index, value);
+          onChanged();
+        } else {
+          argumentsBuilder_.setMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .PFunctionArg arguments = 2;</code>
+       */
+      public Builder setArguments(
+          int index, org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg.Builder builderForValue) {
+        if (argumentsBuilder_ == null) {
+          ensureArgumentsIsMutable();
+          arguments_.set(index, builderForValue.build());
+          onChanged();
+        } else {
+          argumentsBuilder_.setMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .PFunctionArg arguments = 2;</code>
+       */
+      public Builder addArguments(org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg value) {
+        if (argumentsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureArgumentsIsMutable();
+          arguments_.add(value);
+          onChanged();
+        } else {
+          argumentsBuilder_.addMessage(value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .PFunctionArg arguments = 2;</code>
+       */
+      public Builder addArguments(
+          int index, org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg value) {
+        if (argumentsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureArgumentsIsMutable();
+          arguments_.add(index, value);
+          onChanged();
+        } else {
+          argumentsBuilder_.addMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .PFunctionArg arguments = 2;</code>
+       */
+      public Builder addArguments(
+          org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg.Builder builderForValue) {
+        if (argumentsBuilder_ == null) {
+          ensureArgumentsIsMutable();
+          arguments_.add(builderForValue.build());
+          onChanged();
+        } else {
+          argumentsBuilder_.addMessage(builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .PFunctionArg arguments = 2;</code>
+       */
+      public Builder addArguments(
+          int index, org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg.Builder builderForValue) {
+        if (argumentsBuilder_ == null) {
+          ensureArgumentsIsMutable();
+          arguments_.add(index, builderForValue.build());
+          onChanged();
+        } else {
+          argumentsBuilder_.addMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .PFunctionArg arguments = 2;</code>
+       */
+      public Builder addAllArguments(
+          java.lang.Iterable<? extends org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg> values) {
+        if (argumentsBuilder_ == null) {
+          ensureArgumentsIsMutable();
+          super.addAll(values, arguments_);
+          onChanged();
+        } else {
+          argumentsBuilder_.addAllMessages(values);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .PFunctionArg arguments = 2;</code>
+       */
+      public Builder clearArguments() {
+        if (argumentsBuilder_ == null) {
+          arguments_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000002);
+          onChanged();
+        } else {
+          argumentsBuilder_.clear();
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .PFunctionArg arguments = 2;</code>
+       */
+      public Builder removeArguments(int index) {
+        if (argumentsBuilder_ == null) {
+          ensureArgumentsIsMutable();
+          arguments_.remove(index);
+          onChanged();
+        } else {
+          argumentsBuilder_.remove(index);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .PFunctionArg arguments = 2;</code>
+       */
+      public org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg.Builder getArgumentsBuilder(
+          int index) {
+        return getArgumentsFieldBuilder().getBuilder(index);
+      }
+      /**
+       * <code>repeated .PFunctionArg arguments = 2;</code>
+       */
+      public org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArgOrBuilder getArgumentsOrBuilder(
+          int index) {
+        if (argumentsBuilder_ == null) {
+          return arguments_.get(index);  } else {
+          return argumentsBuilder_.getMessageOrBuilder(index);
+        }
+      }
+      /**
+       * <code>repeated .PFunctionArg arguments = 2;</code>
+       */
+      public java.util.List<? extends org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArgOrBuilder> 
+           getArgumentsOrBuilderList() {
+        if (argumentsBuilder_ != null) {
+          return argumentsBuilder_.getMessageOrBuilderList();
+        } else {
+          return java.util.Collections.unmodifiableList(arguments_);
+        }
+      }
+      /**
+       * <code>repeated .PFunctionArg arguments = 2;</code>
+       */
+      public org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg.Builder addArgumentsBuilder() {
+        return getArgumentsFieldBuilder().addBuilder(
+            org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .PFunctionArg arguments = 2;</code>
+       */
+      public org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg.Builder addArgumentsBuilder(
+          int index) {
+        return getArgumentsFieldBuilder().addBuilder(
+            index, org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .PFunctionArg arguments = 2;</code>
+       */
+      public java.util.List<org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg.Builder> 
+           getArgumentsBuilderList() {
+        return getArgumentsFieldBuilder().getBuilderList();
+      }
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg, org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg.Builder, org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArgOrBuilder> 
+          getArgumentsFieldBuilder() {
+        if (argumentsBuilder_ == null) {
+ 

<TRUNCATED>

[7/7] phoenix git commit: PHOENIX-538 Support UDFs(Rajeshbabu Chintaguntla)

Posted by ra...@apache.org.
PHOENIX-538 Support UDFs(Rajeshbabu Chintaguntla)


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

Branch: refs/heads/master
Commit: 66bd3e35c0d2105dcc393116f8bb5851ce1f5ec4
Parents: cd29be2
Author: Rajeshbabu Chintaguntla <ra...@apache.org>
Authored: Mon Apr 27 14:03:44 2015 +0530
Committer: Rajeshbabu Chintaguntla <ra...@apache.org>
Committed: Mon Apr 27 14:03:44 2015 +0530

----------------------------------------------------------------------
 bin/phoenix_utils.py                            |    9 +
 bin/sqlline.py                                  |    2 +-
 .../end2end/QueryDatabaseMetaDataIT.java        |    5 +
 .../end2end/TenantSpecificTablesDDLIT.java      |    5 +
 .../phoenix/end2end/UserDefinedFunctionsIT.java |  605 +++
 phoenix-core/src/main/antlr3/PhoenixSQL.g       |   77 +-
 .../org/apache/phoenix/cache/GlobalCache.java   |   30 +-
 .../apache/phoenix/compile/ColumnResolver.java  |   17 +
 .../phoenix/compile/CreateFunctionCompiler.java |   80 +
 .../phoenix/compile/CreateIndexCompiler.java    |    2 +-
 .../apache/phoenix/compile/DeleteCompiler.java  |    2 +-
 .../phoenix/compile/ExpressionCompiler.java     |   17 +-
 .../apache/phoenix/compile/FromCompiler.java    |  199 +-
 .../apache/phoenix/compile/JoinCompiler.java    |    9 +-
 .../apache/phoenix/compile/PostDDLCompiler.java |   14 +
 .../phoenix/compile/ProjectionCompiler.java     |    2 +-
 .../apache/phoenix/compile/QueryCompiler.java   |   18 +-
 .../apache/phoenix/compile/RowProjector.java    |   32 +-
 .../phoenix/compile/StatementNormalizer.java    |    5 +-
 .../phoenix/compile/SubqueryRewriter.java       |    4 +-
 .../phoenix/compile/SubselectRewriter.java      |    2 +-
 .../coprocessor/MetaDataEndpointImpl.java       |  651 ++-
 .../phoenix/coprocessor/MetaDataProtocol.java   |   30 +-
 .../coprocessor/generated/MetaDataProtos.java   | 4274 +++++++++++++++---
 .../coprocessor/generated/PFunctionProtos.java  | 2942 ++++++++++++
 .../phoenix/exception/SQLExceptionCode.java     |   20 +-
 .../phoenix/exception/SQLExceptionInfo.java     |   16 +
 .../phoenix/expression/ExpressionType.java      |    4 +-
 .../expression/function/ScalarFunction.java     |    2 +-
 .../expression/function/UDFExpression.java      |  220 +
 .../visitor/CloneExpressionVisitor.java         |    6 +
 .../apache/phoenix/index/IndexMaintainer.java   |   50 +-
 .../apache/phoenix/jdbc/PhoenixConnection.java  |   35 +-
 .../phoenix/jdbc/PhoenixDatabaseMetaData.java   |   30 +
 .../apache/phoenix/jdbc/PhoenixStatement.java   |  132 +-
 .../apache/phoenix/optimize/QueryOptimizer.java |    4 +-
 .../phoenix/parse/CreateFunctionStatement.java  |   42 +
 .../phoenix/parse/CreateIndexStatement.java     |    8 +-
 .../org/apache/phoenix/parse/DMLStatement.java  |   11 +-
 .../apache/phoenix/parse/DeleteStatement.java   |    5 +-
 .../phoenix/parse/DropFunctionStatement.java    |   41 +
 .../apache/phoenix/parse/FunctionParseNode.java |   75 +-
 .../parse/IndexExpressionParseNodeRewriter.java |    4 +-
 .../org/apache/phoenix/parse/NamedNode.java     |    2 +-
 .../org/apache/phoenix/parse/PFunction.java     |  255 ++
 .../apache/phoenix/parse/ParseNodeFactory.java  |   70 +-
 .../apache/phoenix/parse/ParseNodeRewriter.java |    2 +-
 .../apache/phoenix/parse/SelectStatement.java   |   22 +-
 .../org/apache/phoenix/parse/UDFParseNode.java  |   27 +
 .../apache/phoenix/parse/UpsertStatement.java   |    9 +-
 .../apache/phoenix/protobuf/ProtobufUtil.java   |   10 +
 .../phoenix/query/ConnectionQueryServices.java  |    4 +
 .../query/ConnectionQueryServicesImpl.java      |  161 +-
 .../query/ConnectionlessQueryServicesImpl.java  |   52 +-
 .../query/DelegateConnectionQueryServices.java  |   31 +
 .../apache/phoenix/query/MetaDataMutated.java   |    3 +
 .../apache/phoenix/query/QueryConstants.java    |   36 +
 .../org/apache/phoenix/query/QueryServices.java |    2 +
 .../phoenix/query/QueryServicesOptions.java     |    4 +-
 .../schema/FunctionAlreadyExistsException.java  |   58 +
 .../schema/FunctionNotFoundException.java       |   52 +
 .../apache/phoenix/schema/MetaDataClient.java   |  256 +-
 .../NewerFunctionAlreadyExistsException.java    |   39 +
 .../org/apache/phoenix/schema/PMetaData.java    |    6 +-
 .../apache/phoenix/schema/PMetaDataEntity.java  |   22 +
 .../apache/phoenix/schema/PMetaDataImpl.java    |  118 +-
 .../java/org/apache/phoenix/schema/PTable.java  |    3 +-
 .../org/apache/phoenix/util/MetaDataUtil.java   |    7 +-
 .../org/apache/phoenix/util/SchemaUtil.java     |   17 +-
 .../apache/phoenix/parse/QueryParserTest.java   |   18 -
 .../query/ParallelIteratorsSplitTest.java       |   15 +
 phoenix-protocol/src/main/MetaDataService.proto |   37 +-
 phoenix-protocol/src/main/PFunction.proto       |   45 +
 73 files changed, 10222 insertions(+), 899 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/bin/phoenix_utils.py
----------------------------------------------------------------------
diff --git a/bin/phoenix_utils.py b/bin/phoenix_utils.py
index 36c7b82..2cf7db7 100755
--- a/bin/phoenix_utils.py
+++ b/bin/phoenix_utils.py
@@ -64,6 +64,15 @@ def setPath():
  phoenix_client_jar = find("phoenix-*-client.jar", phoenix_jar_path)
  global phoenix_test_jar_path
  phoenix_test_jar_path = os.path.join(current_dir, "..", "phoenix-core", "target","*")
+ global hadoop_common_jar_path
+ hadoop_common_jar_path = os.path.join(current_dir, "..", "phoenix-assembly", "target","*")
+ global hadoop_common_jar
+ hadoop_common_jar = find("hadoop-common*.jar", hadoop_common_jar_path)
+ global hadoop_hdfs_jar_path
+ hadoop_hdfs_jar_path = os.path.join(current_dir, "..", "phoenix-assembly", "target","*")
+ global hadoop_hdfs_jar
+ hadoop_hdfs_jar = find("hadoop-hdfs*.jar", hadoop_hdfs_jar_path)
+
  global hbase_conf_dir
  hbase_conf_dir = os.getenv('HBASE_CONF_DIR', os.getenv('HBASE_CONF_PATH', '.'))
  global hbase_conf_path # keep conf_path around for backward compatibility

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/bin/sqlline.py
----------------------------------------------------------------------
diff --git a/bin/sqlline.py b/bin/sqlline.py
index 6e5b5fa..80b5ff7 100755
--- a/bin/sqlline.py
+++ b/bin/sqlline.py
@@ -53,7 +53,7 @@ colorSetting = "true"
 if os.name == 'nt':
     colorSetting = "false"
 
-java_cmd = 'java -cp "' + phoenix_utils.hbase_conf_dir + os.pathsep + phoenix_utils.phoenix_client_jar + \
+java_cmd = 'java -cp "' + phoenix_utils.hbase_conf_dir + os.pathsep + phoenix_utils.phoenix_client_jar + os.pathsep + phoenix_utils.hadoop_common_jar + os.pathsep + phoenix_utils.hadoop_hdfs_jar + \
     '" -Dlog4j.configuration=file:' + \
     os.path.join(phoenix_utils.current_dir, "log4j.properties") + \
     " sqlline.SqlLine -d org.apache.phoenix.jdbc.PhoenixDriver \

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java
index c9ec0ce..61459a5 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java
@@ -19,6 +19,7 @@ package org.apache.phoenix.end2end;
 
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_FUNCTION_TABLE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TYPE_SEQUENCE;
 import static org.apache.phoenix.util.TestUtil.ATABLE_NAME;
 import static org.apache.phoenix.util.TestUtil.ATABLE_SCHEMA_NAME;
@@ -125,6 +126,10 @@ public class QueryDatabaseMetaDataIT extends BaseClientManagedTimeIT {
         assertEquals(PTableType.SYSTEM.toString(), rs.getString("TABLE_TYPE"));
         assertTrue(rs.next());
         assertEquals(rs.getString("TABLE_SCHEM"),SYSTEM_CATALOG_SCHEMA);
+        assertEquals(rs.getString("TABLE_NAME"),SYSTEM_FUNCTION_TABLE);
+        assertEquals(PTableType.SYSTEM.toString(), rs.getString("TABLE_TYPE"));
+        assertTrue(rs.next());
+        assertEquals(rs.getString("TABLE_SCHEM"),SYSTEM_CATALOG_SCHEMA);
         assertEquals(rs.getString("TABLE_NAME"),TYPE_SEQUENCE);
         assertEquals(PTableType.SYSTEM.toString(), rs.getString("TABLE_TYPE"));
         assertTrue(rs.next());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java
index 4d0b45d..a7c7291 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java
@@ -26,6 +26,7 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.KEY_SEQ;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ORDINAL_POSITION;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_FUNCTION_TABLE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TYPE_SEQUENCE;
 import static org.apache.phoenix.schema.PTableType.SYSTEM;
 import static org.apache.phoenix.schema.PTableType.TABLE;
@@ -473,6 +474,8 @@ public class TenantSpecificTablesDDLIT extends BaseTenantSpecificTablesIT {
             assertTrue(rs.next());
             assertTableMetaData(rs, SYSTEM_CATALOG_SCHEMA, SYSTEM_CATALOG_TABLE, SYSTEM);
             assertTrue(rs.next());
+            assertTableMetaData(rs, SYSTEM_CATALOG_SCHEMA, SYSTEM_FUNCTION_TABLE, SYSTEM);
+            assertTrue(rs.next());
             assertTableMetaData(rs, SYSTEM_CATALOG_SCHEMA, TYPE_SEQUENCE, SYSTEM);
             assertTrue(rs.next());
             assertTableMetaData(rs, SYSTEM_CATALOG_SCHEMA, PhoenixDatabaseMetaData.SYSTEM_STATS_TABLE, SYSTEM);
@@ -539,6 +542,8 @@ public class TenantSpecificTablesDDLIT extends BaseTenantSpecificTablesIT {
             assertTrue(rs.next());
             assertTableMetaData(rs, PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA, PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE, PTableType.SYSTEM);
             assertTrue(rs.next());
+            assertTableMetaData(rs, SYSTEM_CATALOG_SCHEMA, SYSTEM_FUNCTION_TABLE, SYSTEM);
+            assertTrue(rs.next());
             assertTableMetaData(rs, PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA, PhoenixDatabaseMetaData.TYPE_SEQUENCE, PTableType.SYSTEM);
             assertTrue(rs.next());
             assertTableMetaData(rs, SYSTEM_CATALOG_SCHEMA, PhoenixDatabaseMetaData.SYSTEM_STATS_TABLE, PTableType.SYSTEM);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/phoenix-core/src/it/java/org/apache/phoenix/end2end/UserDefinedFunctionsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UserDefinedFunctionsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UserDefinedFunctionsIT.java
new file mode 100644
index 0000000..d56004b
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UserDefinedFunctionsIT.java
@@ -0,0 +1,605 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.end2end;
+
+import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL;
+import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR;
+import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL_TERMINATOR;
+import static org.apache.phoenix.util.PhoenixRuntime.PHOENIX_TEST_DRIVER_URL_PARAM;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_FUNCTION_TABLE;
+import static org.apache.phoenix.query.QueryServices.DYNAMIC_JARS_DIR_KEY;
+import static org.apache.phoenix.util.TestUtil.LOCALHOST;
+import static org.junit.Assert.*;
+
+import java.io.BufferedInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.Statement;
+import java.util.Map;
+import java.util.Properties;
+import java.util.jar.Attributes;
+import java.util.jar.JarEntry;
+import java.util.jar.JarOutputStream;
+import java.util.jar.Manifest;
+
+import javax.tools.JavaCompiler;
+import javax.tools.ToolProvider;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.phoenix.expression.function.UDFExpression;
+import org.apache.phoenix.jdbc.PhoenixTestDriver;
+import org.apache.phoenix.query.BaseTest;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.schema.FunctionAlreadyExistsException;
+import org.apache.phoenix.schema.FunctionNotFoundException;
+import org.apache.phoenix.schema.ValueRangeExcpetion;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.QueryUtil;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.google.common.collect.Maps;
+
+public class UserDefinedFunctionsIT extends BaseTest{
+    
+    protected static final String TENANT_ID = "ZZTop";
+    private static String url;
+    private static PhoenixTestDriver driver;
+    private static HBaseTestingUtility util;
+
+    private static String STRING_REVERSE_EVALUATE_METHOD =
+            new StringBuffer()
+                    .append("    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {\n")
+                    .append("        Expression arg = getChildren().get(0);\n")
+                    .append("        if (!arg.evaluate(tuple, ptr)) {\n")
+                    .append("           return false;\n")
+                    .append("       }\n")
+                    .append("       int targetOffset = ptr.getLength();\n")
+                    .append("       if (targetOffset == 0) {\n")
+                    .append("            return true;\n")
+                    .append("        }\n")
+                    .append("        byte[] source = ptr.get();\n")
+                    .append("        byte[] target = new byte[targetOffset];\n")
+                    .append("        int sourceOffset = ptr.getOffset(); \n")
+                    .append("        int endOffset = sourceOffset + ptr.getLength();\n")
+                    .append("        SortOrder sortOrder = arg.getSortOrder();\n")
+                    .append("        while (sourceOffset < endOffset) {\n")
+                    .append("            int nBytes = StringUtil.getBytesInChar(source[sourceOffset], sortOrder);\n")
+                    .append("            targetOffset -= nBytes;\n")
+                    .append("            System.arraycopy(source, sourceOffset, target, targetOffset, nBytes);\n")
+                    .append("            sourceOffset += nBytes;\n")
+                    .append("        }\n")
+                    .append("        ptr.set(target);\n")
+                    .append("        return true;\n")
+                    .append("    }\n").toString();
+
+    private static String SUM_COLUMN_VALUES_EVALUATE_METHOD =
+            new StringBuffer()
+                    .append("    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {\n")
+                    .append("        int[] array = new int[getChildren().size()];\n")
+                    .append("        int i = 0;\n")
+                    .append("        for(Expression child:getChildren()) {\n")
+                    .append("            if (!child.evaluate(tuple, ptr)) {\n")
+                    .append("                return false;\n")
+                    .append("            }\n")
+                    .append("            int targetOffset = ptr.getLength();\n")
+                    .append("            if (targetOffset == 0) {\n")
+                    .append("                return true;\n")
+                    .append("            }\n")
+                    .append("            array[i++] = (Integer) PInteger.INSTANCE.toObject(ptr);\n")
+                    .append("        }\n")
+                    .append("        int sum = 0;\n")
+                    .append("        for(i=0;i<getChildren().size();i++) {\n")
+                    .append("            sum+=array[i];\n")
+                    .append("        }\n")
+                    .append("        ptr.set(PInteger.INSTANCE.toBytes((Integer)sum));\n")
+                    .append("        return true;\n")
+                    .append("    }\n").toString();
+
+    private static String MY_REVERSE_CLASS_NAME = "MyReverse";
+    private static String MY_SUM_CLASS_NAME = "MySum";
+    private static String MY_REVERSE_PROGRAM = getProgram(MY_REVERSE_CLASS_NAME, STRING_REVERSE_EVALUATE_METHOD, "PVarchar");
+    private static String MY_SUM_PROGRAM = getProgram(MY_SUM_CLASS_NAME, SUM_COLUMN_VALUES_EVALUATE_METHOD, "PInteger");
+    private static Properties EMPTY_PROPS = new Properties();
+    
+
+    private static String getProgram(String className, String evaluateMethod, String returnType) {
+        return new StringBuffer()
+                .append("package org.apache.phoenix.end2end;\n")
+                .append("import java.sql.SQLException;\n")
+                .append("import java.sql.SQLException;\n")
+                .append("import java.util.List;\n")
+                .append("import org.apache.hadoop.hbase.io.ImmutableBytesWritable;\n")
+                .append("import org.apache.phoenix.expression.Expression;\n")
+                .append("import org.apache.phoenix.expression.function.ScalarFunction;\n")
+                .append("import org.apache.phoenix.schema.SortOrder;\n")
+                .append("import org.apache.phoenix.schema.tuple.Tuple;\n")
+                .append("import org.apache.phoenix.schema.types.PDataType;\n")
+                .append("import org.apache.phoenix.schema.types.PInteger;\n")
+                .append("import org.apache.phoenix.schema.types.PVarchar;\n")
+                .append("import org.apache.phoenix.util.StringUtil;\n")
+                .append("public class "+className+" extends ScalarFunction{\n")
+                .append("    public static final String NAME = \"MY_REVERSE\";\n")
+                .append("    public "+className+"() {\n")
+                .append("    }\n")
+                .append("    public "+className+"(List<Expression> children) throws SQLException {\n")
+                .append("        super(children);\n")
+                .append("    }\n")
+                .append("    @Override\n")
+                .append(evaluateMethod)
+                .append("    @Override\n")
+                .append("    public SortOrder getSortOrder() {\n")
+                .append("        return getChildren().get(0).getSortOrder();\n")
+                .append("    }\n")
+                .append("  @Override\n")
+                .append("   public PDataType getDataType() {\n")
+                .append("       return "+returnType+".INSTANCE;\n")
+                .append("    }\n")
+                .append("    @Override\n")
+                .append("    public String getName() {\n")
+                .append("        return NAME;\n")
+                .append("    }\n")
+                .append("}\n").toString();
+    }
+
+    @BeforeClass
+    public static void doSetup() throws Exception {
+        Configuration conf = HBaseConfiguration.create();
+        setUpConfigForMiniCluster(conf);
+        util = new HBaseTestingUtility(conf);
+        util.startMiniDFSCluster(1);
+        util.startMiniZKCluster(1);
+        String string = util.getConfiguration().get("fs.defaultFS");
+        conf.set(DYNAMIC_JARS_DIR_KEY, string+"/hbase/tmpjars");
+        util.startMiniHBaseCluster(1, 1);
+        UDFExpression.setConfig(conf);
+        compileTestClass(MY_REVERSE_CLASS_NAME, MY_REVERSE_PROGRAM, 1);
+        compileTestClass(MY_SUM_CLASS_NAME, MY_SUM_PROGRAM, 2);
+        String clientPort = util.getConfiguration().get(QueryServices.ZOOKEEPER_PORT_ATTRIB);
+        url =
+                JDBC_PROTOCOL + JDBC_PROTOCOL_SEPARATOR + LOCALHOST + JDBC_PROTOCOL_SEPARATOR
+                        + clientPort + JDBC_PROTOCOL_TERMINATOR + PHOENIX_TEST_DRIVER_URL_PARAM;
+        Map<String, String> props = Maps.newHashMapWithExpectedSize(1);
+        props.put(QueryServices.ALLOW_USER_DEFINED_FUNCTIONS_ATTRIB, "true");
+        driver = initAndRegisterDriver(url, new ReadOnlyProps(props.entrySet().iterator()));
+    }
+    
+    @Test
+    public void testCreateFunction() throws Exception {
+        Connection conn = driver.connect(url, EMPTY_PROPS);
+        Statement stmt = conn.createStatement();
+        conn.createStatement().execute("create table t(k integer primary key, firstname varchar, lastname varchar)");
+        stmt.execute("upsert into t values(1,'foo','jock')");
+        conn.commit();
+        stmt.execute("create function myreverse(VARCHAR) returns VARCHAR as 'org.apache.phoenix.end2end."+MY_REVERSE_CLASS_NAME+"' using jar "
+                + "'"+util.getConfiguration().get(DYNAMIC_JARS_DIR_KEY) + "/myjar1.jar"+"'");
+        ResultSet rs = stmt.executeQuery("select myreverse(firstname) from t");
+        assertTrue(rs.next());
+        assertEquals("oof", rs.getString(1));
+        assertFalse(rs.next());
+        rs = stmt.executeQuery("select * from t where myreverse(firstname)='oof'");
+        assertTrue(rs.next());
+        assertEquals(1, rs.getInt(1));
+        assertEquals("foo", rs.getString(2));        
+        assertEquals("jock", rs.getString(3));
+        assertFalse(rs.next());
+        
+        try {
+            stmt.execute("create function myreverse(VARCHAR) returns VARCHAR as 'org.apache.phoenix.end2end."+MY_REVERSE_CLASS_NAME+"' using jar "
+                    + "'"+util.getConfiguration().get(DYNAMIC_JARS_DIR_KEY) + "/myjar1.jar"+"'");
+            fail("Duplicate function should not be created.");
+        } catch(FunctionAlreadyExistsException e) {
+        }
+        // without specifying the jar should pick the class from path of hbase.dynamic.jars.dir configuration. 
+        stmt.execute("create function myreverse2(VARCHAR) returns VARCHAR as 'org.apache.phoenix.end2end."+MY_REVERSE_CLASS_NAME+"'");
+        rs = stmt.executeQuery("select myreverse2(firstname) from t");
+        assertTrue(rs.next());
+        assertEquals("oof", rs.getString(1));        
+        assertFalse(rs.next());
+        rs = stmt.executeQuery("select * from t where myreverse2(firstname)='oof'");
+        assertTrue(rs.next());
+        assertEquals(1, rs.getInt(1));
+        assertEquals("foo", rs.getString(2));        
+        assertEquals("jock", rs.getString(3));
+        assertFalse(rs.next());
+        conn.createStatement().execute("create table t3(tenant_id varchar not null, k integer not null, firstname varchar, lastname varchar constraint pk primary key(tenant_id,k)) MULTI_TENANT=true");
+        // Function created with global id should be accessible.
+        Connection conn2 = driver.connect(url+";"+PhoenixRuntime.TENANT_ID_ATTRIB+"="+TENANT_ID, EMPTY_PROPS);
+        try {
+            conn2.createStatement().execute("upsert into t3 values(1,'foo','jock')");
+            conn2.commit();
+            conn2.createStatement().execute("create function myreverse(VARCHAR) returns VARCHAR as 'org.apache.phoenix.end2end."+MY_REVERSE_CLASS_NAME+"' using jar "
+                    + "'"+util.getConfiguration().get(DYNAMIC_JARS_DIR_KEY) + "/myjar1.jar"+"'");
+            rs = conn2.createStatement().executeQuery("select myreverse(firstname) from t3");
+            assertTrue(rs.next());
+            assertEquals("oof", rs.getString(1)); 
+        } catch(FunctionAlreadyExistsException e) {
+            fail("FunctionAlreadyExistsException should not be thrown");
+        }
+        // calling global udf on tenant specific specific connection.
+        rs = conn2.createStatement().executeQuery("select myreverse2(firstname) from t3");
+        assertTrue(rs.next());
+        assertEquals("oof", rs.getString(1));
+        try {
+            conn2.createStatement().execute("drop function myreverse2");
+            fail("FunctionNotFoundException should be thrown");
+        } catch(FunctionNotFoundException e){
+            
+        }
+        conn.createStatement().execute("drop function myreverse2");
+        try {
+            rs = conn2.createStatement().executeQuery("select myreverse2(firstname) from t3");
+            fail("FunctionNotFoundException should be thrown.");
+        } catch(FunctionNotFoundException e){
+            
+        }
+        try{
+            rs = conn2.createStatement().executeQuery("select unknownFunction(firstname) from t3");
+            fail("FunctionNotFoundException should be thrown.");
+        } catch(FunctionNotFoundException e) {
+            
+        }
+    }
+
+    @Test
+    public void testSameUDFWithDifferentImplementationsInDifferentTenantConnections() throws Exception {
+        Connection nonTenantConn = driver.connect(url, EMPTY_PROPS);
+        nonTenantConn.createStatement().execute("create function myfunction(VARCHAR) returns VARCHAR as 'org.apache.phoenix.end2end."+MY_REVERSE_CLASS_NAME+"' using jar "
+                + "'"+util.getConfiguration().get(DYNAMIC_JARS_DIR_KEY) + "/myjar1.jar"+"'");
+        try {
+            nonTenantConn.createStatement().execute("create function myfunction(VARCHAR) returns VARCHAR as 'org.apache.phoenix.end2end.UnknownClass' using jar "
+                    + "'"+util.getConfiguration().get(DYNAMIC_JARS_DIR_KEY) + "/myjar1.jar"+"'");
+            fail("FunctionAlreadyExistsException should be thrown.");
+        } catch(FunctionAlreadyExistsException e) {
+            
+        }
+        String tenantId1="tenId1";
+        String tenantId2="tenId2";
+        nonTenantConn.createStatement().execute("create table t7(tenant_id varchar not null, k integer not null, k1 integer, name varchar constraint pk primary key(tenant_id, k)) multi_tenant=true");
+        Connection tenant1Conn = driver.connect(url+";"+PhoenixRuntime.TENANT_ID_ATTRIB+"="+tenantId1, EMPTY_PROPS);
+        Connection tenant2Conn = driver.connect(url+";"+PhoenixRuntime.TENANT_ID_ATTRIB+"="+tenantId2, EMPTY_PROPS);
+        tenant1Conn.createStatement().execute("upsert into t7 values(1,1,'jock')");
+        tenant1Conn.commit();
+        tenant2Conn.createStatement().execute("upsert into t7 values(1,2,'jock')");
+        tenant2Conn.commit();
+        tenant1Conn.createStatement().execute("create function myfunction(VARCHAR) returns VARCHAR as 'org.apache.phoenix.end2end."+MY_REVERSE_CLASS_NAME+"' using jar "
+                + "'"+util.getConfiguration().get(DYNAMIC_JARS_DIR_KEY) + "/myjar1.jar"+"'");
+        try {
+            tenant1Conn.createStatement().execute("create function myfunction(VARCHAR) returns VARCHAR as 'org.apache.phoenix.end2end.UnknownClass' using jar "
+                    + "'"+util.getConfiguration().get(DYNAMIC_JARS_DIR_KEY) + "/myjar1.jar"+"'");
+            fail("FunctionAlreadyExistsException should be thrown.");
+        } catch(FunctionAlreadyExistsException e) {
+            
+        }
+
+        tenant2Conn.createStatement().execute("create function myfunction(INTEGER, INTEGER CONSTANT defaultValue='10' minvalue='1' maxvalue='15' ) returns INTEGER as 'org.apache.phoenix.end2end."+MY_SUM_CLASS_NAME+"' using jar "
+                + "'"+util.getConfiguration().get(DYNAMIC_JARS_DIR_KEY) + "/myjar2.jar"+"'");
+        try {
+            tenant2Conn.createStatement().execute("create function myfunction(VARCHAR) returns VARCHAR as 'org.apache.phoenix.end2end.UnknownClass' using jar "
+                    + "'"+util.getConfiguration().get(DYNAMIC_JARS_DIR_KEY) + "/unknown.jar"+"'");
+            fail("FunctionAlreadyExistsException should be thrown.");
+        } catch(FunctionAlreadyExistsException e) {
+            
+        }
+
+        ResultSet rs = tenant1Conn.createStatement().executeQuery("select MYFUNCTION(name) from t7");
+        assertTrue(rs.next());
+        assertEquals("kcoj", rs.getString(1));
+        assertFalse(rs.next());
+        rs = tenant1Conn.createStatement().executeQuery("select * from t7 where MYFUNCTION(name)='kcoj'");
+        assertTrue(rs.next());
+        assertEquals(1, rs.getInt(1));
+        assertEquals(1, rs.getInt(2));        
+        assertEquals("jock", rs.getString(3));
+        assertFalse(rs.next());
+
+        rs = tenant2Conn.createStatement().executeQuery("select MYFUNCTION(k) from t7");
+        assertTrue(rs.next());
+        assertEquals(11, rs.getInt(1));
+        assertFalse(rs.next());
+        rs = tenant2Conn.createStatement().executeQuery("select * from t7 where MYFUNCTION(k1)=12");
+        assertTrue(rs.next());
+        assertEquals(1, rs.getInt(1));
+        assertEquals(2, rs.getInt(2));        
+        assertEquals("jock", rs.getString(3));
+        assertFalse(rs.next());
+    }
+
+    @Test
+    public void testUDFsWithMultipleConnections() throws Exception {
+        Connection conn1 = driver.connect(url, EMPTY_PROPS);
+        conn1.createStatement().execute("create function myfunction(VARCHAR) returns VARCHAR as 'org.apache.phoenix.end2end."+MY_REVERSE_CLASS_NAME+"' using jar "
+                + "'"+util.getConfiguration().get(DYNAMIC_JARS_DIR_KEY) + "/myjar1.jar"+"'");
+        Connection conn2 = driver.connect(url, EMPTY_PROPS);
+        try{
+            conn2.createStatement().execute("create function myfunction(VARCHAR) returns VARCHAR as 'org.apache.phoenix.end2end."+MY_REVERSE_CLASS_NAME+"' using jar "
+                    + "'"+util.getConfiguration().get(DYNAMIC_JARS_DIR_KEY) + "/myjar1.jar"+"'");
+            fail("FunctionAlreadyExistsException should be thrown.");
+        } catch(FunctionAlreadyExistsException e) {
+            
+        }
+        conn2.createStatement().execute("create table t8(k integer not null primary key, k1 integer, name varchar)");
+        conn2.createStatement().execute("upsert into t8 values(1,1,'jock')");
+        conn2.commit();
+        ResultSet rs = conn2.createStatement().executeQuery("select MYFUNCTION(name) from t8");
+        assertTrue(rs.next());
+        assertEquals("kcoj", rs.getString(1));
+        assertFalse(rs.next());
+        rs = conn2.createStatement().executeQuery("select * from t8 where MYFUNCTION(name)='kcoj'");
+        assertTrue(rs.next());
+        assertEquals(1, rs.getInt(1));
+        assertEquals(1, rs.getInt(2));
+        assertEquals("jock", rs.getString(3));
+        assertFalse(rs.next());
+        conn2.createStatement().execute("drop function MYFUNCTION");
+        try {
+            rs = conn1.createStatement().executeQuery("select MYFUNCTION(name) from t8");
+            fail("FunctionNotFoundException should be thrown");
+        } catch(FunctionNotFoundException e) {
+            
+        }
+    }
+    @Test
+    public void testUsingUDFFunctionInDifferentQueries() throws Exception {
+        Connection conn = driver.connect(url, EMPTY_PROPS);
+        Statement stmt = conn.createStatement();
+        conn.createStatement().execute("create table t1(k integer primary key, firstname varchar, lastname varchar)");
+        stmt.execute("upsert into t1 values(1,'foo','jock')");
+        conn.commit();
+        conn.createStatement().execute("create table t2(k integer primary key, k1 integer, lastname_reverse varchar)");
+        conn.commit();
+        stmt.execute("create function mysum3(INTEGER, INTEGER CONSTANT defaultValue='10' minvalue='1' maxvalue='15' ) returns INTEGER as 'org.apache.phoenix.end2end."+MY_SUM_CLASS_NAME+"' using jar "
+                + "'"+util.getConfiguration().get(DYNAMIC_JARS_DIR_KEY) + "/myjar2.jar"+"'");
+        stmt.execute("create function myreverse3(VARCHAR) returns VARCHAR as 'org.apache.phoenix.end2end."+MY_REVERSE_CLASS_NAME+"' using jar "
+                + "'"+util.getConfiguration().get(DYNAMIC_JARS_DIR_KEY) + "/myjar1.jar"+"'");
+        stmt.execute("upsert into t2(k,k1,lastname_reverse) select mysum3(k),mysum3(k,11),myreverse3(lastname) from t1");
+        conn.commit();
+        ResultSet rs = stmt.executeQuery("select * from t2");
+        assertTrue(rs.next());
+        assertEquals(11, rs.getInt(1));
+        assertEquals(12, rs.getInt(2));
+        assertEquals("kcoj", rs.getString(3));
+        assertFalse(rs.next());
+        stmt.execute("delete from t2 where myreverse3(lastname_reverse)='jock' and mysum3(k)=21");
+        conn.commit();
+        rs = stmt.executeQuery("select * from t2");
+        assertFalse(rs.next());
+        stmt.execute("create function myreverse4(VARCHAR CONSTANT defaultValue='null') returns VARCHAR as 'org.apache.phoenix.end2end."+MY_REVERSE_CLASS_NAME+"'");
+        stmt.execute("upsert into t2 values(11,12,myreverse4('jock'))");
+        conn.commit();
+        rs = stmt.executeQuery("select * from t2");
+        assertTrue(rs.next());
+        assertEquals(11, rs.getInt(1));
+        assertEquals(12, rs.getInt(2));
+        assertEquals("kcoj", rs.getString(3));
+        assertFalse(rs.next());
+    }
+
+    @Test
+    public void testVerifyCreateFunctionArguments() throws Exception {
+        Connection conn = driver.connect(url, EMPTY_PROPS);
+        Statement stmt = conn.createStatement();
+        conn.createStatement().execute("create table t4(k integer primary key, k1 integer, lastname varchar)");
+        stmt.execute("upsert into t4 values(1,1,'jock')");
+        conn.commit();
+        stmt.execute("create function mysum(INTEGER, INTEGER CONSTANT defaultValue='10' minvalue='1' maxvalue='15' ) returns INTEGER as 'org.apache.phoenix.end2end."+MY_SUM_CLASS_NAME+"' using jar "
+                + "'"+util.getConfiguration().get(DYNAMIC_JARS_DIR_KEY) + "/myjar2.jar"+"'");
+        ResultSet rs = stmt.executeQuery("select mysum(k,12) from t4");
+        assertTrue(rs.next());
+        assertEquals(13, rs.getInt(1));
+        rs = stmt.executeQuery("select mysum(k) from t4");
+        assertTrue(rs.next());
+        assertEquals(11, rs.getInt(1));
+        try {
+            stmt.executeQuery("select mysum(k,20) from t4");
+            fail("Value Range Exception should be thrown.");
+        } catch(ValueRangeExcpetion e) {
+            
+        }
+    }
+
+    @Test
+    public void testTemporaryFunctions() throws Exception {
+        Connection conn = driver.connect(url, EMPTY_PROPS);
+        Statement stmt = conn.createStatement();
+        conn.createStatement().execute("create table t9(k integer primary key, k1 integer, lastname varchar)");
+        stmt.execute("upsert into t9 values(1,1,'jock')");
+        conn.commit();
+        stmt.execute("create temporary function mysum9(INTEGER, INTEGER CONSTANT defaultValue='10' minvalue='1' maxvalue='15' ) returns INTEGER as 'org.apache.phoenix.end2end."+MY_SUM_CLASS_NAME+"' using jar "
+                + "'"+util.getConfiguration().get(DYNAMIC_JARS_DIR_KEY) + "/myjar2.jar"+"'");
+        ResultSet rs = stmt.executeQuery("select mysum9(k,12) from t9");
+        assertTrue(rs.next());
+        assertEquals(13, rs.getInt(1));
+        rs = stmt.executeQuery("select mysum9(k) from t9");
+        assertTrue(rs.next());
+        assertEquals(11, rs.getInt(1));
+        rs = stmt.executeQuery("select k from t9 where mysum9(k)=11");
+        assertTrue(rs.next());
+        assertEquals(1, rs.getInt(1));
+        stmt.execute("drop function mysum9");
+        try {
+            rs = stmt.executeQuery("select k from t9 where mysum9(k)=11");
+            fail("FunctionNotFoundException should be thrown");
+        } catch(FunctionNotFoundException e){
+            
+        }
+    }
+
+    @Test
+    public void testDropFunction() throws Exception {
+        Connection conn = driver.connect(url, EMPTY_PROPS);
+        Statement stmt = conn.createStatement();
+        String query = "select count(*) from "+ SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_FUNCTION_TABLE + "\"";
+        ResultSet rs = stmt.executeQuery(query);
+        rs.next();
+        int numRowsBefore = rs.getInt(1);
+        stmt.execute("create function mysum6(INTEGER, INTEGER CONSTANT defaultValue='10' minvalue='1' maxvalue='15' ) returns INTEGER as 'org.apache.phoenix.end2end."+MY_SUM_CLASS_NAME+"' using jar "
+                + "'"+util.getConfiguration().get(DYNAMIC_JARS_DIR_KEY) + "/myjar2.jar"+"'");
+        rs = stmt.executeQuery(query);
+        rs.next();
+        int numRowsAfter= rs.getInt(1);
+        assertEquals(3, numRowsAfter - numRowsBefore);
+        stmt.execute("drop function mysum6");
+        rs = stmt.executeQuery(query);
+        rs.next();
+        assertEquals(numRowsBefore, rs.getInt(1));
+        conn.createStatement().execute("create table t6(k integer primary key, k1 integer, lastname varchar)");
+        try {
+            rs = stmt.executeQuery("select mysum6(k1) from t6");
+            fail("FunctionNotFoundException should be thrown");
+        } catch(FunctionNotFoundException e) {
+            
+        }
+        try {
+            stmt.execute("drop function mysum6");
+            fail("FunctionNotFoundException should be thrown");
+        } catch(FunctionNotFoundException e) {
+            
+        }
+        try {
+            stmt.execute("drop function if exists mysum6");
+        } catch(FunctionNotFoundException e) {
+            fail("FunctionNotFoundException should not be thrown");
+        }
+        stmt.execute("create function mysum6(INTEGER, INTEGER CONSTANT defaultValue='10' minvalue='1' maxvalue='15' ) returns INTEGER as 'org.apache.phoenix.end2end."+MY_SUM_CLASS_NAME+"' using jar "
+                + "'"+util.getConfiguration().get(DYNAMIC_JARS_DIR_KEY) + "/myjar2.jar"+"'");
+        try {
+            rs = stmt.executeQuery("select mysum6(k1) from t6");
+        } catch(FunctionNotFoundException e) {
+            fail("FunctionNotFoundException should not be thrown");
+        }
+    }
+
+    @Test
+    public void testFunctionalIndexesWithUDFFunction() throws Exception {
+        Connection conn = driver.connect(url, EMPTY_PROPS);
+        Statement stmt = conn.createStatement();
+        stmt.execute("create table t5(k integer primary key, k1 integer, lastname_reverse varchar)");
+        stmt.execute("create function myreverse5(VARCHAR) returns VARCHAR as 'org.apache.phoenix.end2end."+MY_REVERSE_CLASS_NAME+"'");
+        stmt.execute("upsert into t5 values(1,1,'jock')");
+        conn.commit();
+        stmt.execute("create index idx on t5(myreverse5(lastname_reverse))");
+        String query = "select myreverse5(lastname_reverse) from t5";
+        ResultSet rs = stmt.executeQuery("explain " + query);
+        assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER IDX\n"
+                + "    SERVER FILTER BY FIRST KEY ONLY", QueryUtil.getExplainPlan(rs));
+        rs = stmt.executeQuery(query);
+        assertTrue(rs.next());
+        assertEquals("kcoj", rs.getString(1));
+        assertFalse(rs.next());
+        stmt.execute("create local index idx2 on t5(myreverse5(lastname_reverse))");
+        query = "select k,k1,myreverse5(lastname_reverse) from t5 where myreverse5(lastname_reverse)='kcoj'";
+        rs = stmt.executeQuery("explain " + query);
+        assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_T5 [-32768,'kcoj']\n"
+                + "    SERVER FILTER BY FIRST KEY ONLY\n"
+                +"CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
+        rs = stmt.executeQuery(query);
+        assertTrue(rs.next());
+        assertEquals(1, rs.getInt(1));
+        assertEquals(1, rs.getInt(2));
+        assertEquals("kcoj", rs.getString(3));
+        assertFalse(rs.next());
+    }
+
+    @AfterClass
+    public static void tearDown() throws Exception {
+        try {
+            destroyDriver(driver);
+        } finally {
+            util.shutdownMiniCluster();
+        }
+    }
+
+    /**
+     * Compiles the test class with bogus code into a .class file.
+     */
+    private static void compileTestClass(String className, String program, int counter) throws Exception {
+        String javaFileName = className+".java";
+        File javaFile = new File(javaFileName);
+        String classFileName = className+".class";
+        File classFile = new File(classFileName);
+        String jarName = "myjar"+counter+".jar";
+        String jarPath = "." + File.separator + jarName;
+        File jarFile = new File(jarPath);
+        try {
+            String packageName = "org.apache.phoenix.end2end";
+            FileOutputStream fos = new FileOutputStream(javaFileName);
+            fos.write(program.getBytes());
+            fos.close();
+            
+            JavaCompiler jc = ToolProvider.getSystemJavaCompiler();
+            int result = jc.run(null, null, null, javaFileName);
+            assertEquals(0, result);
+            
+            Manifest manifest = new Manifest();
+            manifest.getMainAttributes().put(Attributes.Name.MANIFEST_VERSION, "1.0");
+            FileOutputStream jarFos = new FileOutputStream(jarPath);
+            JarOutputStream jarOutputStream = new JarOutputStream(jarFos, manifest);
+            String pathToAdd =packageName.replace('.', File.separatorChar)
+                    + File.separator;
+            jarOutputStream.putNextEntry(new JarEntry(pathToAdd));
+            jarOutputStream.closeEntry();
+            jarOutputStream.putNextEntry(new JarEntry(pathToAdd + classFile.getName()));
+            byte[] allBytes = new byte[(int) classFile.length()];
+            FileInputStream fis = new FileInputStream(classFile);
+            fis.read(allBytes);
+            fis.close();
+            jarOutputStream.write(allBytes);
+            jarOutputStream.closeEntry();
+            jarOutputStream.close();
+            jarFos.close();
+            
+            assertTrue(jarFile.exists());
+            
+            InputStream inputStream = new BufferedInputStream(new FileInputStream(jarPath));
+            FileSystem fs = util.getDefaultRootDirPath().getFileSystem(util.getConfiguration());
+            Path jarsLocation = new Path(util.getConfiguration().get(DYNAMIC_JARS_DIR_KEY));
+            Path myJarPath;
+            if (jarsLocation.toString().endsWith("/")) {
+                myJarPath = new Path(jarsLocation.toString() + jarName);
+            } else {
+                myJarPath = new Path(jarsLocation.toString() + "/" + jarName);
+            }
+            OutputStream outputStream = fs.create(myJarPath);
+            try {
+                IOUtils.copyBytes(inputStream, outputStream, 4096, false);
+            } finally {
+                IOUtils.closeStream(inputStream);
+                IOUtils.closeStream(outputStream);
+            }
+        } finally {
+            if (javaFile != null) javaFile.delete();
+            if (classFile != null) classFile.delete();
+            if (jarFile != null) jarFile.delete();
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/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
index f57c5cc..d2bb241 100644
--- a/phoenix-core/src/main/antlr3/PhoenixSQL.g
+++ b/phoenix-core/src/main/antlr3/PhoenixSQL.g
@@ -114,6 +114,14 @@ tokens
     ASYNC='async';
     SAMPLING='sampling';
     UNION='union';
+    FUNCTION='function';
+    AS='as';
+    TEMPORARY='temporary';
+    RETURNS='returns';
+    USING='using';
+    JAR='jar';
+    DEFAULTVALUE='defaultvalue';
+    CONSTANT = 'constant';
 }
 
 
@@ -144,13 +152,18 @@ import com.google.common.collect.ArrayListMultimap;
 import com.google.common.collect.ListMultimap;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+import java.lang.Boolean;
 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.expression.LiteralExpression;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.parse.PFunction;
+import org.apache.phoenix.parse.PFunction.FunctionArgument;
+import org.apache.phoenix.parse.UDFParseNode;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.IllegalDataException;
@@ -206,6 +219,7 @@ package org.apache.phoenix.parse;
     private int anonBindNum;
     private ParseNodeFactory factory;
     private ParseContext.Stack contextStack = new ParseContext.Stack();
+    private Map<String, UDFParseNode> udfParseNodes = new HashMap<String, UDFParseNode>(1);
 
     public void setParseNodeFactory(ParseNodeFactory factory) {
         this.factory = factory;
@@ -341,13 +355,25 @@ package org.apache.phoenix.parse;
 // 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
+    :  s=oneStatement {
+    		try {
+    			$ret = s;
+    		} finally {
+    			udfParseNodes.clear();
+    		}
+    	} SEMICOLON
     |  EOF
     ;
 
 // Parses a single SQL statement (expects an EOF after the select statement).
 statement returns [BindableStatement ret]
-    :   s=oneStatement {$ret = s;} EOF
+    :   s=oneStatement {
+        		try {
+    			$ret = s;
+    		} finally {
+    			udfParseNodes.clear();
+    		}
+    	} EOF
     ;
 
 // Parses a select statement which must be the only statement (expects an EOF after the statement).
@@ -369,6 +395,8 @@ oneStatement returns [BindableStatement ret]
     |   s=alter_index_node
     |   s=alter_table_node
     |   s=trace_node
+    |   s=create_function_node
+    |   s=drop_function_node
     |   s=alter_session_node
     |	s=create_sequence_node
     |	s=drop_sequence_node
@@ -409,7 +437,7 @@ create_index_node returns [CreateIndexStatement ret]
         (async=ASYNC)?
         (p=fam_properties)?
         (SPLIT ON v=value_expression_list)?
-        {ret = factory.createIndex(i, factory.namedTable(null,t), ik, icrefs, v, p, ex!=null, l==null ? IndexType.getDefault() : IndexType.LOCAL, async != null, getBindCount()); }
+        {ret = factory.createIndex(i, factory.namedTable(null,t), ik, icrefs, v, p, ex!=null, l==null ? IndexType.getDefault() : IndexType.LOCAL, async != null, getBindCount(), new HashMap<String, UDFParseNode>(udfParseNodes)); }
     ;
 
 // Parse a create sequence statement.
@@ -510,6 +538,25 @@ trace_node returns [TraceStatement ret]
        {ret = factory.trace(Tracing.isTraceOn(flag.getText()), s == null ? Tracing.isTraceOn(flag.getText()) ? 1.0 : 0.0 : (((BigDecimal)s.getValue())).doubleValue());}
     ;
 
+// Parse a trace statement.
+create_function_node returns [CreateFunctionStatement ret]
+    :   CREATE (temp=TEMPORARY)? FUNCTION function=identifier 
+       (LPAREN args=zero_or_more_data_types RPAREN)
+       RETURNS r=identifier AS (className= jar_path)
+       (USING JAR (jarPath = jar_path))?
+        {
+            $ret = factory.createFunction(new PFunction(SchemaUtil.normalizeIdentifier(function), args,r,(String)className.getValue(), jarPath == null ? null : (String)jarPath.getValue()), temp!=null);;
+        } 
+    ;
+
+jar_path returns [LiteralParseNode ret]
+    : l=literal { $ret = l; }
+    ;
+
+drop_function_node returns [DropFunctionStatement ret]
+    : DROP FUNCTION (IF ex=EXISTS)? function=identifier {$ret = factory.dropFunction(SchemaUtil.normalizeIdentifier(function), ex!=null);}
+    ;
+
 // Parse an alter session statement.
 alter_session_node returns [AlterSessionStatement ret]
     :   ALTER SESSION (SET p=properties)
@@ -586,7 +633,7 @@ single_select returns [SelectStatement ret]
         (WHERE where=expression)?
         (GROUP BY group=group_by)?
         (HAVING having=expression)?
-        { ParseContext context = contextStack.peek(); $ret = factory.select(from, h, d!=null, sel, where, group, having, null, null, getBindCount(), context.isAggregate(), context.hasSequences(), null); }
+        { ParseContext context = contextStack.peek(); $ret = factory.select(from, h, d!=null, sel, where, group, having, null, null, getBindCount(), context.isAggregate(), context.hasSequences(), null, new HashMap<String,UDFParseNode>(udfParseNodes)); }
     ;
 finally{ contextStack.pop(); }
 
@@ -610,7 +657,7 @@ upsert_node returns [UpsertStatement ret]
     :   UPSERT (hint=hintClause)? INTO t=from_table_name
         (LPAREN p=upsert_column_refs RPAREN)?
         ((VALUES LPAREN v=one_or_more_expressions RPAREN) | s=select_node)
-        {ret = factory.upsert(factory.namedTable(null,t,p == null ? null : p.getFirst()), hint, p == null ? null : p.getSecond(), v, s, getBindCount()); }
+        {ret = factory.upsert(factory.namedTable(null,t,p == null ? null : p.getFirst()), hint, p == null ? null : p.getSecond(), v, s, getBindCount(), new HashMap<String, UDFParseNode>(udfParseNodes)); }
     ;
 
 upsert_column_refs returns [Pair<List<ColumnDef>,List<ColumnName>> ret]
@@ -625,7 +672,7 @@ delete_node returns [DeleteStatement ret]
         (WHERE v=expression)?
         (ORDER BY order=order_by)?
         (LIMIT l=limit)?
-        {ret = factory.delete(factory.namedTable(null,t), hint, v, order, l, getBindCount()); }
+        {ret = factory.delete(factory.namedTable(null,t), hint, v, order, l, getBindCount(), new HashMap<String, UDFParseNode>(udfParseNodes)); }
     ;
 
 limit returns [LimitNode ret]
@@ -813,17 +860,19 @@ term returns [ParseNode ret]
             if (!contextStack.isEmpty()) {
             	contextStack.peek().setAggregate(f.isAggregate());
             }
+            if(f instanceof UDFParseNode) udfParseNodes.put(f.getName(),(UDFParseNode)f);
             $ret = f;
         } 
     |   field=identifier LPAREN t=ASTERISK RPAREN 
         {
             if (!isCountFunction(field)) {
-                throwRecognitionException(t); 
+                throwRecognitionException(t);
             }
             FunctionParseNode f = factory.function(field, LiteralParseNode.STAR);
             if (!contextStack.isEmpty()) {
             	contextStack.peek().setAggregate(f.isAggregate());
             }
+            if(f instanceof UDFParseNode) udfParseNodes.put(f.getName(),(UDFParseNode)f);
             $ret = f;
         } 
     |   field=identifier LPAREN t=DISTINCT l=zero_or_more_expressions RPAREN 
@@ -832,6 +881,7 @@ term returns [ParseNode ret]
             if (!contextStack.isEmpty()) {
             	contextStack.peek().setAggregate(f.isAggregate());
             }
+            if(f instanceof UDFParseNode) udfParseNodes.put(f.getName(),(UDFParseNode)f);
             $ret = f;
         }
     |   e=case_statement { $ret = e; }
@@ -865,6 +915,19 @@ zero_or_more_expressions returns [List<ParseNode> ret]
     :  (v = expression {$ret.add(v);})?  (COMMA v = expression {$ret.add(v);} )*
 ;
 
+zero_or_more_data_types returns [List<FunctionArgument> ret]
+@init{ret = new ArrayList<FunctionArgument>(); }
+    : (dt = identifier (LPAREN l=NUMBER (COMMA s=NUMBER)? RPAREN)? ar=ARRAY? (lsq=LSQUARE (a=NUMBER)? RSQUARE)? (c = CONSTANT)? (DEFAULTVALUE EQ dv = value_expression)? (MINVALUE EQ minv = value_expression)?  (MAXVALUE EQ maxv = value_expression)? 
+    {$ret.add(new FunctionArgument(dt,  ar != null || lsq != null, c!=null, 
+    dv == null ? null : LiteralExpression.newConstant(((LiteralParseNode)dv).getValue()), 
+    minv == null ? null : LiteralExpression.newConstant(((LiteralParseNode)minv).getValue()), 
+    maxv == null ? null : LiteralExpression.newConstant(((LiteralParseNode)maxv).getValue())));})? (COMMA (dt = identifier (LPAREN l=NUMBER (COMMA s=NUMBER)? RPAREN)? ar=ARRAY? (lsq=LSQUARE (a=NUMBER)? RSQUARE)? (c = CONSTANT)? (DEFAULTVALUE EQ dv = value_expression)? (MINVALUE EQ minv = value_expression)?  (MAXVALUE EQ maxv = value_expression)?
+    {$ret.add(new FunctionArgument(dt,  ar != null || lsq != null, c!=null, 
+    dv == null ? null : LiteralExpression.newConstant(((LiteralParseNode)dv).getValue()), 
+    minv == null ? null : LiteralExpression.newConstant(((LiteralParseNode)minv).getValue()), 
+    maxv == null ? null : LiteralExpression.newConstant(((LiteralParseNode)maxv).getValue())));} ))*
+;
+
 value_expression_list returns [List<ParseNode> ret]
 @init{ret = new ArrayList<ParseNode>(); }
     :  LPAREN e = value_expression {$ret.add(e);}  (COMMA e = value_expression {$ret.add(e);} )* RPAREN

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/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
index fcef0ec..643112d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/cache/GlobalCache.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/cache/GlobalCache.java
@@ -32,8 +32,10 @@ import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.memory.ChildMemoryManager;
 import org.apache.phoenix.memory.GlobalMemoryManager;
+import org.apache.phoenix.parse.PFunction;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.PMetaDataEntity;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.util.SizedUtil;
 
@@ -57,17 +59,17 @@ public class GlobalCache extends TenantCacheImpl {
     // 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 Cache<ImmutableBytesPtr,PTable> metaDataCache;
+    private Cache<ImmutableBytesPtr,PMetaDataEntity> metaDataCache;
     
     public void clearTenantCache() {
         perTenantCacheMap.clear();
     }
     
-    public Cache<ImmutableBytesPtr,PTable> getMetaDataCache() {
+    public Cache<ImmutableBytesPtr,PMetaDataEntity> getMetaDataCache() {
         // Lazy initialize QueryServices so that we only attempt to create an HBase Configuration
         // object upon the first attempt to connect to any cluster. Otherwise, an attempt will be
         // made at driver initialization time which is too early for some systems.
-        Cache<ImmutableBytesPtr,PTable> result = metaDataCache;
+        Cache<ImmutableBytesPtr,PMetaDataEntity> result = metaDataCache;
         if (result == null) {
             synchronized(this) {
                 result = metaDataCache;
@@ -82,9 +84,9 @@ public class GlobalCache extends TenantCacheImpl {
                     metaDataCache = result = CacheBuilder.newBuilder()
                             .maximumWeight(maxSize)
                             .expireAfterAccess(maxTTL, TimeUnit.MILLISECONDS)
-                            .weigher(new Weigher<ImmutableBytesPtr, PTable>() {
+                            .weigher(new Weigher<ImmutableBytesPtr, PMetaDataEntity>() {
                                 @Override
-                                public int weigh(ImmutableBytesPtr key, PTable table) {
+                                public int weigh(ImmutableBytesPtr key, PMetaDataEntity table) {
                                     return SizedUtil.IMMUTABLE_BYTES_PTR_SIZE + key.getLength() + table.getEstimatedSize();
                                 }
                             })
@@ -157,4 +159,22 @@ public class GlobalCache extends TenantCacheImpl {
         }
         return tenantCache;
     }
+
+    public static class FunctionBytesPtr extends ImmutableBytesPtr {
+
+        public FunctionBytesPtr(byte[] key) {
+            super(key);
+        }
+
+        @Override
+        public boolean equals(Object obj) {
+            if(obj instanceof FunctionBytesPtr) return super.equals(obj);
+            return false;
+        }
+
+        @Override
+        public int hashCode() {
+            return super.hashCode();
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/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
index 7bb210b..55253ea 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/ColumnResolver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/ColumnResolver.java
@@ -20,6 +20,7 @@ package org.apache.phoenix.compile;
 import java.sql.SQLException;
 import java.util.List;
 
+import org.apache.phoenix.parse.PFunction;
 import org.apache.phoenix.schema.ColumnRef;
 import org.apache.phoenix.schema.TableRef;
 
@@ -41,6 +42,11 @@ public interface ColumnResolver {
     public List<TableRef> getTables();
     
     /**
+     * Returns the collection of resolved functions.
+     */
+    public List<PFunction> getFunctions();
+
+    /**
      * Resolves table using name or alias.
      * @param schemaName the schema name
      * @param tableName the table name or table alias
@@ -60,4 +66,15 @@ public interface ColumnResolver {
      * @throws AmbiguousColumnException if the column name is ambiguous
      */
     public ColumnRef resolveColumn(String schemaName, String tableName, String colName) throws SQLException;
+        
+    /**
+     * Resolves function using functionName.
+     * @param functionName 
+     * @return the resolved PFunction
+     * @throws ColumnNotFoundException if the column could not be resolved
+     * @throws AmbiguousColumnException if the column name is ambiguous
+     */
+    public PFunction resolveFunction(String functionName) throws SQLException;
+
+    public boolean hasUDFs();
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateFunctionCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateFunctionCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateFunctionCompiler.java
new file mode 100644
index 0000000..2e3a873
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateFunctionCompiler.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.phoenix.compile;
+
+import java.sql.ParameterMetaData;
+import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
+import java.util.Collections;
+
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.phoenix.execute.MutationState;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixStatement;
+import org.apache.phoenix.parse.CreateFunctionStatement;
+import org.apache.phoenix.schema.MetaDataClient;
+
+public class CreateFunctionCompiler {
+
+    private final PhoenixStatement statement;
+    
+    public CreateFunctionCompiler(PhoenixStatement statement) {
+        this.statement = statement;
+    }
+
+    public MutationPlan compile(final CreateFunctionStatement create) throws SQLException {
+        final PhoenixConnection connection = statement.getConnection();
+        PhoenixConnection connectionToBe = connection;
+        final StatementContext context = new StatementContext(statement);
+        final MetaDataClient client = new MetaDataClient(connectionToBe);
+        
+        return new MutationPlan() {
+
+            @Override
+            public ParameterMetaData getParameterMetaData() {
+                return context.getBindManager().getParameterMetaData();
+            }
+
+            @Override
+            public MutationState execute() throws SQLException {
+                try {
+                    return client.createFunction(create);
+                } finally {
+                    if (client.getConnection() != connection) {
+                        client.getConnection().close();
+                    }
+                }
+            }
+
+            @Override
+            public ExplainPlan getExplainPlan() throws SQLException {
+                return new ExplainPlan(Collections.singletonList("CREATE FUNCTION"));
+            }
+
+            @Override
+            public PhoenixConnection getConnection() {
+                return connection;
+            }
+            
+            @Override
+            public StatementContext getContext() {
+                return context;
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/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
index 07d9f56..f1937a5 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateIndexCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateIndexCompiler.java
@@ -46,7 +46,7 @@ public class CreateIndexCompiler {
 
     public MutationPlan compile(final CreateIndexStatement create) throws SQLException {
         final PhoenixConnection connection = statement.getConnection();
-        final ColumnResolver resolver = FromCompiler.getResolver(create, connection);
+        final ColumnResolver resolver = FromCompiler.getResolver(create, connection, create.getUdfParseNodes());
         Scan scan = new Scan();
         final StatementContext context = new StatementContext(statement, resolver, scan, new SequenceManager(statement));
         ExpressionCompiler expressionCompiler = new ExpressionCompiler(context);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
index 4f6a719..575f0f3 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
@@ -323,7 +323,7 @@ public class DeleteCompiler {
                         hint, false, aliasedNodes, delete.getWhere(), 
                         Collections.<ParseNode>emptyList(), null, 
                         delete.getOrderBy(), delete.getLimit(),
-                        delete.getBindCount(), false, false, Collections.<SelectStatement>emptyList());
+                        delete.getBindCount(), false, false, Collections.<SelectStatement>emptyList(), delete.getUdfParseNodes());
                 select = StatementNormalizer.normalize(select, resolver);
                 SelectStatement transformedSelect = SubqueryRewriter.transform(select, resolver, connection);
                 if (transformedSelect != select) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionCompiler.java
index ab6b851..92899a6 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionCompiler.java
@@ -17,6 +17,7 @@
  */
 package org.apache.phoenix.compile;
 
+
 import java.math.BigDecimal;
 import java.sql.SQLException;
 import java.sql.SQLFeatureNotSupportedException;
@@ -70,6 +71,7 @@ import org.apache.phoenix.expression.function.ArrayAnyComparisonExpression;
 import org.apache.phoenix.expression.function.ArrayElemRefExpression;
 import org.apache.phoenix.expression.function.RoundDecimalExpression;
 import org.apache.phoenix.expression.function.RoundTimestampExpression;
+import org.apache.phoenix.expression.function.UDFExpression;
 import org.apache.phoenix.parse.AddParseNode;
 import org.apache.phoenix.parse.AndParseNode;
 import org.apache.phoenix.parse.ArithmeticParseNode;
@@ -95,12 +97,14 @@ import org.apache.phoenix.parse.ModulusParseNode;
 import org.apache.phoenix.parse.MultiplyParseNode;
 import org.apache.phoenix.parse.NotParseNode;
 import org.apache.phoenix.parse.OrParseNode;
+import org.apache.phoenix.parse.PFunction;
 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.SubqueryParseNode;
 import org.apache.phoenix.parse.SubtractParseNode;
+import org.apache.phoenix.parse.UDFParseNode;
 import org.apache.phoenix.parse.UnsupportedAllParseNodeVisitor;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
@@ -313,8 +317,19 @@ public class ExpressionCompiler extends UnsupportedAllParseNodeVisitor<Expressio
      * @param children the child expression arguments to the function expression node.
      */
     public Expression visitLeave(FunctionParseNode node, List<Expression> children) throws SQLException {
+        PFunction function = null;
+        if(node instanceof UDFParseNode) {
+            function = context.getResolver().resolveFunction(node.getName());
+            BuiltInFunctionInfo info = new BuiltInFunctionInfo(function);
+            node = new UDFParseNode(node.getName(), node.getChildren(), info);
+        }
         children = node.validate(children, context);
-        Expression expression = node.create(children, context);
+        Expression expression = null;
+        if (function == null) {
+            expression = node.create(children, context);
+        } else {
+            expression = node.create(children, function, context);
+        }
         ImmutableBytesWritable ptr = context.getTempPtr();
         BuiltInFunctionInfo info = node.getInfo();
         for (int i = 0; i < info.getRequiredArgCount(); i++) { 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
index da78b24..5fe0e6f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
@@ -22,9 +22,11 @@ import java.sql.SQLFeatureNotSupportedException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.client.HTableInterface;
@@ -35,12 +37,15 @@ import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.parse.AliasedNode;
 import org.apache.phoenix.parse.BindTableNode;
 import org.apache.phoenix.parse.ColumnDef;
+import org.apache.phoenix.parse.CreateFunctionStatement;
 import org.apache.phoenix.parse.CreateTableStatement;
 import org.apache.phoenix.parse.DMLStatement;
 import org.apache.phoenix.parse.DerivedTableNode;
 import org.apache.phoenix.parse.FamilyWildcardParseNode;
 import org.apache.phoenix.parse.JoinTableNode;
+import org.apache.phoenix.parse.NamedNode;
 import org.apache.phoenix.parse.NamedTableNode;
+import org.apache.phoenix.parse.PFunction;
 import org.apache.phoenix.parse.ParseNode;
 import org.apache.phoenix.parse.ParseNodeFactory;
 import org.apache.phoenix.parse.SelectStatement;
@@ -49,6 +54,7 @@ import org.apache.phoenix.parse.TableName;
 import org.apache.phoenix.parse.TableNode;
 import org.apache.phoenix.parse.TableNodeVisitor;
 import org.apache.phoenix.parse.TableWildcardParseNode;
+import org.apache.phoenix.parse.UDFParseNode;
 import org.apache.phoenix.parse.WildcardParseNode;
 import org.apache.phoenix.query.ConnectionQueryServices;
 import org.apache.phoenix.query.QueryConstants;
@@ -57,6 +63,7 @@ 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.FunctionNotFoundException;
 import org.apache.phoenix.schema.MetaDataClient;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PColumnFamily;
@@ -103,6 +110,11 @@ public class FromCompiler {
         }
 
         @Override
+        public List<PFunction> getFunctions() {
+            return Collections.emptyList();
+        }
+
+        @Override
         public TableRef resolveTable(String schemaName, String tableName)
                 throws SQLException {
             throw new UnsupportedOperationException();
@@ -112,6 +124,14 @@ public class FromCompiler {
         public ColumnRef resolveColumn(String schemaName, String tableName, String colName) throws SQLException {
             throw new UnsupportedOperationException();
         }
+        
+        public PFunction resolveFunction(String functionName) throws SQLException {
+            throw new UnsupportedOperationException();
+        };
+
+        public boolean hasUDFs() {
+            return false;
+        };
     };
 
     public static ColumnResolver getResolverForCreation(final CreateTableStatement statement, final PhoenixConnection connection)
@@ -141,7 +161,7 @@ public class FromCompiler {
                     if (htable != null) Closeables.closeQuietly(htable);
                 }
                 tableNode = NamedTableNode.create(null, baseTable, statement.getColumnDefs());
-                return new SingleTableColumnResolver(connection, tableNode, e.getTimeStamp());
+                return new SingleTableColumnResolver(connection, tableNode, e.getTimeStamp(), new HashMap<String, UDFParseNode>(1));
             }
             throw e;
         }
@@ -166,9 +186,9 @@ public class FromCompiler {
     	if (fromNode == null)
     	    return EMPTY_TABLE_RESOLVER;
         if (fromNode instanceof NamedTableNode)
-            return new SingleTableColumnResolver(connection, (NamedTableNode) fromNode, true, 1);
+            return new SingleTableColumnResolver(connection, (NamedTableNode) fromNode, true, 1, statement.getUdfParseNodes());
 
-        MultiTableColumnResolver visitor = new MultiTableColumnResolver(connection, 1);
+        MultiTableColumnResolver visitor = new MultiTableColumnResolver(connection, 1, statement.getUdfParseNodes());
         fromNode.accept(visitor);
         return visitor;
     }
@@ -178,12 +198,24 @@ public class FromCompiler {
         return visitor;
     }
 
+    public static ColumnResolver getResolver(NamedTableNode tableNode, PhoenixConnection connection, Map<String, UDFParseNode> udfParseNodes) throws SQLException {
+        SingleTableColumnResolver visitor =
+                new SingleTableColumnResolver(connection, tableNode, true, 0, udfParseNodes);
+        return visitor;
+    }
+
     public static ColumnResolver getResolver(SingleTableStatement statement, PhoenixConnection connection)
             throws SQLException {
         SingleTableColumnResolver visitor = new SingleTableColumnResolver(connection, statement.getTable(), true);
         return visitor;
     }
 
+    public static ColumnResolver getResolver(SingleTableStatement statement, PhoenixConnection connection, Map<String, UDFParseNode> udfParseNodes)
+            throws SQLException {
+        SingleTableColumnResolver visitor = new SingleTableColumnResolver(connection, statement.getTable(), true, 0, udfParseNodes);
+        return visitor;
+    }
+
     public static ColumnResolver getResolverForCompiledDerivedTable(PhoenixConnection connection, TableRef tableRef, RowProjector projector)
             throws SQLException {
         List<PColumn> projectedColumns = new ArrayList<PColumn>();
@@ -205,26 +237,32 @@ public class FromCompiler {
         return visitor;
     }
 
+    public static ColumnResolver getResolver(PhoenixConnection connection, TableRef tableRef, Map<String, UDFParseNode> udfParseNodes)
+            throws SQLException {
+        SingleTableColumnResolver visitor = new SingleTableColumnResolver(connection, tableRef, udfParseNodes);
+        return visitor;
+    }
+
     public static ColumnResolver getResolverForMutation(DMLStatement statement, PhoenixConnection connection)
             throws SQLException {
         /*
          * We validate the meta data at commit time for mutations, as this allows us to do many UPSERT VALUES calls
          * without hitting the server each time to check if the meta data is up-to-date.
          */
-        SingleTableColumnResolver visitor = new SingleTableColumnResolver(connection, statement.getTable(), false);
+        SingleTableColumnResolver visitor = new SingleTableColumnResolver(connection, statement.getTable(), false, 0,  statement.getUdfParseNodes());
         return visitor;
     }
     
-    public static ColumnResolver getResolverForProjectedTable(PTable projectedTable) {
-        return new ProjectedTableColumnResolver(projectedTable);
+    public static ColumnResolver getResolverForProjectedTable(PTable projectedTable, PhoenixConnection connection, Map<String, UDFParseNode> udfParseNodes) throws SQLException {
+        return new ProjectedTableColumnResolver(projectedTable, connection, udfParseNodes);
     }
 
     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, 0);
+       public SingleTableColumnResolver(PhoenixConnection connection, NamedTableNode table, long timeStamp, Map<String, UDFParseNode> udfParseNodes) throws SQLException  {
+           super(connection, 0, false, udfParseNodes);
            List<PColumnFamily> families = Lists.newArrayListWithExpectedSize(table.getDynamicColumns().size());
            for (ColumnDef def : table.getDynamicColumns()) {
                if (def.getColumnDefName().getFamilyName() != null) {
@@ -239,11 +277,13 @@ public class FromCompiler {
        }
 
         public SingleTableColumnResolver(PhoenixConnection connection, NamedTableNode tableNode, boolean updateCacheImmediately) throws SQLException {
-            this(connection, tableNode, updateCacheImmediately, 0);
+            this(connection, tableNode, updateCacheImmediately, 0, new HashMap<String,UDFParseNode>(1));
         }
 
-        public SingleTableColumnResolver(PhoenixConnection connection, NamedTableNode tableNode, boolean updateCacheImmediately, int tsAddition) throws SQLException {
-            super(connection, tsAddition);
+        public SingleTableColumnResolver(PhoenixConnection connection, NamedTableNode tableNode,
+                boolean updateCacheImmediately, int tsAddition,
+                Map<String, UDFParseNode> udfParseNodes) throws SQLException {
+            super(connection, tsAddition, updateCacheImmediately, udfParseNodes);
             alias = tableNode.getAlias();
             TableRef tableRef = createTableRef(tableNode, updateCacheImmediately);
             tableRefs = ImmutableList.of(tableRef);
@@ -255,6 +295,12 @@ public class FromCompiler {
             tableRefs = ImmutableList.of(tableRef);
         }
 
+        public SingleTableColumnResolver(PhoenixConnection connection, TableRef tableRef, Map<String, UDFParseNode> udfParseNodes) throws SQLException {
+            super(connection, 0, false, udfParseNodes);
+            alias = tableRef.getTableAlias();
+            tableRefs = ImmutableList.of(tableRef);
+        }
+
         public SingleTableColumnResolver(TableRef tableRef) throws SQLException {
             super(null, 0);
             alias = tableRef.getTableAlias();
@@ -267,6 +313,11 @@ public class FromCompiler {
 		}
 
         @Override
+        public List<PFunction> getFunctions() {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
         public TableRef resolveTable(String schemaName, String tableName)
                 throws SQLException {
             TableRef tableRef = tableRefs.get(0);
@@ -316,7 +367,6 @@ public class FromCompiler {
         			: tableRef.getTable().getColumn(colName);
             return new ColumnRef(tableRef, column.getPosition());
 		}
-
     }
 
     private static abstract class BaseColumnResolver implements ColumnResolver {
@@ -326,11 +376,30 @@ public class FromCompiler {
         // on Windows because the millis timestamp granularity is so bad we sometimes won't
         // get the data back that we just upsert.
         private final int tsAddition;
+        protected final Map<String, PFunction> functionMap;
+        protected List<PFunction> functions;
 
         private BaseColumnResolver(PhoenixConnection connection, int tsAddition) {
+            this.connection = connection;
+            this.client = connection == null ? null : new MetaDataClient(connection);
+            this.tsAddition = tsAddition;
+            functionMap = new HashMap<String, PFunction>(1);
+            this.functions = Collections.<PFunction>emptyList();
+        }
+
+        private BaseColumnResolver(PhoenixConnection connection, int tsAddition, boolean updateCacheImmediately, Map<String, UDFParseNode> udfParseNodes) throws SQLException {
         	this.connection = connection;
             this.client = connection == null ? null : new MetaDataClient(connection);
             this.tsAddition = tsAddition;
+            functionMap = new HashMap<String, PFunction>(1);
+            if (udfParseNodes.isEmpty()) {
+                functions = Collections.<PFunction> emptyList();
+            } else {
+                functions = createFunctionRef(new ArrayList<String>(udfParseNodes.keySet()), updateCacheImmediately);
+                for (PFunction function : functions) {
+                    functionMap.put(function.getFunctionName(), function);
+                }
+            }
         }
 
         protected TableRef createTableRef(NamedTableNode tableNode, boolean updateCacheImmediately) throws SQLException {
@@ -383,6 +452,85 @@ public class FromCompiler {
             return tableRef;
         }
 
+        @Override
+        public List<PFunction> getFunctions() {
+            return functions;
+        }
+
+        private List<PFunction> createFunctionRef(List<String> functionNames, boolean updateCacheImmediately) throws SQLException {
+            long timeStamp = QueryConstants.UNSET_TIMESTAMP;
+            int numFunctions = functionNames.size();
+            List<PFunction> functionsFound = new ArrayList<PFunction>(functionNames.size());
+            if (updateCacheImmediately || connection.getAutoCommit()) {
+                getFunctionFromCache(functionNames, functionsFound, true);
+                if(functionNames.isEmpty()) {
+                    return functionsFound;
+                }
+                MetaDataMutationResult result = client.updateCache(functionNames);
+                timeStamp = result.getMutationTime();
+                functionsFound = result.getFunctions();
+                if(functionNames.size() != functionsFound.size()){
+                    throw new FunctionNotFoundException("Some of the functions in "+functionNames.toString()+" are not found");
+                }
+            } else {
+                getFunctionFromCache(functionNames, functionsFound, false);
+                // We always attempt to update the cache in the event of a FunctionNotFoundException
+                MetaDataMutationResult result = null;
+                if (!functionNames.isEmpty()) {
+                    result = client.updateCache(functionNames);
+                }
+                if(result!=null) {
+                    if (!result.getFunctions().isEmpty()) {
+                        functionsFound.addAll(result.getFunctions());
+                    }
+                    if(result.wasUpdated()) {
+                        timeStamp = result.getMutationTime();
+                    }
+                }
+                if (functionsFound.size()!=numFunctions) {
+                    throw new FunctionNotFoundException("Some of the functions in "+functionNames.toString()+" are not found", timeStamp);
+                }
+            }
+            if (timeStamp != QueryConstants.UNSET_TIMESTAMP) {
+                timeStamp += tsAddition;
+            }
+            
+            if (logger.isDebugEnabled() && timeStamp != QueryConstants.UNSET_TIMESTAMP) {
+                logger.debug(LogUtil.addCustomAnnotations("Re-resolved stale function " + functionNames.toString() + "at timestamp " + timeStamp, connection));
+            }
+            return functionsFound;
+        }
+
+        private void getFunctionFromCache(List<String> functionNames,
+                List<PFunction> functionsFound, boolean getOnlyTemporyFunctions) {
+            Iterator<String> iterator = functionNames.iterator();
+            while(iterator.hasNext()) {
+                PFunction function = null;
+                String functionName = iterator.next();
+                try {
+                    function = connection.getMetaDataCache().getFunction(new PTableKey(connection.getTenantId(), functionName));
+                } catch (FunctionNotFoundException e1) {
+                    if (connection.getTenantId() != null) { // Check with null tenantId next
+                        try {
+                            function = connection.getMetaDataCache().getFunction(new PTableKey(null, functionName));
+                        } catch (FunctionNotFoundException e2) {
+                        }
+                    }
+                }
+                if (function != null) {
+                    if (getOnlyTemporyFunctions) {
+                        if (function.isTemporaryFunction()) {
+                            functionsFound.add(function);
+                            iterator.remove();
+                        }
+                    } else {
+                        functionsFound.add(function);
+                        iterator.remove();
+                    }
+                }
+            }
+        }
+
         protected PTable addDynamicColumns(List<ColumnDef> dynColumns, PTable theTable)
                 throws SQLException {
             if (!dynColumns.isEmpty()) {
@@ -409,6 +557,20 @@ public class FromCompiler {
             }
             return theTable;
         }
+        
+        @Override
+        public PFunction resolveFunction(String functionName) throws SQLException {
+            PFunction function = functionMap.get(functionName);
+            if(function == null) {
+                throw new FunctionNotFoundException(functionName);
+            }
+            return function;
+        }
+
+        @Override
+        public boolean hasUDFs() {
+            return !functions.isEmpty();
+        }
     }
 
     private static class MultiTableColumnResolver extends BaseColumnResolver implements TableNodeVisitor<Void> {
@@ -421,6 +583,12 @@ public class FromCompiler {
             tables = Lists.newArrayList();
         }
 
+        private MultiTableColumnResolver(PhoenixConnection connection, int tsAddition, Map<String, UDFParseNode> udfParseNodes) throws SQLException {
+            super(connection, tsAddition, false, udfParseNodes);
+            tableMap = ArrayListMultimap.<String, TableRef> create();
+            tables = Lists.newArrayList();
+        }
+
         @Override
         public List<TableRef> getTables() {
             return tables;
@@ -580,16 +748,14 @@ public class FromCompiler {
                 }
             }
         }
-
     }
     
     private static class ProjectedTableColumnResolver extends MultiTableColumnResolver {
         private final boolean isLocalIndex;
         private final List<TableRef> theTableRefs;
         private final Map<ColumnRef, Integer> columnRefMap;
-        
-        private ProjectedTableColumnResolver(PTable projectedTable) {
-            super(null, 0);
+        private ProjectedTableColumnResolver(PTable projectedTable, PhoenixConnection conn, Map<String, UDFParseNode> udfParseNodes) throws SQLException {
+            super(conn, 0, udfParseNodes);
             Preconditions.checkArgument(projectedTable.getType() == PTableType.PROJECTED);
             this.isLocalIndex = projectedTable.getIndexType() == IndexType.LOCAL;
             this.columnRefMap = new HashMap<ColumnRef, Integer>();
@@ -615,6 +781,7 @@ public class FromCompiler {
                 this.columnRefMap.put(new ColumnRef(tableRef, colRef.getColumnPosition()), column.getPosition());
             }
             this.theTableRefs = ImmutableList.of(new TableRef(ParseNodeFactory.createTempAlias(), projectedTable, ts, false));
+            
         }
         
         @Override


[3/7] phoenix git commit: PHOENIX-538 Support UDFs(Rajeshbabu Chintaguntla)

Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java b/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
index 9c38348..cf72384 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
@@ -23,12 +23,15 @@ import java.util.Map;
 
 import org.apache.phoenix.hbase.index.util.IndexManagementUtil;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.query.QueryServices;
 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.FunctionAlreadyExistsException;
+import org.apache.phoenix.schema.FunctionNotFoundException;
 import org.apache.phoenix.schema.ReadOnlyTableException;
 import org.apache.phoenix.schema.SequenceAlreadyExistsException;
 import org.apache.phoenix.schema.SequenceNotFoundException;
@@ -320,7 +323,22 @@ public enum SQLExceptionCode {
             return new SQLTimeoutException(OPERATION_TIMED_OUT.getMessage(),
                     OPERATION_TIMED_OUT.getSQLState(), OPERATION_TIMED_OUT.getErrorCode());
         }
-    }), 
+    }),
+    FUNCTION_UNDEFINED(6001, "42F01", "Function undefined.", new Factory() {
+        @Override
+        public SQLException newException(SQLExceptionInfo info) {
+            return new FunctionNotFoundException(info.getFunctionName());
+        }
+    }),
+    FUNCTION_ALREADY_EXIST(6002, "42F02", "Function already exists.", new Factory() {
+        @Override
+        public SQLException newException(SQLExceptionInfo info) {
+            return new FunctionAlreadyExistsException(info.getSchemaName(), info.getTableName());
+        }
+    }),
+    UNALLOWED_USER_DEFINED_FUNCTIONS(6003, "42F03",
+            "User defined functions are configured to not be allowed. To allow configure "
+                    + QueryServices.ALLOW_USER_DEFINED_FUNCTIONS_ATTRIB + " to true."),
     ;
 
     private final int errorCode;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/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
index fa31190..50dffde 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionInfo.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionInfo.java
@@ -37,6 +37,7 @@ public class SQLExceptionInfo {
     public static final String TABLE_NAME = "tableName";
     public static final String FAMILY_NAME = "familyName";
     public static final String COLUMN_NAME = "columnName";
+    public static final String FUNCTION_NAME = "functionName";
 
     private final Throwable rootCause;
     private final SQLExceptionCode code; // Should always have one.
@@ -45,6 +46,7 @@ public class SQLExceptionInfo {
     private final String tableName;
     private final String familyName;
     private final String columnName;
+    private final String functionName;
 
     public static class Builder {
 
@@ -55,6 +57,7 @@ public class SQLExceptionInfo {
         private String tableName;
         private String familyName;
         private String columnName;
+        private String functionName;
 
         public Builder(SQLExceptionCode code) {
             this.code = code;
@@ -90,6 +93,10 @@ public class SQLExceptionInfo {
             return this;
         }
 
+        public Builder setFunctionName(String functionName) {
+            this.functionName = functionName;
+            return this;
+        }
         public SQLExceptionInfo build() {
             return new SQLExceptionInfo(this);
         }
@@ -108,6 +115,7 @@ public class SQLExceptionInfo {
         tableName = builder.tableName;
         familyName = builder.familyName;
         columnName = builder.columnName;
+        functionName = builder.functionName;
     }
 
     @Override
@@ -116,6 +124,10 @@ public class SQLExceptionInfo {
         if (message != null) {
             builder.append(" ").append(message);
         }
+        if (functionName != null) {
+            builder.append(" ").append(FUNCTION_NAME).append("=").append(functionName);
+            return builder.toString();
+        }
         String columnDisplayName = SchemaUtil.getMetaDataEntityName(schemaName, tableName, familyName, columnName);
         if (columnName != null) {
             builder.append(" ").append(COLUMN_NAME).append("=").append(columnDisplayName);
@@ -153,6 +165,10 @@ public class SQLExceptionInfo {
         return columnName;
     }
 
+    public String getFunctionName() {
+        return functionName;
+    }
+    
     public SQLExceptionCode getCode() {
         return code;
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
index 5f598b9..69a2049 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
@@ -95,6 +95,7 @@ import org.apache.phoenix.expression.function.ToTimeFunction;
 import org.apache.phoenix.expression.function.ToTimestampFunction;
 import org.apache.phoenix.expression.function.TrimFunction;
 import org.apache.phoenix.expression.function.TruncFunction;
+import org.apache.phoenix.expression.function.UDFExpression;
 import org.apache.phoenix.expression.function.UpperFunction;
 import org.apache.phoenix.expression.function.WeekFunction;
 import org.apache.phoenix.expression.function.YearFunction;
@@ -227,7 +228,8 @@ public enum ExpressionType {
     InstrFunction(InstrFunction.class),
     MinuteFunction(MinuteFunction.class),
     DayOfMonthFunction(DayOfMonthFunction.class),
-    ArrayAppendFunction(ArrayAppendFunction.class)
+    ArrayAppendFunction(ArrayAppendFunction.class),
+    UDFExpression(UDFExpression.class)
     ;
 
     ExpressionType(Class<? extends Expression> clazz) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/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
index e694680..014bda4 100644
--- 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
@@ -53,7 +53,7 @@ public abstract class ScalarFunction extends FunctionExpression {
     }
     
     @Override
-    public final <T> T accept(ExpressionVisitor<T> visitor) {
+    public <T> T accept(ExpressionVisitor<T> visitor) {
         List<T> l = acceptChildren(visitor, visitor.visitEnter(this));
         T t = visitor.visitLeave(this, l);
         if (t == null) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/phoenix-core/src/main/java/org/apache/phoenix/expression/function/UDFExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/UDFExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/UDFExpression.java
new file mode 100644
index 0000000..4e12b17
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/UDFExpression.java
@@ -0,0 +1,220 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.query.QueryServices.DYNAMIC_JARS_DIR_KEY;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+import java.util.List;
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.DynamicClassLoader;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.phoenix.compile.KeyPart;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.visitor.ExpressionVisitor;
+import org.apache.phoenix.parse.PFunction;
+import org.apache.phoenix.schema.PName;
+import org.apache.phoenix.schema.PNameFactory;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.schema.types.PDataType;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.MapMaker;
+
+public class UDFExpression extends ScalarFunction {
+    
+    private static Configuration config = HBaseConfiguration.create();
+
+    private static final ConcurrentMap<PName, DynamicClassLoader> tenantIdSpecificCls =
+            new MapMaker().concurrencyLevel(3).weakValues().makeMap();
+
+    private static final ConcurrentMap<String, DynamicClassLoader> pathSpecificCls =
+            new MapMaker().concurrencyLevel(3).weakValues().makeMap();
+
+    private PName tenantId;
+    private String functionClassName;
+    private String jarPath;
+    private ScalarFunction udfFunction;
+    
+    public UDFExpression() {
+    }
+
+    public UDFExpression(List<Expression> children,PFunction functionInfo) {
+        super(children);
+        this.tenantId =
+                functionInfo.getTenantId() == null ? PName.EMPTY_NAME : functionInfo.getTenantId();
+        this.functionClassName = functionInfo.getClassName();
+        this.jarPath = functionInfo.getJarPath();
+        constructUDFFunction();
+    }
+
+    public UDFExpression(List<Expression> children, PName tenantId, String functionClassName,
+            String jarPath, ScalarFunction udfFunction) {
+        super(children);
+        this.tenantId = tenantId;
+        this.functionClassName = functionClassName;
+        this.jarPath = jarPath;
+        if(udfFunction != null) {
+            this.udfFunction = udfFunction;
+        } else {
+            constructUDFFunction();
+        }
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        return udfFunction.evaluate(tuple, ptr);
+    }
+
+    @Override
+    public <T> T accept(ExpressionVisitor<T> visitor) {
+        return udfFunction.accept(visitor);
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return udfFunction.getDataType();
+    }
+
+    @Override
+    public String getName() {
+        return udfFunction.getName();
+    }
+
+    @Override
+    public OrderPreserving preservesOrder() {
+        return udfFunction.preservesOrder();
+    }
+
+    @Override
+    public KeyPart newKeyPart(KeyPart childPart) {
+        return udfFunction.newKeyPart(childPart);
+    }
+
+    @Override
+    public int getKeyFormationTraversalIndex() {
+        return udfFunction.getKeyFormationTraversalIndex();
+    }
+
+    public PName getTenantId() {
+        return tenantId;
+    }
+
+    public String getFunctionClassName() {
+        return functionClassName;
+    }
+
+    public String getJarPath() {
+        return jarPath;
+    }
+
+    public ScalarFunction getUdfFunction() {
+        return udfFunction;
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+        super.write(output);
+        WritableUtils.writeString(output, tenantId.getString());
+        WritableUtils.writeString(output, this.functionClassName);
+        if(this.jarPath == null) {
+            WritableUtils.writeString(output, "");
+        } else {
+            WritableUtils.writeString(output, this.jarPath);
+        }
+    }
+    
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        super.readFields(input);
+        this.tenantId = PNameFactory.newName(WritableUtils.readString(input));
+        this.functionClassName = WritableUtils.readString(input);
+        String str = WritableUtils.readString(input);
+        this.jarPath = str.length() == 0 ? null: str;
+        constructUDFFunction();
+    }
+
+    private void constructUDFFunction() {
+        try {
+            DynamicClassLoader classLoader = getClassLoader(this.tenantId, this.jarPath);
+            Class<?> clazz = classLoader.loadClass(this.functionClassName);
+            Constructor<?> constructor = clazz.getConstructor(List.class);
+            udfFunction = (ScalarFunction)constructor.newInstance(this.children);
+        } catch (ClassNotFoundException | NoSuchMethodException | SecurityException
+                | InstantiationException | IllegalAccessException | IllegalArgumentException
+                | InvocationTargetException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public static DynamicClassLoader getClassLoader(final PName tenantId, final String jarPath) {
+        DynamicClassLoader cl = tenantIdSpecificCls.get(tenantId);
+        String parent = null;
+        if (cl != null) return cl;
+        if(jarPath != null && !jarPath.isEmpty()) {
+            cl = pathSpecificCls.get(jarPath);
+            if (cl != null) return cl;
+            Path path = new Path(jarPath);
+            if(jarPath.endsWith(".jar")) {
+                parent = path.getParent().toString();
+            } else {
+                parent = path.toString();
+            }
+        }
+        if (jarPath == null || jarPath.isEmpty() || config.get(DYNAMIC_JARS_DIR_KEY) != null
+                && (parent != null && parent.equals(config.get(DYNAMIC_JARS_DIR_KEY)))) {
+            cl = tenantIdSpecificCls.get(tenantId);
+            if (cl == null) {
+                cl = new DynamicClassLoader(config, UDFExpression.class.getClassLoader());
+            }
+            // Cache class loader as a weak value, will be GC'ed when no reference left
+            DynamicClassLoader prev = tenantIdSpecificCls.putIfAbsent(tenantId, cl);
+            if (prev != null) {
+                cl = prev;
+            }
+            return cl;
+        } else {
+            cl = pathSpecificCls.get(jarPath);
+            if (cl == null) {
+                Configuration conf = HBaseConfiguration.create(config);
+                conf.set(DYNAMIC_JARS_DIR_KEY, parent);
+                cl = new DynamicClassLoader(conf, UDFExpression.class.getClassLoader());
+            }
+            // Cache class loader as a weak value, will be GC'ed when no reference left
+            DynamicClassLoader prev = pathSpecificCls.putIfAbsent(jarPath, cl);
+            if (prev != null) {
+                cl = prev;
+            }
+            return cl;
+        }
+    }
+    
+    @VisibleForTesting
+    public static void setConfig(Configuration conf) {
+        config = conf;
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/CloneExpressionVisitor.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/CloneExpressionVisitor.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/CloneExpressionVisitor.java
index e6ede7c..b252358 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/CloneExpressionVisitor.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/CloneExpressionVisitor.java
@@ -47,6 +47,7 @@ import org.apache.phoenix.expression.function.ArrayAnyComparisonExpression;
 import org.apache.phoenix.expression.function.ArrayElemRefExpression;
 import org.apache.phoenix.expression.function.ScalarFunction;
 import org.apache.phoenix.expression.function.SingleAggregateFunction;
+import org.apache.phoenix.expression.function.UDFExpression;
 
 public class CloneExpressionVisitor extends TraverseAllExpressionVisitor<Expression> {
 
@@ -100,6 +101,11 @@ public class CloneExpressionVisitor extends TraverseAllExpressionVisitor<Express
         return Determinism.PER_INVOCATION.compareTo(node.getDeterminism()) > 0 ? node :  node.clone(l);
     }
 
+    public Expression visitLeave(UDFExpression node, List<Expression> l) {
+        return new UDFExpression(l, node.getTenantId(), node.getFunctionClassName(),
+                node.getJarPath(), node.getUdfFunction());
+    }
+
     @Override
     public Expression visitLeave(ComparisonExpression node, List<Expression> l) {
         return Determinism.PER_INVOCATION.compareTo(node.getDeterminism()) > 0 ? node :  node.clone(l);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java b/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
index 4f785eb..4565f39 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
@@ -24,9 +24,11 @@ import java.io.DataOutput;
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.sql.SQLException;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -57,8 +59,15 @@ import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.hbase.index.util.KeyValueBuilder;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixStatement;
+import org.apache.phoenix.parse.AndParseNode;
+import org.apache.phoenix.parse.BaseParseNodeVisitor;
+import org.apache.phoenix.parse.BooleanParseNodeVisitor;
+import org.apache.phoenix.parse.FunctionParseNode;
 import org.apache.phoenix.parse.ParseNode;
 import org.apache.phoenix.parse.SQLParser;
+import org.apache.phoenix.parse.StatelessTraverseAllParseNodeVisitor;
+import org.apache.phoenix.parse.TraverseAllParseNodeVisitor;
+import org.apache.phoenix.parse.UDFParseNode;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.ColumnNotFoundException;
 import org.apache.phoenix.schema.PColumn;
@@ -328,8 +337,21 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
         this.immutableRows = dataTable.isImmutableRows();
         int indexColByteSize = 0;
         ColumnResolver resolver = null;
+        List<ParseNode> parseNodes = new ArrayList<ParseNode>(1);
+        UDFParseNodeVisitor visitor = new UDFParseNodeVisitor();
+        for (int i = indexPosOffset; i < index.getPKColumns().size(); i++) {
+            PColumn indexColumn = index.getPKColumns().get(i);
+            String expressionStr = IndexUtil.getIndexColumnExpressionStr(indexColumn);
+            try {
+                ParseNode parseNode  = SQLParser.parseCondition(expressionStr);
+                parseNode.accept(visitor);
+                parseNodes.add(parseNode);
+            } catch (SQLException e) {
+                throw new RuntimeException(e);
+            }
+        }
         try {
-            resolver = FromCompiler.getResolver(new TableRef(dataTable));
+            resolver = FromCompiler.getResolver(connection, new TableRef(dataTable), visitor.getUdfParseNodes());
         } catch (SQLException e) {
             throw new RuntimeException(e); // Impossible
         }
@@ -341,9 +363,7 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
             Expression expression = null;
             try {
                 expressionIndexCompiler.reset();
-                String expressionStr = IndexUtil.getIndexColumnExpressionStr(indexColumn);
-                ParseNode parseNode  = SQLParser.parseCondition(expressionStr);
-                expression = parseNode.accept(expressionIndexCompiler);
+                expression = parseNodes.get(indexPos).accept(expressionIndexCompiler);
             } catch (SQLException e) {
                 throw new RuntimeException(e); // Impossible
             }
@@ -861,7 +881,7 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
             }
         }
         return delete;
-  }
+    }
 
     public byte[] getIndexTableName() {
         return indexTableName;
@@ -1337,4 +1357,24 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
     public Set<ColumnReference> getIndexedColumns() {
         return indexedColumns;
     }
+
+    public static class UDFParseNodeVisitor extends StatelessTraverseAllParseNodeVisitor {
+
+        private Map<String, UDFParseNode> udfParseNodes;
+        public UDFParseNodeVisitor() {
+            udfParseNodes = new HashMap<String, UDFParseNode>(1);
+        }
+
+        @Override
+        public boolean visitEnter(FunctionParseNode node) throws SQLException {
+            if(node instanceof UDFParseNode) {
+                udfParseNodes.put(node.getName(), (UDFParseNode)node);
+            }
+            return super.visitEnter(node);
+        }
+        
+        public Map<String, UDFParseNode> getUdfParseNodes() {
+            return udfParseNodes;
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/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
index b4a5c53..c22a7fa 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
@@ -64,6 +64,7 @@ import org.apache.phoenix.execute.MutationState;
 import org.apache.phoenix.expression.function.FunctionArgumentType;
 import org.apache.phoenix.hbase.index.util.KeyValueBuilder;
 import org.apache.phoenix.jdbc.PhoenixStatement.PhoenixStatementParser;
+import org.apache.phoenix.parse.PFunction;
 import org.apache.phoenix.query.ConnectionQueryServices;
 import org.apache.phoenix.query.DelegateConnectionQueryServices;
 import org.apache.phoenix.query.MetaDataMutated;
@@ -118,7 +119,7 @@ import com.google.common.collect.Maps;
  * 
  * @since 0.1
  */
-public class PhoenixConnection implements Connection, org.apache.phoenix.jdbc.Jdbc7Shim.Connection, MetaDataMutated  {
+public class PhoenixConnection implements Connection, org.apache.phoenix.jdbc.Jdbc7Shim.Connection, MetaDataMutated{
     private final String url;
     private final ConnectionQueryServices services;
     private final Properties info;
@@ -233,7 +234,7 @@ public class PhoenixConnection implements Connection, org.apache.phoenix.jdbc.Jd
         formatters.put(PDecimal.INSTANCE, FunctionArgumentType.NUMERIC.getFormatter(numberPattern));
         // We do not limit the metaData on a connection less than the global one,
         // as there's not much that will be cached here.
-        this.metaData = metaData.pruneTables(new Pruner() {
+        Pruner pruner = new Pruner() {
 
             @Override
             public boolean prune(PTable table) {
@@ -243,8 +244,16 @@ public class PhoenixConnection implements Connection, org.apache.phoenix.jdbc.Jd
                          ! Objects.equal(tenantId, table.getTenantId())) );
             }
             
-        });
+            @Override
+            public boolean prune(PFunction function) {
+                long maxTimestamp = scn == null ? HConstants.LATEST_TIMESTAMP : scn;
+                return ( function.getTimeStamp() >= maxTimestamp ||
+                         ! Objects.equal(tenantId, function.getTenantId()));
+            }
+        };
         this.mutationState = newMutationState(maxSize);
+        this.metaData = metaData.pruneTables(pruner);
+        this.metaData = metaData.pruneFunctions(pruner);
         this.services.addConnection(this);
 
         // setup tracing, if its enabled
@@ -777,6 +786,18 @@ public class PhoenixConnection implements Connection, org.apache.phoenix.jdbc.Jd
         getQueryServices().addTable(table);
         return metaData;
     }
+    
+    @Override
+    public PMetaData addFunction(PFunction function) 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 > function.getTimeStamp()) {
+            metaData = metaData.addFunction(function);
+        }
+        //Cascade through to connectionQueryServices too
+        getQueryServices().addFunction(function);
+        return metaData;
+    }
 
     @Override
     public PMetaData addColumn(PName tenantId, String tableName, List<PColumn> columns, long tableTimeStamp, long tableSeqNum, boolean isImmutableRows, boolean isWalDisabled, boolean isMultitenant, boolean storeNulls)
@@ -796,6 +817,14 @@ public class PhoenixConnection implements Connection, org.apache.phoenix.jdbc.Jd
     }
 
     @Override
+    public PMetaData removeFunction(PName tenantId, String functionName, long tableTimeStamp) throws SQLException {
+        metaData = metaData.removeFunction(tenantId, functionName, tableTimeStamp);
+        //Cascade through to connectionQueryServices too
+        getQueryServices().removeFunction(tenantId, functionName, tableTimeStamp);
+        return metaData;
+    }
+
+    @Override
     public PMetaData removeColumn(PName tenantId, String tableName, List<PColumn> columnsToRemove, long tableTimeStamp,
             long tableSeqNum) throws SQLException {
         metaData = metaData.removeColumn(tenantId, tableName, columnsToRemove, tableTimeStamp, tableSeqNum);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
index 1b8b57d..3a0b03b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
@@ -106,6 +106,11 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData, org.apache.pho
     public static final int SCHEMA_NAME_INDEX = 1;
     public static final int TENANT_ID_INDEX = 0;
 
+
+    public static final int TYPE_INDEX = 2;
+    public static final int FUNTION_NAME_INDEX = 1;
+    
+
     public static final String SYSTEM_CATALOG_SCHEMA = QueryConstants.SYSTEM_SCHEMA_NAME;
     public static final byte[] SYSTEM_CATALOG_SCHEMA_BYTES = QueryConstants.SYSTEM_SCHEMA_NAME_BYTES;
     public static final String SYSTEM_CATALOG_TABLE = "CATALOG";
@@ -209,6 +214,31 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData, org.apache.pho
     public static final byte[] TABLE_FAMILY_BYTES = QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES;
 
     public static final String TYPE_SEQUENCE = "SEQUENCE";
+    public static final String SYSTEM_FUNCTION_TABLE = "FUNCTION";
+    public static final String SYSTEM_FUNCTION_NAME = SchemaUtil.getTableName(SYSTEM_CATALOG_SCHEMA, SYSTEM_FUNCTION_TABLE);
+    public static final byte[] SYSTEM_FUNCTION_NAME_BYTES = Bytes.toBytes(SYSTEM_FUNCTION_NAME);
+
+    public static final String FUNCTION_NAME = "FUNCTION_NAME";
+    public static final byte[] FUNCTION_NAME_BYTES = Bytes.toBytes(FUNCTION_NAME);
+    public static final String CLASS_NAME = "CLASS_NAME";
+    public static final byte[] CLASS_NAME_BYTES = Bytes.toBytes(CLASS_NAME);
+    public static final String JAR_PATH = "JAR_PATH";
+    public static final byte[] JAR_PATH_BYTES = Bytes.toBytes(JAR_PATH);
+    public static final String TYPE = "TYPE";
+    public static final byte[] TYPE_BYTES = Bytes.toBytes(TYPE);
+    public static final String ARG_POSITION = "ARG_POSITION";
+    public static final byte[] ARG_POSITION_TYPE = Bytes.toBytes(ARG_POSITION);
+    public static final String RETURN_TYPE = "RETURN_TYPE";
+    public static final byte[] RETURN_TYPE_BYTES = Bytes.toBytes(RETURN_TYPE);
+    public static final String IS_ARRAY = "IS_ARRAY";
+    public static final byte[] IS_ARRAY_BYTES = Bytes.toBytes(IS_ARRAY);
+    public static final String IS_CONSTANT = "IS_CONSTANT";
+    public static final byte[] IS_CONSTANT_BYTES = Bytes.toBytes(IS_CONSTANT);
+    public static final String DEFAULT_VALUE = "DEFAULT_VALUE";
+    public static final byte[] DEFAULT_VALUE_BYTES = Bytes.toBytes(DEFAULT_VALUE);
+    public static final String NUM_ARGS = "NUM_ARGS";
+    public static final byte[] NUM_ARGS_BYTES = Bytes.toBytes(NUM_ARGS);
+    
     public static final byte[] SEQUENCE_FAMILY_BYTES = QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES;
     public static final String SEQUENCE_SCHEMA_NAME = SYSTEM_CATALOG_SCHEMA;
     public static final byte[] SEQUENCE_SCHEMA_NAME_BYTES = Bytes.toBytes(SEQUENCE_SCHEMA_NAME);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/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
index a70a36e..7c94d62 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.call.CallRunner;
 import org.apache.phoenix.compile.ColumnProjector;
 import org.apache.phoenix.compile.ColumnResolver;
+import org.apache.phoenix.compile.CreateFunctionCompiler;
 import org.apache.phoenix.compile.CreateIndexCompiler;
 import org.apache.phoenix.compile.CreateSequenceCompiler;
 import org.apache.phoenix.compile.CreateTableCompiler;
@@ -78,11 +79,13 @@ import org.apache.phoenix.parse.AlterSessionStatement;
 import org.apache.phoenix.parse.BindableStatement;
 import org.apache.phoenix.parse.ColumnDef;
 import org.apache.phoenix.parse.ColumnName;
+import org.apache.phoenix.parse.CreateFunctionStatement;
 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.DropFunctionStatement;
 import org.apache.phoenix.parse.DropIndexStatement;
 import org.apache.phoenix.parse.DropSequenceStatement;
 import org.apache.phoenix.parse.DropTableStatement;
@@ -94,6 +97,7 @@ 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.PFunction;
 import org.apache.phoenix.parse.ParseNode;
 import org.apache.phoenix.parse.ParseNodeFactory;
 import org.apache.phoenix.parse.PrimaryKeyConstraint;
@@ -102,6 +106,7 @@ import org.apache.phoenix.parse.SelectStatement;
 import org.apache.phoenix.parse.TableName;
 import org.apache.phoenix.parse.TableNode;
 import org.apache.phoenix.parse.TraceStatement;
+import org.apache.phoenix.parse.UDFParseNode;
 import org.apache.phoenix.parse.UpdateStatisticsStatement;
 import org.apache.phoenix.parse.UpsertStatement;
 import org.apache.phoenix.query.KeyRange;
@@ -332,19 +337,22 @@ public class PhoenixStatement implements Statement, SQLCloseable, org.apache.pho
     
     private static class ExecutableSelectStatement extends SelectStatement implements CompilableStatement {
         private ExecutableSelectStatement(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, boolean hasSequence) {
-            this(from, hint, isDistinct, select, where, groupBy, having, orderBy, limit, bindCount, isAggregate, hasSequence, Collections.<SelectStatement>emptyList());
+                List<ParseNode> groupBy, ParseNode having, List<OrderByNode> orderBy, LimitNode limit, int bindCount, boolean isAggregate, boolean hasSequence, Map<String, UDFParseNode> udfParseNodes) {
+            this(from, hint, isDistinct, select, where, groupBy, having, orderBy, limit, bindCount, isAggregate, hasSequence, Collections.<SelectStatement>emptyList(), udfParseNodes);
         }
 
         private ExecutableSelectStatement(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,
-                boolean hasSequence, List<SelectStatement> selects) {
-            super(from, hint, isDistinct, select, where, groupBy, having, orderBy, limit, bindCount, isAggregate, hasSequence, selects);
+                boolean hasSequence, List<SelectStatement> selects, Map<String, UDFParseNode> udfParseNodes) {
+            super(from, hint, isDistinct, select, where, groupBy, having, orderBy, limit, bindCount, isAggregate, hasSequence, selects, udfParseNodes);
         }
         
         @SuppressWarnings("unchecked")
         @Override
         public QueryPlan compilePlan(PhoenixStatement stmt, Sequence.ValueOp seqAction) throws SQLException {
+            if(!getUdfParseNodes().isEmpty()) {
+                stmt.throwIfUnallowedUserDefinedFunctions();
+            }
             SelectStatement select = SubselectRewriter.flatten(this, stmt.getConnection());
             ColumnResolver resolver = FromCompiler.getResolverForQuery(select, stmt.getConnection());
             select = StatementNormalizer.normalize(select, resolver);
@@ -357,6 +365,7 @@ public class PhoenixStatement implements Statement, SQLCloseable, org.apache.pho
             plan.getContext().getSequenceManager().validateSequences(seqAction);
             return plan;
         }
+
     }
     
     private static final byte[] EXPLAIN_PLAN_FAMILY = QueryConstants.SINGLE_COLUMN_FAMILY;
@@ -505,13 +514,16 @@ public class PhoenixStatement implements Statement, SQLCloseable, org.apache.pho
     }
 
     private static class ExecutableUpsertStatement extends UpsertStatement implements CompilableStatement {
-        private ExecutableUpsertStatement(NamedTableNode table, HintNode hintNode, List<ColumnName> columns, List<ParseNode> values, SelectStatement select, int bindCount) {
-            super(table, hintNode, columns, values, select, bindCount);
+        private ExecutableUpsertStatement(NamedTableNode table, HintNode hintNode, List<ColumnName> columns, List<ParseNode> values, SelectStatement select, int bindCount, Map<String, UDFParseNode> udfParseNodes) {
+            super(table, hintNode, columns, values, select, bindCount, udfParseNodes);
         }
 
         @SuppressWarnings("unchecked")
         @Override
         public MutationPlan compilePlan(PhoenixStatement stmt, Sequence.ValueOp seqAction) throws SQLException {
+            if(!getUdfParseNodes().isEmpty()) {
+                stmt.throwIfUnallowedUserDefinedFunctions();
+            }
             UpsertCompiler compiler = new UpsertCompiler(stmt);
             MutationPlan plan = compiler.compile(this);
             plan.getContext().getSequenceManager().validateSequences(seqAction);
@@ -520,13 +532,16 @@ public class PhoenixStatement implements Statement, SQLCloseable, org.apache.pho
     }
     
     private static class ExecutableDeleteStatement extends DeleteStatement implements CompilableStatement {
-        private ExecutableDeleteStatement(NamedTableNode table, HintNode hint, ParseNode whereNode, List<OrderByNode> orderBy, LimitNode limit, int bindCount) {
-            super(table, hint, whereNode, orderBy, limit, bindCount);
+        private ExecutableDeleteStatement(NamedTableNode table, HintNode hint, ParseNode whereNode, List<OrderByNode> orderBy, LimitNode limit, int bindCount, Map<String, UDFParseNode> udfParseNodes) {
+            super(table, hint, whereNode, orderBy, limit, bindCount, udfParseNodes);
         }
 
         @SuppressWarnings("unchecked")
         @Override
         public MutationPlan compilePlan(PhoenixStatement stmt, Sequence.ValueOp seqAction) throws SQLException {
+            if(!getUdfParseNodes().isEmpty()) {
+                stmt.throwIfUnallowedUserDefinedFunctions();
+            }
             DeleteCompiler compiler = new DeleteCompiler(stmt);
             MutationPlan plan = compiler.compile(this);
             plan.getContext().getSequenceManager().validateSequences(seqAction);
@@ -549,16 +564,76 @@ public class PhoenixStatement implements Statement, SQLCloseable, org.apache.pho
         }
     }
 
+    private static class ExecutableCreateFunctionStatement extends CreateFunctionStatement implements CompilableStatement {
+
+        public ExecutableCreateFunctionStatement(PFunction functionInfo, boolean temporary) {
+            super(functionInfo, temporary);
+        }
+
+
+        @SuppressWarnings("unchecked")
+        @Override
+        public MutationPlan compilePlan(final PhoenixStatement stmt, Sequence.ValueOp seqAction) throws SQLException {
+            stmt.throwIfUnallowedUserDefinedFunctions();
+            CreateFunctionCompiler compiler = new CreateFunctionCompiler(stmt);
+            return compiler.compile(this);
+        }
+    }
+
+    private static class ExecutableDropFunctionStatement extends DropFunctionStatement implements CompilableStatement {
+
+        public ExecutableDropFunctionStatement(String functionName, boolean ifNotExists) {
+            super(functionName, ifNotExists);
+        }
+
+        @SuppressWarnings("unchecked")
+        @Override
+        public MutationPlan compilePlan(final PhoenixStatement stmt, Sequence.ValueOp seqAction) throws SQLException {
+                final StatementContext context = new StatementContext(stmt);
+                return new MutationPlan() {
+
+                    @Override
+                    public StatementContext getContext() {
+                        return context;
+                    }
+
+                    @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 PhoenixConnection getConnection() {
+                        return stmt.getConnection();
+                    }
+
+                    @Override
+                    public MutationState execute() throws SQLException {
+                        MetaDataClient client = new MetaDataClient(getConnection());
+                        return client.dropFunction(ExecutableDropFunctionStatement.this);
+                    }
+                };
+        }
+    }
+    
     private static class ExecutableCreateIndexStatement extends CreateIndexStatement implements CompilableStatement {
 
         public ExecutableCreateIndexStatement(NamedNode indexName, NamedTableNode dataTable, IndexKeyConstraint ikConstraint, List<ColumnName> includeColumns, List<ParseNode> splits,
-                ListMultimap<String,Pair<String,Object>> props, boolean ifNotExists, IndexType indexType, boolean async, int bindCount) {
-            super(indexName, dataTable, ikConstraint, includeColumns, splits, props, ifNotExists, indexType, async , bindCount);
+                ListMultimap<String,Pair<String,Object>> props, boolean ifNotExists, IndexType indexType, boolean async, int bindCount, Map<String, UDFParseNode> udfParseNodes) {
+            super(indexName, dataTable, ikConstraint, includeColumns, splits, props, ifNotExists, indexType, async , bindCount, udfParseNodes);
         }
 
         @SuppressWarnings("unchecked")
         @Override
         public MutationPlan compilePlan(PhoenixStatement stmt, Sequence.ValueOp seqAction) throws SQLException {
+            if(!getUdfParseNodes().isEmpty()) {
+                stmt.throwIfUnallowedUserDefinedFunctions();
+            }
             CreateIndexCompiler compiler = new CreateIndexCompiler(stmt);
             return compiler.compile(this);
         }
@@ -908,19 +983,19 @@ public class PhoenixStatement implements Statement, SQLCloseable, org.apache.pho
         @Override
         public ExecutableSelectStatement select(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,
-                boolean hasSequence, List<SelectStatement> selects) {
+                boolean hasSequence, List<SelectStatement> selects, Map<String, UDFParseNode> udfParseNodes) {
             return new ExecutableSelectStatement(from, hint, isDistinct, select, where, groupBy == null ? Collections.<ParseNode>emptyList() : groupBy,
-                    having, orderBy == null ? Collections.<OrderByNode>emptyList() : orderBy, limit, bindCount, isAggregate, hasSequence, selects == null ? Collections.<SelectStatement>emptyList() : selects);
+                    having, orderBy == null ? Collections.<OrderByNode>emptyList() : orderBy, limit, bindCount, isAggregate, hasSequence, selects == null ? Collections.<SelectStatement>emptyList() : selects, udfParseNodes);
         }
 
         @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);
+        public ExecutableUpsertStatement upsert(NamedTableNode table, HintNode hintNode, List<ColumnName> columns, List<ParseNode> values, SelectStatement select, int bindCount, Map<String, UDFParseNode> udfParseNodes) {
+            return new ExecutableUpsertStatement(table, hintNode, columns, values, select, bindCount, udfParseNodes);
         }
         
         @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);
+        public ExecutableDeleteStatement delete(NamedTableNode table, HintNode hint, ParseNode whereNode, List<OrderByNode> orderBy, LimitNode limit, int bindCount, Map<String, UDFParseNode> udfParseNodes) {
+            return new ExecutableDeleteStatement(table, hint, whereNode, orderBy, limit, bindCount, udfParseNodes);
         }
         
         @Override
@@ -938,14 +1013,18 @@ public class PhoenixStatement implements Statement, SQLCloseable, org.apache.pho
         }
         
         @Override
+        public CreateFunctionStatement createFunction(PFunction functionInfo, boolean temporary) {
+            return new ExecutableCreateFunctionStatement(functionInfo, temporary);
+        }
+        @Override
         public DropSequenceStatement dropSequence(TableName tableName, boolean ifExists, int bindCount){
             return new ExecutableDropSequenceStatement(tableName, ifExists, bindCount);
         }
         
         @Override
         public CreateIndexStatement createIndex(NamedNode indexName, NamedTableNode dataTable, IndexKeyConstraint ikConstraint, List<ColumnName> includeColumns, List<ParseNode> splits,
-                ListMultimap<String,Pair<String,Object>> props, boolean ifNotExists, IndexType indexType, boolean async, int bindCount) {
-            return new ExecutableCreateIndexStatement(indexName, dataTable, ikConstraint, includeColumns, splits, props, ifNotExists, indexType, async, bindCount);
+                ListMultimap<String,Pair<String,Object>> props, boolean ifNotExists, IndexType indexType, boolean async, int bindCount, Map<String, UDFParseNode> udfParseNodes) {
+            return new ExecutableCreateIndexStatement(indexName, dataTable, ikConstraint, includeColumns, splits, props, ifNotExists, indexType, async, bindCount, udfParseNodes);
         }
         
         @Override
@@ -962,6 +1041,11 @@ public class PhoenixStatement implements Statement, SQLCloseable, org.apache.pho
         public DropTableStatement dropTable(TableName tableName, PTableType tableType, boolean ifExists, boolean cascade) {
             return new ExecutableDropTableStatement(tableName, tableType, ifExists, cascade);
         }
+
+        @Override
+        public DropFunctionStatement dropFunction(String functionName, boolean ifExists) {
+            return new ExecutableDropFunctionStatement(functionName, ifExists);
+        }
         
         @Override
         public DropIndexStatement dropIndex(NamedNode indexName, TableName tableName, boolean ifExists) {
@@ -1410,4 +1494,16 @@ public class PhoenixStatement implements Statement, SQLCloseable, org.apache.pho
     private void setLastQueryPlan(QueryPlan lastQueryPlan) {
         this.lastQueryPlan = lastQueryPlan;
     }
+    
+    private void throwIfUnallowedUserDefinedFunctions() throws SQLException {
+        if (!connection
+                .getQueryServices()
+                .getProps()
+                .getBoolean(QueryServices.ALLOW_USER_DEFINED_FUNCTIONS_ATTRIB,
+                    QueryServicesOptions.DEFAULT_ALLOW_USER_DEFINED_FUNCTIONS)) {
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode.UNALLOWED_USER_DEFINED_FUNCTIONS)
+                    .build().buildException();
+        }
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/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
index 7b3a63a..99ca46e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/optimize/QueryOptimizer.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/optimize/QueryOptimizer.java
@@ -235,7 +235,7 @@ public class QueryOptimizer {
         if (PIndexState.ACTIVE.equals(resolver.getTables().get(0).getTable().getIndexState())) {
             try {
             	// translate nodes that match expressions that are indexed to the associated column parse node
-                indexSelect = ParseNodeRewriter.rewrite(indexSelect, new  IndexExpressionParseNodeRewriter(index, statement.getConnection()));
+                indexSelect = ParseNodeRewriter.rewrite(indexSelect, new  IndexExpressionParseNodeRewriter(index, statement.getConnection(), indexSelect.getUdfParseNodes()));
                 QueryCompiler compiler = new QueryCompiler(statement, indexSelect, resolver, targetColumns, parallelIteratorFactory, dataPlan.getContext().getSequenceManager(), isProjected);
                 
                 QueryPlan plan = compiler.compile();
@@ -290,7 +290,7 @@ public class QueryOptimizer {
                             aliasedNodes.add(FACTORY.aliasedNode(null, indexColNode));
                             nodes.add(new ColumnParseNode(null, '"' + column.getName().getString() + '"'));
                         }
-                        SelectStatement innerSelect = FACTORY.select(indexSelect.getFrom(), indexSelect.getHint(), false, aliasedNodes, where, null, null, null, indexSelect.getLimit(), indexSelect.getBindCount(), false, indexSelect.hasSequence(), Collections.<SelectStatement>emptyList());
+                        SelectStatement innerSelect = FACTORY.select(indexSelect.getFrom(), indexSelect.getHint(), false, aliasedNodes, where, null, null, null, indexSelect.getLimit(), indexSelect.getBindCount(), false, indexSelect.hasSequence(), Collections.<SelectStatement>emptyList(), indexSelect.getUdfParseNodes());
                         ParseNode outerWhere = FACTORY.in(nodes.size() == 1 ? nodes.get(0) : FACTORY.rowValueConstructor(nodes), FACTORY.subquery(innerSelect, false), false, true);
                         ParseNode extractedCondition = whereRewriter.getExtractedCondition();
                         if (extractedCondition != null) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/phoenix-core/src/main/java/org/apache/phoenix/parse/CreateFunctionStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/CreateFunctionStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/CreateFunctionStatement.java
new file mode 100644
index 0000000..741e4df
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/CreateFunctionStatement.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 CreateFunctionStatement extends MutableStatement {
+    private final PFunction functionInfo;
+    private final boolean temporary;
+
+    public CreateFunctionStatement(PFunction functionInfo, boolean temporary) {
+        this.functionInfo = functionInfo;
+        this.temporary = temporary;
+    }
+
+    @Override
+    public int getBindCount() {
+        return 0;
+    }
+
+    public PFunction getFunctionInfo() {
+        return functionInfo;
+    }
+    
+    public boolean isTemporary() {
+        return temporary;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/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
index 5f52f59..6da0ff3 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/CreateIndexStatement.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/CreateIndexStatement.java
@@ -19,6 +19,7 @@ package org.apache.phoenix.parse;
 
 import java.util.Collections;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.schema.PTable.IndexType;
@@ -36,10 +37,11 @@ public class CreateIndexStatement extends SingleTableStatement {
     private final boolean ifNotExists;
     private final IndexType indexType;
     private final boolean async;
+    private final Map<String, UDFParseNode> udfParseNodes;
 
     public CreateIndexStatement(NamedNode indexTableName, NamedTableNode dataTable, 
             IndexKeyConstraint indexKeyConstraint, List<ColumnName> includeColumns, List<ParseNode> splits,
-            ListMultimap<String,Pair<String,Object>> props, boolean ifNotExists, IndexType indexType, boolean async, int bindCount) {
+            ListMultimap<String,Pair<String,Object>> props, boolean ifNotExists, IndexType indexType, boolean async, int bindCount, Map<String, UDFParseNode> udfParseNodes) {
         super(dataTable, bindCount);
         this.indexTableName =TableName.create(dataTable.getName().getSchemaName(),indexTableName.getName());
         this.indexKeyConstraint = indexKeyConstraint == null ? IndexKeyConstraint.EMPTY : indexKeyConstraint;
@@ -49,6 +51,7 @@ public class CreateIndexStatement extends SingleTableStatement {
         this.ifNotExists = ifNotExists;
         this.indexType = indexType;
         this.async = async;
+        this.udfParseNodes = udfParseNodes;
     }
 
     public IndexKeyConstraint getIndexConstraint() {
@@ -84,4 +87,7 @@ public class CreateIndexStatement extends SingleTableStatement {
         return async;
     }
 
+    public Map<String, UDFParseNode> getUdfParseNodes() {
+        return udfParseNodes;
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/phoenix-core/src/main/java/org/apache/phoenix/parse/DMLStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/DMLStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/DMLStatement.java
index 46bd907..3b9bd97 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/DMLStatement.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/DMLStatement.java
@@ -17,11 +17,18 @@
  */
 package org.apache.phoenix.parse;
 
+import java.util.Map;
+
 public class DMLStatement extends SingleTableStatement {
 
-    public DMLStatement(NamedTableNode table, int bindCount) {
+    private final Map<String, UDFParseNode> udfParseNodes;
+    
+    public DMLStatement(NamedTableNode table, int bindCount, Map<String, UDFParseNode> udfParseNodes) {
         super(table, bindCount);
+        this.udfParseNodes = udfParseNodes;
     }
     
-
+    public Map<String, UDFParseNode> getUdfParseNodes() {
+        return udfParseNodes;
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/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
index d295e85..276e6aa 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/DeleteStatement.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/DeleteStatement.java
@@ -19,6 +19,7 @@ package org.apache.phoenix.parse;
 
 import java.util.Collections;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.phoenix.jdbc.PhoenixStatement.Operation;
 
@@ -28,8 +29,8 @@ public class DeleteStatement extends DMLStatement implements FilterableStatement
     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);
+    public DeleteStatement(NamedTableNode table, HintNode hint, ParseNode whereNode, List<OrderByNode> orderBy, LimitNode limit, int bindCount, Map<String, UDFParseNode> udfParseNodes) {
+        super(table, bindCount, udfParseNodes);
         this.whereNode = whereNode;
         this.orderBy = orderBy == null ? Collections.<OrderByNode>emptyList() : orderBy;
         this.limit = limit;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/phoenix-core/src/main/java/org/apache/phoenix/parse/DropFunctionStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/DropFunctionStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/DropFunctionStatement.java
new file mode 100644
index 0000000..a959eb7
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/DropFunctionStatement.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 DropFunctionStatement extends MutableStatement {
+
+    private final String functionName;
+    private final boolean ifExists;
+    public DropFunctionStatement(String functionName, boolean ifExists) {
+        this.functionName = functionName;
+        this.ifExists = ifExists;
+    }
+
+    @Override
+    public int getBindCount() {
+        return 0;
+    }
+
+    public String getFunctionName() {
+        return functionName;
+    }
+
+    public boolean ifExists() {
+        return ifExists;
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/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
index 9764f52..bd13fb5 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/FunctionParseNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/FunctionParseNode.java
@@ -38,6 +38,9 @@ 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.expression.function.UDFExpression;
+import org.apache.phoenix.jdbc.PhoenixStatement;
+import org.apache.phoenix.parse.PFunction.FunctionArgument;
 import org.apache.phoenix.schema.ArgumentTypeMismatchException;
 import org.apache.phoenix.schema.ValueRangeExcpetion;
 import org.apache.phoenix.schema.types.PDataType;
@@ -58,7 +61,7 @@ import com.google.common.collect.ImmutableSet;
  */
 public class FunctionParseNode extends CompoundParseNode {
     private final String name;
-    private final BuiltInFunctionInfo info;
+    private BuiltInFunctionInfo info;
 
     FunctionParseNode(String name, List<ParseNode> children, BuiltInFunctionInfo info) {
         super(children);
@@ -84,6 +87,7 @@ public class FunctionParseNode extends CompoundParseNode {
     }
 
     public boolean isAggregate() {
+        if(getInfo()==null) return false;
         return getInfo().isAggregate();
     }
 
@@ -113,9 +117,17 @@ public class FunctionParseNode extends CompoundParseNode {
     }
 
     private static Constructor<? extends FunctionExpression> getExpressionCtor(Class<? extends FunctionExpression> clazz) {
+        return getExpressionCtor(clazz, null);
+    }
+
+    private static Constructor<? extends FunctionExpression> getExpressionCtor(Class<? extends FunctionExpression> clazz, PFunction function) {
         Constructor<? extends FunctionExpression> ctor;
         try {
-            ctor = clazz.getDeclaredConstructor(List.class);
+            if(function == null) {
+                ctor = clazz.getDeclaredConstructor(List.class);
+            } else {
+                ctor = clazz.getDeclaredConstructor(List.class, PFunction.class);
+            }
         } catch (Exception e) {
             throw new RuntimeException(e);
         }
@@ -223,8 +235,24 @@ public class FunctionParseNode extends CompoundParseNode {
      * @throws SQLException
      */
     public Expression create(List<Expression> children, StatementContext context) throws SQLException {
+        return create(children, null, context);
+    }
+
+    /**
+     * Entry point for parser to instantiate compiled representation of built-in function
+     * @param children Compiled expressions for child nodes
+     * @param function
+     * @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, PFunction function, StatementContext context) throws SQLException {
         try {
-            return info.getFuncCtor().newInstance(children);
+            if(function == null) {
+                return info.getFuncCtor().newInstance(children);
+            } else {
+                return info.getFuncCtor().newInstance(children, function);
+            }
         } catch (InstantiationException e) {
             throw new SQLException(e);
         } catch (IllegalAccessException e) {
@@ -275,9 +303,9 @@ public class FunctionParseNode extends CompoundParseNode {
         private final boolean isAggregate;
         private final int requiredArgCount;
 
-        BuiltInFunctionInfo(Class<? extends FunctionExpression> f, BuiltInFunction d) {
+        public BuiltInFunctionInfo(Class<? extends FunctionExpression> f, BuiltInFunction d) {
             this.name = SchemaUtil.normalizeIdentifier(d.name());
-            this.funcCtor = d.nodeClass() == FunctionParseNode.class ? getExpressionCtor(f) : null;
+            this.funcCtor = d.nodeClass() == FunctionParseNode.class ? getExpressionCtor(f, null) : null;
             this.nodeCtor = d.nodeClass() == FunctionParseNode.class ? null : getParseNodeCtor(d.nodeClass());
             this.args = new BuiltInFunctionArgInfo[d.args().length];
             int requiredArgCount = 0;
@@ -291,6 +319,22 @@ public class FunctionParseNode extends CompoundParseNode {
             this.isAggregate = AggregateFunction.class.isAssignableFrom(f);
         }
 
+        public BuiltInFunctionInfo(PFunction function) {
+            this.name = SchemaUtil.normalizeIdentifier(function.getFunctionName());
+            this.funcCtor = getExpressionCtor(UDFExpression.class, function);
+            this.nodeCtor = getParseNodeCtor(UDFParseNode.class);
+            this.args = new BuiltInFunctionArgInfo[function.getFunctionArguments().size()];
+            int requiredArgCount = 0;
+            for (int i = 0; i < args.length; i++) {
+                this.args[i] = new BuiltInFunctionArgInfo(function.getFunctionArguments().get(i));
+                if (this.args[i].getDefaultValue() == null) {
+                    requiredArgCount = i + 1;
+                }
+            }
+            this.requiredArgCount = requiredArgCount;
+            this.isAggregate = AggregateFunction.class.isAssignableFrom(UDFExpression.class);
+        }
+
         public int getRequiredArgCount() {
             return requiredArgCount;
         }
@@ -365,6 +409,27 @@ public class FunctionParseNode extends CompoundParseNode {
             }
         }
 
+        @SuppressWarnings({ "unchecked", "rawtypes" })
+        BuiltInFunctionArgInfo(FunctionArgument arg) {
+            PDataType dataType =
+                    arg.isArrayType() ? PDataType.fromTypeId(PDataType.sqlArrayType(SchemaUtil
+                            .normalizeIdentifier(SchemaUtil.normalizeIdentifier(arg
+                                    .getArgumentType())))) : PDataType.fromSqlTypeName(SchemaUtil
+                            .normalizeIdentifier(arg.getArgumentType()));
+            this.allowedValues = Collections.emptySet();
+            this.allowedTypes = new Class[] { dataType.getClass() };
+            this.isConstant = arg.isConstant();
+            this.defaultValue =
+                    arg.getDefaultValue() == null ? null : getExpFromConstant((String) arg
+                            .getDefaultValue().getValue());
+            this.minValue =
+                    arg.getMinValue() == null ? null : getExpFromConstant((String) arg
+                            .getMinValue().getValue());
+            this.maxValue =
+                    arg.getMaxValue() == null ? null : getExpFromConstant((String) arg
+                            .getMaxValue().getValue());
+        }
+        
         private LiteralExpression getExpFromConstant(String strValue) {
             LiteralExpression exp = null;
             if (strValue.length() > 0) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexExpressionParseNodeRewriter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexExpressionParseNodeRewriter.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexExpressionParseNodeRewriter.java
index 0273041..9f7b2bf 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexExpressionParseNodeRewriter.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexExpressionParseNodeRewriter.java
@@ -37,12 +37,12 @@ public class IndexExpressionParseNodeRewriter extends ParseNodeRewriter {
 
     private final Map<ParseNode, ParseNode> indexedParseNodeToColumnParseNodeMap;
     
-    public IndexExpressionParseNodeRewriter(PTable index, PhoenixConnection connection) throws SQLException {
+    public IndexExpressionParseNodeRewriter(PTable index, PhoenixConnection connection, Map<String, UDFParseNode> udfParseNodes) throws SQLException {
         indexedParseNodeToColumnParseNodeMap = Maps.newHashMapWithExpectedSize(index.getColumns().size());
         NamedTableNode tableNode = NamedTableNode.create(null,
                 TableName.create(index.getParentSchemaName().getString(), index.getParentTableName().getString()),
                 Collections.<ColumnDef> emptyList());
-        ColumnResolver dataResolver = FromCompiler.getResolver(tableNode, connection);
+        ColumnResolver dataResolver = FromCompiler.getResolver(tableNode, connection, udfParseNodes);
         StatementContext context = new StatementContext(new PhoenixStatement(connection), dataResolver);
         IndexStatementRewriter rewriter = new IndexStatementRewriter(dataResolver, null);
         ExpressionCompiler expressionCompiler = new ExpressionCompiler(context);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/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
index 3f1becc..1957f0e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/NamedNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/NamedNode.java
@@ -27,7 +27,7 @@ public class NamedNode {
     public static NamedNode caseSensitiveNamedNode(String name) {
         return new NamedNode(name,true);
     }
-    
+
     NamedNode(String name, boolean isCaseSensitive) {
         this.name = name;
         this.isCaseSensitive = isCaseSensitive;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/66bd3e35/phoenix-core/src/main/java/org/apache/phoenix/parse/PFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/PFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/PFunction.java
new file mode 100644
index 0000000..351bec7
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/PFunction.java
@@ -0,0 +1,255 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.phoenix.coprocessor.generated.PFunctionProtos;
+import org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunctionArg;
+import org.apache.phoenix.expression.LiteralExpression;
+import org.apache.phoenix.schema.PMetaDataEntity;
+import org.apache.phoenix.schema.PName;
+import org.apache.phoenix.schema.PNameFactory;
+import org.apache.phoenix.schema.PTableKey;
+import org.apache.phoenix.util.SizedUtil;
+
+import com.google.protobuf.HBaseZeroCopyByteString;
+
+public class PFunction implements PMetaDataEntity {
+
+    private PName tenantId = null;
+    private final PName functionName;
+    private List<FunctionArgument> args;
+    private PName className;
+    private PName jarPath;
+    private PName returnType;
+    private PTableKey functionKey;
+    private long timeStamp;
+    private int estimatedSize;
+    private boolean temporary;
+
+    public PFunction(long timeStamp) { // For index delete marker
+        this.timeStamp = timeStamp;
+        this.args = Collections.emptyList();
+        this.functionName = null;
+    }
+
+    public PFunction(String functionName, List<FunctionArgument> args, String returnType,
+            String className, String jarPath) {
+        this(functionName,args,returnType,className, jarPath, HConstants.LATEST_TIMESTAMP);
+    }
+
+    public PFunction(String functionName, List<FunctionArgument> args, String returnType,
+            String className, String jarPath, long timeStamp) {
+        this(null, functionName, args, returnType, className, jarPath, timeStamp);
+    }    
+
+    public PFunction(PName tenantId, String functionName, List<FunctionArgument> args, String returnType,
+            String className, String jarPath, long timeStamp) {
+        this(tenantId, functionName, args, returnType, className, jarPath, timeStamp, false);
+    }
+    
+    public PFunction(PFunction function, boolean temporary) {
+        this(function.getTenantId(), function.getFunctionName(), function.getFunctionArguments(),
+                function.getReturnType(), function.getClassName(), function.getJarPath(), function
+                        .getTimeStamp(), temporary);
+    }
+
+    public PFunction(PName tenantId, String functionName, List<FunctionArgument> args, String returnType,
+            String className, String jarPath, long timeStamp, boolean temporary) {
+        this.tenantId = tenantId;
+        this.functionName = PNameFactory.newName(functionName);
+        if (args == null){ 
+            this.args = new ArrayList<FunctionArgument>();
+        } else {
+            this.args = args;
+        }
+        this.className = PNameFactory.newName(className);
+        this.jarPath = jarPath == null ? null : PNameFactory.newName(jarPath);
+        this.returnType = PNameFactory.newName(returnType);
+        this.functionKey = new PTableKey(this.tenantId, this.functionName.getString());
+        this.timeStamp = timeStamp;
+        int estimatedSize = SizedUtil.OBJECT_SIZE * 2 + 23 * SizedUtil.POINTER_SIZE + 4 * SizedUtil.INT_SIZE + 2 * SizedUtil.LONG_SIZE + 2 * SizedUtil.INT_OBJECT_SIZE +
+                PNameFactory.getEstimatedSize(tenantId) +
+                PNameFactory.getEstimatedSize(this.functionName) +
+                PNameFactory.getEstimatedSize(this.className) +
+                 (jarPath==null?0:PNameFactory.getEstimatedSize(this.jarPath));
+        this.temporary = temporary;
+    }
+
+    public PFunction(PFunction function) {
+        this(function.getTenantId(), function.getFunctionName(), function.getFunctionArguments(),
+                function.getReturnType(), function.getClassName(), function.getJarPath(), function
+                        .getTimeStamp());
+    }
+
+    public String getFunctionName() {
+        return functionName == null ? null : functionName.getString();
+    }
+
+    public List<FunctionArgument> getFunctionArguments() {
+        return args;
+    }
+
+    public String getClassName() {
+        return className.getString();
+    }
+
+    public String getJarPath() {
+        return jarPath == null ? null : jarPath.getString();
+    }
+
+    public String getReturnType() {
+        return returnType.getString();
+    }
+    
+    public PTableKey getKey() {
+        return this.functionKey;
+    }
+    
+    public long getTimeStamp() {
+        return this.timeStamp;
+    }
+    
+    public PName getTenantId() {
+        return this.tenantId;
+    }
+    
+    public boolean isTemporaryFunction() {
+        return temporary;
+    }
+    
+    public static class FunctionArgument {
+        private final PName argumentType;
+        private final boolean isArrayType;
+        private final boolean isConstant;
+        private final LiteralExpression defaultValue;
+        private final LiteralExpression minValue;
+        private final LiteralExpression maxValue;
+        private short argPosition;
+        
+        public FunctionArgument(String argumentType, boolean isArrayType, boolean isConstant, LiteralExpression defaultValue,
+                LiteralExpression minValue, LiteralExpression maxValue) {
+            this.argumentType = PNameFactory.newName(argumentType);
+            this.isArrayType = isArrayType;
+            this.isConstant = isConstant;
+            this.defaultValue = defaultValue;
+            this.minValue = minValue;
+            this.maxValue = maxValue;
+        }
+        public FunctionArgument(String argumentType, boolean isArrayType, boolean isConstant, LiteralExpression defaultValue,
+                LiteralExpression minValue, LiteralExpression maxValue, short argPosition) {
+            this(argumentType, isArrayType, isConstant, defaultValue, minValue, maxValue);
+            this.argPosition = argPosition;
+        }
+
+        public String getArgumentType() {
+            return argumentType.getString();
+        }
+
+        public boolean isConstant() {
+            return isConstant;
+        }
+
+        public boolean isArrayType() {
+            return isArrayType;
+        }
+
+        public LiteralExpression getDefaultValue() {
+            return defaultValue;
+        }
+
+        public LiteralExpression getMinValue() {
+            return minValue;
+        }
+
+        public LiteralExpression getMaxValue() {
+            return maxValue;
+        }
+        
+        public short getArgPosition() {
+            return argPosition;
+        }
+    }
+    
+    public static PFunctionProtos.PFunction toProto(PFunction function) {
+        PFunctionProtos.PFunction.Builder builder = PFunctionProtos.PFunction.newBuilder();
+        if(function.getTenantId() != null){
+          builder.setTenantId(HBaseZeroCopyByteString.wrap(function.getTenantId().getBytes()));
+        }
+        builder.setFunctionName(function.getFunctionName());
+        builder.setClassname(function.getClassName());
+        if (function.getJarPath() != null) {
+            builder.setJarPath(function.getJarPath());
+        }
+        builder.setReturnType(function.getReturnType());
+        builder.setTimeStamp(function.getTimeStamp());
+        for(FunctionArgument arg: function.getFunctionArguments()) {
+            PFunctionProtos.PFunctionArg.Builder argBuilder = PFunctionProtos.PFunctionArg.newBuilder();
+            argBuilder.setArgumentType(arg.getArgumentType());
+            argBuilder.setIsArrayType(arg.isArrayType);
+            argBuilder.setIsConstant(arg.isConstant);
+            if(arg.getDefaultValue() != null) {
+                argBuilder.setDefaultValue((String)arg.getDefaultValue().getValue());
+            }
+            if(arg.getMinValue() != null) {
+                argBuilder.setMinValue((String)arg.getMinValue().getValue());
+            }
+            if(arg.getMaxValue() != null) {
+                argBuilder.setMaxValue((String)arg.getMaxValue().getValue());
+            }
+            builder.addArguments(argBuilder.build());
+        }
+        return builder.build();
+      }
+
+    public static PFunction createFromProto(
+            org.apache.phoenix.coprocessor.generated.PFunctionProtos.PFunction function) {
+        PName tenantId = null;
+        if(function.hasTenantId()){
+          tenantId = PNameFactory.newName(function.getTenantId().toByteArray());
+        }
+        String functionName = function.getFunctionName();
+        long timeStamp = function.getTimeStamp();
+        String className = function.getClassname();
+        String jarPath = function.getJarPath();
+        String returnType = function.getReturnType();
+        List<FunctionArgument> args = new ArrayList<FunctionArgument>(function.getArgumentsCount());
+        for(PFunctionArg arg: function.getArgumentsList()) {
+            String argType = arg.getArgumentType();
+            boolean isArrayType = arg.hasIsArrayType()?arg.getIsArrayType():false;
+            boolean isConstant = arg.hasIsConstant()?arg.getIsConstant():false;
+            String defaultValue = arg.hasDefaultValue()?arg.getDefaultValue():null;
+            String minValue = arg.hasMinValue()?arg.getMinValue():null;
+            String maxValue = arg.hasMaxValue()?arg.getMaxValue():null;
+            args.add(new FunctionArgument(argType, isArrayType, isConstant,
+                    defaultValue == null ? null : LiteralExpression.newConstant(defaultValue),
+                    minValue == null ? null : LiteralExpression.newConstant(minValue),
+                    maxValue == null ? null : LiteralExpression.newConstant(maxValue)));
+        }
+        return new PFunction(tenantId,functionName, args, returnType, className, jarPath, timeStamp);
+    }
+
+    public int getEstimatedSize() {
+        return estimatedSize;
+    }
+}
+