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;
+}

Reply via email to