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/07/09 13:52:37 UTC

phoenix git commit: PHOENIX-1889 Support alter/replace and drop functions(Rajeshbabu)

Repository: phoenix
Updated Branches:
  refs/heads/master fa2d79a01 -> 11bdb0eed


PHOENIX-1889 Support alter/replace and drop functions(Rajeshbabu)


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

Branch: refs/heads/master
Commit: 11bdb0eedbbea1e58b60fe8ebba21e5c168261b7
Parents: fa2d79a
Author: Rajeshbabu Chintaguntla <ra...@apache.org>
Authored: Thu Jul 9 17:24:39 2015 +0530
Committer: Rajeshbabu Chintaguntla <ra...@apache.org>
Committed: Thu Jul 9 17:24:39 2015 +0530

----------------------------------------------------------------------
 .../phoenix/end2end/UserDefinedFunctionsIT.java | 145 ++++++++++++++++++-
 phoenix-core/src/main/antlr3/PhoenixSQL.g       |   5 +-
 .../phoenix/compile/CreateFunctionCompiler.java |   4 +-
 .../coprocessor/MetaDataEndpointImpl.java       |  40 +++--
 .../coprocessor/generated/PFunctionProtos.java  | 100 ++++++++++++-
 .../apache/phoenix/jdbc/PhoenixStatement.java   |  28 ++--
 .../phoenix/parse/CreateFunctionStatement.java  |   7 +-
 .../org/apache/phoenix/parse/PFunction.java     |  28 +++-
 .../apache/phoenix/parse/ParseNodeFactory.java  |   4 +-
 .../query/ConnectionQueryServicesImpl.java      |   1 +
 .../apache/phoenix/schema/MetaDataClient.java   |  20 ++-
 phoenix-protocol/src/main/PFunction.proto       |   1 +
 12 files changed, 336 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/11bdb0ee/phoenix-core/src/it/java/org/apache/phoenix/end2end/UserDefinedFunctionsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UserDefinedFunctionsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UserDefinedFunctionsIT.java
index 613231d..e2b7b4c 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UserDefinedFunctionsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UserDefinedFunctionsIT.java
@@ -17,15 +17,18 @@
  */
 package org.apache.phoenix.end2end;
 
+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.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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.io.BufferedInputStream;
 import java.io.File;
@@ -34,6 +37,7 @@ import java.io.FileOutputStream;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.sql.Connection;
+import java.sql.DriverManager;
 import java.sql.ResultSet;
 import java.sql.Statement;
 import java.util.HashSet;
@@ -60,8 +64,6 @@ 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.schema.types.PDataType;
-import org.apache.phoenix.schema.types.PArrayDataType;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
@@ -561,6 +563,104 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT{
     }
 
     @Test
+    public void testUDFsWhenTimestampManagedAtClient() throws Exception {
+        long ts = 100;
+        Properties props = new Properties();
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts));
+        Connection conn = DriverManager.getConnection(url, 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 mysum61(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"+"'");
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 10));
+        conn = DriverManager.getConnection(url, props);
+        stmt = conn.createStatement();
+        rs = stmt.executeQuery(query);
+        rs.next();
+        int numRowsAfter= rs.getInt(1);
+        assertEquals(3, numRowsAfter - numRowsBefore);
+        stmt.execute("drop function mysum61");
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 20));
+        conn = DriverManager.getConnection(url, props);
+        stmt = conn.createStatement();
+        rs = stmt.executeQuery(query);
+        rs.next();
+        assertEquals(numRowsBefore, rs.getInt(1));
+        conn.createStatement().execute("create table t62(k integer primary key, k1 integer, lastname varchar)");
+        try {
+            rs = stmt.executeQuery("select mysum61(k1) from t62");
+            fail("FunctionNotFoundException should be thrown");
+        } catch(FunctionNotFoundException e) {
+            
+        }
+        try {
+            stmt.execute("drop function mysum61");
+            fail("FunctionNotFoundException should be thrown");
+        } catch(FunctionNotFoundException e) {
+            
+        }
+        try {
+            stmt.execute("drop function if exists mysum61");
+        } catch(FunctionNotFoundException e) {
+            fail("FunctionNotFoundException should not be thrown");
+        }
+        stmt.execute("create function mysum61(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"+"'");
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 30));
+        conn = DriverManager.getConnection(url, props);
+        stmt = conn.createStatement();
+        try {
+            rs = stmt.executeQuery("select mysum61(k1) from t62");
+        } catch(FunctionNotFoundException e) {
+            fail("FunctionNotFoundException should not be thrown");
+        }
+        conn.createStatement().execute("create table t61(k integer primary key, k1 integer, lastname varchar)");
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 40));
+        conn = DriverManager.getConnection(url, props);
+        stmt = conn.createStatement();
+        stmt.execute("upsert into t61 values(1,1,'jock')");
+        conn.commit();
+        stmt.execute("create function myfunction6(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("create or replace function myfunction6(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"+"'");
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 50));
+        conn = DriverManager.getConnection(url, props);
+        stmt = conn.createStatement();
+        rs = stmt.executeQuery("select myfunction6(k,12) from t61");
+        assertTrue(rs.next());
+        assertEquals(13, rs.getInt(1));
+        rs = stmt.executeQuery("select myfunction6(k) from t61");
+        assertTrue(rs.next());
+        assertEquals(11, rs.getInt(1));
+        rs = stmt.executeQuery("select k from t61 where myfunction6(k)=11");
+        assertTrue(rs.next());
+        assertEquals(1, rs.getInt(1));
+        stmt.execute("create or replace function myfunction6(VARCHAR) returns VARCHAR as 'org.apache.phoenix.end2end."+MY_REVERSE_CLASS_NAME+"' using jar "
+                + "'"+util.getConfiguration().get(DYNAMIC_JARS_DIR_KEY) + "/myjar1.jar"+"'");
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 60));
+        conn = DriverManager.getConnection(url, props);
+        stmt = conn.createStatement();
+        rs = stmt.executeQuery("select k from t61 where myfunction6(lastname)='kcoj'");
+        assertTrue(rs.next());
+        assertEquals(1, rs.getInt(1));
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 60));
+        props.setProperty(QueryServices.ALLOW_USER_DEFINED_FUNCTIONS_ATTRIB, "false");
+        conn = DriverManager.getConnection(url, props);
+        stmt = conn.createStatement();
+        try {
+            rs = stmt.executeQuery("select k from t61 where reverse(lastname,11)='kcoj'");
+            fail("FunctionNotFoundException should be thrown.");
+        } catch(FunctionNotFoundException e) {
+            
+        }
+
+    }
+
+    @Test
     public void testFunctionalIndexesWithUDFFunction() throws Exception {
         Connection conn = driver.connect(url, EMPTY_PROPS);
         Statement stmt = conn.createStatement();
@@ -591,6 +691,39 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT{
         assertFalse(rs.next());
     }
 
+    @Test
+    public void testReplaceFunction() throws Exception {
+        Connection conn = driver.connect(url, EMPTY_PROPS);
+        Statement stmt = conn.createStatement();
+        conn.createStatement().execute("create table t10(k integer primary key, k1 integer, lastname varchar)");
+        stmt.execute("upsert into t10 values(1,1,'jock')");
+        conn.commit();
+        stmt.execute("create function myfunction63(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("create or replace function myfunction63(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 myfunction63(k,12) from t10");
+        assertTrue(rs.next());
+        assertEquals(13, rs.getInt(1));
+        rs = stmt.executeQuery("select myfunction63(k) from t10");
+        assertTrue(rs.next());
+        assertEquals(11, rs.getInt(1));
+        rs = stmt.executeQuery("select k from t10 where myfunction63(k)=11");
+        assertTrue(rs.next());
+        assertEquals(1, rs.getInt(1));
+        Connection conn2 = driver.connect(url, EMPTY_PROPS);
+        stmt = conn2.createStatement();
+        rs = stmt.executeQuery("select myfunction63(k,12) from t10");
+        assertTrue(rs.next());
+        assertEquals(13, rs.getInt(1));
+        rs = stmt.executeQuery("select myfunction63(k) from t10");
+        assertTrue(rs.next());
+        assertEquals(11, rs.getInt(1));
+        rs = stmt.executeQuery("select k from t10 where myfunction63(k)=11");
+        assertTrue(rs.next());
+        assertEquals(1, rs.getInt(1));
+    }
+
     /**
      * Compiles the test class with bogus code into a .class file.
      */

http://git-wip-us.apache.org/repos/asf/phoenix/blob/11bdb0ee/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 69014a2..ca5e778 100644
--- a/phoenix-core/src/main/antlr3/PhoenixSQL.g
+++ b/phoenix-core/src/main/antlr3/PhoenixSQL.g
@@ -122,6 +122,7 @@ tokens
     JAR='jar';
     DEFAULTVALUE='defaultvalue';
     CONSTANT = 'constant';
+    REPLACE = 'replace';
 }
 
 
@@ -540,12 +541,12 @@ trace_node returns [TraceStatement ret]
 
 // Parse a trace statement.
 create_function_node returns [CreateFunctionStatement ret]
-    :   CREATE (temp=TEMPORARY)? FUNCTION function=identifier 
+    :   CREATE (OR replace=REPLACE)? (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);;
+            $ret = factory.createFunction(new PFunction(SchemaUtil.normalizeIdentifier(function), args,r,(String)className.getValue(), jarPath == null ? null : (String)jarPath.getValue()), temp!=null, replace!=null);
         } 
     ;
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/11bdb0ee/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
index 2e3a873..138c75d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateFunctionCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateFunctionCompiler.java
@@ -63,7 +63,9 @@ public class CreateFunctionCompiler {
 
             @Override
             public ExplainPlan getExplainPlan() throws SQLException {
-                return new ExplainPlan(Collections.singletonList("CREATE FUNCTION"));
+                return new ExplainPlan(Collections.singletonList("CREATE"
+                        + (create.getFunctionInfo().isReplace() ? " OR REPLACE" : "")
+                        + " FUNCTION"));
             }
 
             @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/11bdb0ee/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 b27beb4..f786768 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
@@ -478,7 +478,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     }
 
     private List<PFunction> buildFunctions(List<byte[]> keys, Region region,
-            long clientTimeStamp) throws IOException, SQLException {
+            long clientTimeStamp, boolean isReplace, List<Mutation> deleteMutationsForReplace) throws IOException, SQLException {
         List<KeyRange> keyRanges = Lists.newArrayListWithExpectedSize(keys.size());
         for (byte[] key : keys) {
             byte[] stopKey = ByteUtil.concat(key, QueryConstants.SEPARATOR_BYTE_ARRAY);
@@ -501,7 +501,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         try {
             for(int i = 0; i< keys.size(); i++) {
                 function = null;
-                function = getFunction(scanner);
+                function =
+                        getFunction(scanner, isReplace, clientTimeStamp, deleteMutationsForReplace);
                 if (function == null) {
                     return null;
                 }
@@ -828,7 +829,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             disableWAL, multiTenant, storeNulls, viewType, viewIndexId, indexType, stats, baseColumnCount);
     }
 
-    private PFunction getFunction(RegionScanner scanner)
+    private PFunction getFunction(RegionScanner scanner, final boolean isReplace, long clientTimeStamp, List<Mutation> deleteMutationsForReplace)
             throws IOException, SQLException {
         List<Cell> results = Lists.newArrayList();
         scanner.next(results);
@@ -837,12 +838,19 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         }
         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();
+        long currentTimeMillis = EnvironmentEdgeManager.currentTimeMillis();
+        if(isReplace) {
+            long deleteTimeStamp =
+                    clientTimeStamp == HConstants.LATEST_TIMESTAMP ? currentTimeMillis - 1
+                            : (keyValue.getTimestamp() < clientTimeStamp ? clientTimeStamp - 1
+                                    : keyValue.getTimestamp());
+            deleteMutationsForReplace.add(new Delete(keyBuffer, keyOffset, keyLength, deleteTimeStamp));
+        }
         PName tenantId = newPName(keyBuffer, keyOffset, keyLength);
         int tenantIdLength = (tenantId == null) ? 0 : tenantId.getBytes().length;
         if (tenantIdLength == 0) {
@@ -908,6 +916,14 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 break;
             }
             Cell typeKv = results.get(0);
+            if(isReplace) {
+                long deleteTimeStamp =
+                        clientTimeStamp == HConstants.LATEST_TIMESTAMP ? currentTimeMillis - 1
+                                : (typeKv.getTimestamp() < clientTimeStamp ? clientTimeStamp - 1
+                                        : typeKv.getTimestamp());
+                deleteMutationsForReplace.add(new Delete(typeKv.getRowArray(), typeKv
+                        .getRowOffset(), typeKv.getRowLength(), deleteTimeStamp));
+            }
             int typeKeyLength = typeKv.getRowLength();
             PName typeName =
                     newPName(typeKv.getRowArray(), typeKv.getRowOffset() + offset, typeKeyLength
@@ -1020,18 +1036,18 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     }
 
     private PFunction loadFunction(RegionCoprocessorEnvironment env, byte[] key,
-            ImmutableBytesPtr cacheKey, long clientTimeStamp, long asOfTimeStamp)
+            ImmutableBytesPtr cacheKey, long clientTimeStamp, long asOfTimeStamp, boolean isReplace, List<Mutation> deleteMutationsForReplace)
             throws IOException, SQLException {
             Region 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) {
+            if (function != null && !isReplace) {
                 return function;
             }
             ArrayList<byte[]> arrayList = new ArrayList<byte[]>(1);
             arrayList.add(key);
-            List<PFunction> functions = buildFunctions(arrayList, region, asOfTimeStamp);
+            List<PFunction> functions = buildFunctions(arrayList, region, asOfTimeStamp, isReplace, deleteMutationsForReplace);
             if(functions != null) return functions.get(0);
             // if not found then check if newer table already exists and add delete marker for timestamp
             // found
@@ -2075,7 +2091,9 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             if(functionsAvailable.size() == numFunctions) return functionsAvailable;
 
             // Query for the latest table first, since it's not cached
-            List<PFunction> buildFunctions = buildFunctions(keys, region, clientTimeStamp);
+            List<PFunction> buildFunctions =
+                    buildFunctions(keys, region, clientTimeStamp, false,
+                        Collections.<Mutation> emptyList());
             if(buildFunctions == null || buildFunctions.isEmpty()) {
                 return null;
             }
@@ -2582,7 +2600,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 // exists without making an additional query
                 ImmutableBytesPtr cacheKey = new FunctionBytesPtr(lockKey);
                 PFunction function =
-                        loadFunction(env, lockKey, cacheKey, clientTimeStamp, clientTimeStamp);
+                        loadFunction(env, lockKey, cacheKey, clientTimeStamp, clientTimeStamp, request.getReplace(), functionMetaData);
                 if (function != null) {
                     if (function.getTimeStamp() < clientTimeStamp) {
                         // If the function is older than the client time stamp and it's deleted,
@@ -2592,7 +2610,9 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                             builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
                             builder.addFunction(PFunction.toProto(function));
                             done.run(builder.build());
-                            return;
+                            if(!request.getReplace()) {
+                                return;
+                            }
                         }
                     } else {
                         builder.setReturnCode(MetaDataProtos.MutationCode.NEWER_FUNCTION_FOUND);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/11bdb0ee/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
index 12927aa..7783a95 100644
--- 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
@@ -1317,6 +1317,16 @@ public final class PFunctionProtos {
      * <code>optional bool isArrayReturnType = 8;</code>
      */
     boolean getIsArrayReturnType();
+
+    // optional bool isReplace = 9;
+    /**
+     * <code>optional bool isReplace = 9;</code>
+     */
+    boolean hasIsReplace();
+    /**
+     * <code>optional bool isReplace = 9;</code>
+     */
+    boolean getIsReplace();
   }
   /**
    * Protobuf type {@code PFunction}
@@ -1412,6 +1422,11 @@ public final class PFunctionProtos {
               isArrayReturnType_ = input.readBool();
               break;
             }
+            case 72: {
+              bitField0_ |= 0x00000080;
+              isReplace_ = input.readBool();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -1711,6 +1726,22 @@ public final class PFunctionProtos {
       return isArrayReturnType_;
     }
 
+    // optional bool isReplace = 9;
+    public static final int ISREPLACE_FIELD_NUMBER = 9;
+    private boolean isReplace_;
+    /**
+     * <code>optional bool isReplace = 9;</code>
+     */
+    public boolean hasIsReplace() {
+      return ((bitField0_ & 0x00000080) == 0x00000080);
+    }
+    /**
+     * <code>optional bool isReplace = 9;</code>
+     */
+    public boolean getIsReplace() {
+      return isReplace_;
+    }
+
     private void initFields() {
       functionName_ = "";
       arguments_ = java.util.Collections.emptyList();
@@ -1720,6 +1751,7 @@ public final class PFunctionProtos {
       tenantId_ = com.google.protobuf.ByteString.EMPTY;
       returnType_ = "";
       isArrayReturnType_ = false;
+      isReplace_ = false;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -1775,6 +1807,9 @@ public final class PFunctionProtos {
       if (((bitField0_ & 0x00000040) == 0x00000040)) {
         output.writeBool(8, isArrayReturnType_);
       }
+      if (((bitField0_ & 0x00000080) == 0x00000080)) {
+        output.writeBool(9, isReplace_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -1816,6 +1851,10 @@ public final class PFunctionProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeBoolSize(8, isArrayReturnType_);
       }
+      if (((bitField0_ & 0x00000080) == 0x00000080)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(9, isReplace_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -1876,6 +1915,11 @@ public final class PFunctionProtos {
         result = result && (getIsArrayReturnType()
             == other.getIsArrayReturnType());
       }
+      result = result && (hasIsReplace() == other.hasIsReplace());
+      if (hasIsReplace()) {
+        result = result && (getIsReplace()
+            == other.getIsReplace());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -1921,6 +1965,10 @@ public final class PFunctionProtos {
         hash = (37 * hash) + ISARRAYRETURNTYPE_FIELD_NUMBER;
         hash = (53 * hash) + hashBoolean(getIsArrayReturnType());
       }
+      if (hasIsReplace()) {
+        hash = (37 * hash) + ISREPLACE_FIELD_NUMBER;
+        hash = (53 * hash) + hashBoolean(getIsReplace());
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -2051,6 +2099,8 @@ public final class PFunctionProtos {
         bitField0_ = (bitField0_ & ~0x00000040);
         isArrayReturnType_ = false;
         bitField0_ = (bitField0_ & ~0x00000080);
+        isReplace_ = false;
+        bitField0_ = (bitField0_ & ~0x00000100);
         return this;
       }
 
@@ -2116,6 +2166,10 @@ public final class PFunctionProtos {
           to_bitField0_ |= 0x00000040;
         }
         result.isArrayReturnType_ = isArrayReturnType_;
+        if (((from_bitField0_ & 0x00000100) == 0x00000100)) {
+          to_bitField0_ |= 0x00000080;
+        }
+        result.isReplace_ = isReplace_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -2187,6 +2241,9 @@ public final class PFunctionProtos {
         if (other.hasIsArrayReturnType()) {
           setIsArrayReturnType(other.getIsArrayReturnType());
         }
+        if (other.hasIsReplace()) {
+          setIsReplace(other.getIsReplace());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -2870,6 +2927,39 @@ public final class PFunctionProtos {
         return this;
       }
 
+      // optional bool isReplace = 9;
+      private boolean isReplace_ ;
+      /**
+       * <code>optional bool isReplace = 9;</code>
+       */
+      public boolean hasIsReplace() {
+        return ((bitField0_ & 0x00000100) == 0x00000100);
+      }
+      /**
+       * <code>optional bool isReplace = 9;</code>
+       */
+      public boolean getIsReplace() {
+        return isReplace_;
+      }
+      /**
+       * <code>optional bool isReplace = 9;</code>
+       */
+      public Builder setIsReplace(boolean value) {
+        bitField0_ |= 0x00000100;
+        isReplace_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bool isReplace = 9;</code>
+       */
+      public Builder clearIsReplace() {
+        bitField0_ = (bitField0_ & ~0x00000100);
+        isReplace_ = false;
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:PFunction)
     }
 
@@ -2903,14 +2993,14 @@ public final class PFunctionProtos {
       "\n\017PFunction.proto\"\207\001\n\014PFunctionArg\022\024\n\014ar" +
       "gumentType\030\001 \002(\t\022\023\n\013isArrayType\030\002 \001(\010\022\022\n" +
       "\nisConstant\030\003 \001(\010\022\024\n\014defaultValue\030\004 \001(\t\022" +
-      "\020\n\010minValue\030\005 \001(\t\022\020\n\010maxValue\030\006 \001(\t\"\273\001\n\t" +
+      "\020\n\010minValue\030\005 \001(\t\022\020\n\010maxValue\030\006 \001(\t\"\316\001\n\t" +
       "PFunction\022\024\n\014functionName\030\001 \002(\t\022 \n\targum" +
       "ents\030\002 \003(\0132\r.PFunctionArg\022\021\n\tclassname\030\003" +
       " \002(\t\022\021\n\ttimeStamp\030\004 \002(\003\022\017\n\007jarPath\030\005 \001(\t" +
       "\022\020\n\010tenantId\030\006 \001(\014\022\022\n\nreturnType\030\007 \001(\t\022\031" +
-      "\n\021isArrayReturnType\030\010 \001(\010BC\n(org.apache." +
-      "phoenix.coprocessor.generatedB\017PFunction",
-      "ProtosH\001\210\001\001\240\001\001"
+      "\n\021isArrayReturnType\030\010 \001(\010\022\021\n\tisReplace\030\t" +
+      " \001(\010BC\n(org.apache.phoenix.coprocessor.g",
+      "eneratedB\017PFunctionProtosH\001\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -2928,7 +3018,7 @@ public final class PFunctionProtos {
           internal_static_PFunction_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_PFunction_descriptor,
-              new java.lang.String[] { "FunctionName", "Arguments", "Classname", "TimeStamp", "JarPath", "TenantId", "ReturnType", "IsArrayReturnType", });
+              new java.lang.String[] { "FunctionName", "Arguments", "Classname", "TimeStamp", "JarPath", "TenantId", "ReturnType", "IsArrayReturnType", "IsReplace", });
           return null;
         }
       };

http://git-wip-us.apache.org/repos/asf/phoenix/blob/11bdb0ee/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 2bb3b92..f323ec4 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
@@ -116,6 +116,7 @@ import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.schema.ExecuteQueryNotApplicableException;
 import org.apache.phoenix.schema.ExecuteUpdateNotApplicableException;
+import org.apache.phoenix.schema.FunctionNotFoundException;
 import org.apache.phoenix.schema.MetaDataClient;
 import org.apache.phoenix.schema.PDatum;
 import org.apache.phoenix.schema.PIndexState;
@@ -354,7 +355,7 @@ public class PhoenixStatement implements Statement, SQLCloseable, org.apache.pho
         @Override
         public QueryPlan compilePlan(PhoenixStatement stmt, Sequence.ValueOp seqAction) throws SQLException {
             if(!getUdfParseNodes().isEmpty()) {
-                stmt.throwIfUnallowedUserDefinedFunctions();
+                stmt.throwIfUnallowedUserDefinedFunctions(getUdfParseNodes());
             }
             SelectStatement select = SubselectRewriter.flatten(this, stmt.getConnection());
             ColumnResolver resolver = FromCompiler.getResolverForQuery(select, stmt.getConnection());
@@ -531,7 +532,7 @@ public class PhoenixStatement implements Statement, SQLCloseable, org.apache.pho
         @Override
         public MutationPlan compilePlan(PhoenixStatement stmt, Sequence.ValueOp seqAction) throws SQLException {
             if(!getUdfParseNodes().isEmpty()) {
-                stmt.throwIfUnallowedUserDefinedFunctions();
+                stmt.throwIfUnallowedUserDefinedFunctions(getUdfParseNodes());
             }
             UpsertCompiler compiler = new UpsertCompiler(stmt);
             MutationPlan plan = compiler.compile(this);
@@ -549,7 +550,7 @@ public class PhoenixStatement implements Statement, SQLCloseable, org.apache.pho
         @Override
         public MutationPlan compilePlan(PhoenixStatement stmt, Sequence.ValueOp seqAction) throws SQLException {
             if(!getUdfParseNodes().isEmpty()) {
-                stmt.throwIfUnallowedUserDefinedFunctions();
+                stmt.throwIfUnallowedUserDefinedFunctions(getUdfParseNodes());
             }
             DeleteCompiler compiler = new DeleteCompiler(stmt);
             MutationPlan plan = compiler.compile(this);
@@ -575,15 +576,15 @@ 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);
+        public ExecutableCreateFunctionStatement(PFunction functionInfo, boolean temporary, boolean isReplace) {
+            super(functionInfo, temporary, isReplace);
         }
 
 
         @SuppressWarnings("unchecked")
         @Override
         public MutationPlan compilePlan(final PhoenixStatement stmt, Sequence.ValueOp seqAction) throws SQLException {
-            stmt.throwIfUnallowedUserDefinedFunctions();
+            stmt.throwIfUnallowedUserDefinedFunctions(Collections.EMPTY_MAP);
             CreateFunctionCompiler compiler = new CreateFunctionCompiler(stmt);
             return compiler.compile(this);
         }
@@ -641,7 +642,7 @@ public class PhoenixStatement implements Statement, SQLCloseable, org.apache.pho
         @Override
         public MutationPlan compilePlan(PhoenixStatement stmt, Sequence.ValueOp seqAction) throws SQLException {
             if(!getUdfParseNodes().isEmpty()) {
-                stmt.throwIfUnallowedUserDefinedFunctions();
+                stmt.throwIfUnallowedUserDefinedFunctions(getUdfParseNodes());
             }
             CreateIndexCompiler compiler = new CreateIndexCompiler(stmt);
             return compiler.compile(this);
@@ -1022,8 +1023,8 @@ public class PhoenixStatement implements Statement, SQLCloseable, org.apache.pho
         }
         
         @Override
-        public CreateFunctionStatement createFunction(PFunction functionInfo, boolean temporary) {
-            return new ExecutableCreateFunctionStatement(functionInfo, temporary);
+        public CreateFunctionStatement createFunction(PFunction functionInfo, boolean temporary, boolean isReplace) {
+            return new ExecutableCreateFunctionStatement(functionInfo, temporary, isReplace);
         }
         @Override
         public DropSequenceStatement dropSequence(TableName tableName, boolean ifExists, int bindCount){
@@ -1504,14 +1505,17 @@ public class PhoenixStatement implements Statement, SQLCloseable, org.apache.pho
         this.lastQueryPlan = lastQueryPlan;
     }
     
-    private void throwIfUnallowedUserDefinedFunctions() throws SQLException {
+    private void throwIfUnallowedUserDefinedFunctions(Map<String, UDFParseNode> udfParseNodes) 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();
+            if(udfParseNodes.isEmpty()) {
+                throw new SQLExceptionInfo.Builder(SQLExceptionCode.UNALLOWED_USER_DEFINED_FUNCTIONS)
+                .build().buildException();
+            }
+            throw new FunctionNotFoundException(udfParseNodes.keySet().toString());
         }
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/11bdb0ee/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
index 741e4df..863783b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/CreateFunctionStatement.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/CreateFunctionStatement.java
@@ -20,10 +20,12 @@ package org.apache.phoenix.parse;
 public class CreateFunctionStatement extends MutableStatement {
     private final PFunction functionInfo;
     private final boolean temporary;
+    private final boolean isReplace;
 
-    public CreateFunctionStatement(PFunction functionInfo, boolean temporary) {
+    public CreateFunctionStatement(PFunction functionInfo, boolean temporary, boolean isReplace) {
         this.functionInfo = functionInfo;
         this.temporary = temporary;
+        this.isReplace = isReplace;
     }
 
     @Override
@@ -39,4 +41,7 @@ public class CreateFunctionStatement extends MutableStatement {
         return temporary;
     }
 
+    public boolean isReplace() {
+        return isReplace;
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/11bdb0ee/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
index a1413de..a5263ad 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/PFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/PFunction.java
@@ -46,6 +46,7 @@ public class PFunction implements PMetaDataEntity {
     private long timeStamp;
     private int estimatedSize;
     private boolean temporary;
+    private boolean replace;
 
     public PFunction(long timeStamp) { // For index delete marker
         this.timeStamp = timeStamp;
@@ -71,11 +72,23 @@ public class PFunction implements PMetaDataEntity {
     public PFunction(PFunction function, boolean temporary) {
         this(function.getTenantId(), function.getFunctionName(), function.getFunctionArguments(),
                 function.getReturnType(), function.getClassName(), function.getJarPath(), function
-                        .getTimeStamp(), temporary);
+                        .getTimeStamp(), temporary, function.isReplace());
+    }
+
+    public PFunction(PFunction function, boolean temporary, boolean isReplace) {
+        this(function.getTenantId(), function.getFunctionName(), function.getFunctionArguments(),
+                function.getReturnType(), function.getClassName(), function.getJarPath(), function
+                        .getTimeStamp(), temporary, isReplace);
+    }
+
+    public PFunction(PName tenantId, String functionName, List<FunctionArgument> args,
+            String returnType, String className, String jarPath, long timeStamp, boolean temporary) {
+        this(tenantId, functionName, args, returnType, className, jarPath, timeStamp, temporary,
+                false);
     }
 
     public PFunction(PName tenantId, String functionName, List<FunctionArgument> args, String returnType,
-            String className, String jarPath, long timeStamp, boolean temporary) {
+            String className, String jarPath, long timeStamp, boolean temporary, boolean replace) {
         this.tenantId = tenantId;
         this.functionName = PNameFactory.newName(functionName);
         if (args == null){ 
@@ -94,6 +107,7 @@ public class PFunction implements PMetaDataEntity {
                 PNameFactory.getEstimatedSize(this.className) +
                  (jarPath==null?0:PNameFactory.getEstimatedSize(this.jarPath));
         this.temporary = temporary;
+        this.replace = replace;
     }
 
     public PFunction(PFunction function) {
@@ -217,6 +231,9 @@ public class PFunction implements PMetaDataEntity {
             }
             builder.addArguments(argBuilder.build());
         }
+        if(builder.hasIsReplace()) {
+            builder.setIsReplace(function.isReplace());
+        }
         return builder.build();
       }
 
@@ -248,11 +265,16 @@ public class PFunction implements PMetaDataEntity {
                     minValue == null ? null : LiteralExpression.newConstant((new LiteralParseNode(dataType.toObject(minValue))).getValue()),
                     maxValue == null ? null : LiteralExpression.newConstant((new LiteralParseNode(dataType.toObject(maxValue))).getValue())));
         }
-        return new PFunction(tenantId,functionName, args, returnType, className, jarPath, timeStamp);
+        return new PFunction(tenantId, functionName, args, returnType, className, jarPath,
+                timeStamp, false, function.hasIsReplace() ? true : false);
     }
 
     public int getEstimatedSize() {
         return estimatedSize;
     }
+
+    public boolean isReplace() {
+        return this.replace;
+    }
 }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/11bdb0ee/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 291c84c..44359a7 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
@@ -297,8 +297,8 @@ public class ParseNodeFactory {
                 maxValue, cycle, ifNotExits, bindCount);
     }
 
-    public CreateFunctionStatement createFunction(PFunction functionInfo, boolean temporary) {
-        return new CreateFunctionStatement(functionInfo, temporary);
+    public CreateFunctionStatement createFunction(PFunction functionInfo, boolean temporary, boolean isReplace) {
+        return new CreateFunctionStatement(functionInfo, temporary, isReplace);
     }
 
     public DropFunctionStatement dropFunction(String functionName, boolean ifExists) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/11bdb0ee/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 ddebf9f..69520b0 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
@@ -2709,6 +2709,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                             builder.addTableMetadataMutations(mp.toByteString());
                         }
                         builder.setTemporary(temporary);
+                        builder.setReplace(function.isReplace());
                         instance.createFunction(controller, builder.build(), rpcCallback);
                         if(controller.getFailedOn() != null) {
                             throw controller.getFailedOn();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/11bdb0ee/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 d2d4338..9e74d2a 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
@@ -1348,7 +1348,7 @@ public class MetaDataClient {
         boolean wasAutoCommit = connection.getAutoCommit();
         connection.rollback();
         try {
-            PFunction function = new PFunction(stmt.getFunctionInfo(), stmt.isTemporary());
+            PFunction function = new PFunction(stmt.getFunctionInfo(), stmt.isTemporary(), stmt.isReplace());
             connection.setAutoCommit(false);
             String tenantIdStr = connection.getTenantId() == null ? null : connection.getTenantId().getString();
             List<Mutation> functionData = Lists.newArrayListWithExpectedSize(function.getFunctionArguments().size() + 1);
@@ -1377,16 +1377,26 @@ public class MetaDataClient {
             MutationCode code = result.getMutationCode();
             switch(code) {
             case FUNCTION_ALREADY_EXISTS:
-                throw new FunctionAlreadyExistsException(function.getFunctionName(), result
+                if (!function.isReplace()) {
+                    throw new FunctionAlreadyExistsException(function.getFunctionName(), result
                         .getFunctions().get(0));
+                } else {
+                    connection.removeFunction(function.getTenantId(), function.getFunctionName(),
+                        result.getMutationTime());
+                    addFunctionToCache(result);
+                }
             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));
+                    // 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);
+                if(function.isReplace()) {
+                    connection.removeFunction(function.getTenantId(), function.getFunctionName(),
+                        result.getMutationTime());
+                }
                 addFunctionToCache(result);
             }
         } finally {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/11bdb0ee/phoenix-protocol/src/main/PFunction.proto
----------------------------------------------------------------------
diff --git a/phoenix-protocol/src/main/PFunction.proto b/phoenix-protocol/src/main/PFunction.proto
index 07cbac7..0b44d5c 100644
--- a/phoenix-protocol/src/main/PFunction.proto
+++ b/phoenix-protocol/src/main/PFunction.proto
@@ -42,4 +42,5 @@ message PFunction {
   optional bytes tenantId = 6;
   optional string returnType = 7;
   optional bool isArrayReturnType = 8;
+  optional bool isReplace = 9;
 }