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 2016/10/10 20:41:05 UTC

[1/2] phoenix git commit: PHOENIX-3242 Support CREATE OR REPLACE FUNCTION in Phoenix-Calcite Integration(Rajeshbabu)

Repository: phoenix
Updated Branches:
  refs/heads/calcite db70ca201 -> 54cc24038


http://git-wip-us.apache.org/repos/asf/phoenix/blob/54cc2403/phoenix-core/src/test/java/org/apache/phoenix/calcite/PhoenixSqlParserTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/calcite/PhoenixSqlParserTest.java b/phoenix-core/src/test/java/org/apache/phoenix/calcite/PhoenixSqlParserTest.java
new file mode 100644
index 0000000..34b3fd9
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/calcite/PhoenixSqlParserTest.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.calcite;
+
+import java.io.IOException;
+
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.calcite.sql.parser.SqlParser;
+import org.apache.calcite.sql.parser.SqlParserTest;
+import org.apache.phoenix.calcite.parser.PhoenixParserImpl;
+import org.junit.Test;
+
+public class PhoenixSqlParserTest extends SqlParserTest {
+    
+    @Test
+    public void testDDL(){
+        getTester().checkNode("create table \"t0\"(\"a\" varchar(20) not null primary key, b integer)", isDdl());
+        getTester().checkNode("drop table \"t0\"", isDdl());
+        getTester().checkNode("create view v1 as select * from \"t0\"", isDdl());
+        getTester().checkNode("drop view v1", isDdl());
+        getTester().checkNode("create index idx1 on itest(b desc, c) include (d)", isDdl());
+        getTester().checkNode("create local index idx1 on itest(b desc, c) include (d)", isDdl());
+        getTester().checkNode("drop index idx1 on itest", isDdl());
+        getTester().checkNode("create sequence if not exists s0 start with 2 increment 3 minvalue 2 maxvalue 90 cycle cache 3", isDdl());
+        getTester().checkNode("drop sequence if exists s0", isDdl());
+        getTester().checkNode("update statistics stest columns set dummy=2", isDdl());
+        getTester().checkNode("create or replace function myfunction(INTEGER, INTEGER CONSTANT defaultValue=10 minvalue=1 maxvalue=15) returns INTEGER as 'org.apache.phoenix.end2end.MyReverse' using jar 'hdfs://localhost:51573/hbase/tmpjars/myjar1.jar'", isDdl());
+        getTester().checkNode("drop function if exists myfunction", isDdl());
+        getTester().checkNode("upload jars './myjar.jar'", isDdl());
+        getTester().checkNode("delete jar '/myjar.jar'", isDdl());
+    }
+
+    private SqlParser getSqlParser(String sql) {
+        return SqlParser.create(sql,
+            SqlParser.configBuilder().setParserFactory(PhoenixParserImpl.FACTORY)
+                .build());
+    }
+
+    @Override
+    protected SqlNode parseStmt(String sql) throws SqlParseException {
+        return getSqlParser(sql).parseStmt();
+    }
+
+    @Override
+    public void testBackTickQuery() {
+        // Noop
+    }
+
+    @Override
+    public void testBracketIdentifier() {
+        // Noop
+    }
+
+    @Override
+    public void testGenerateKeyWords() throws IOException {
+        // Noop
+    }
+}


[2/2] phoenix git commit: PHOENIX-3242 Support CREATE OR REPLACE FUNCTION in Phoenix-Calcite Integration(Rajeshbabu)

Posted by ra...@apache.org.
PHOENIX-3242 Support CREATE OR REPLACE FUNCTION in Phoenix-Calcite Integration(Rajeshbabu)


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

Branch: refs/heads/calcite
Commit: 54cc240385831833c2a51c0dcda6a77f429285bb
Parents: db70ca2
Author: Rajeshbabu Chintaguntla <ra...@apache.org>
Authored: Tue Oct 11 02:01:13 2016 +0530
Committer: Rajeshbabu Chintaguntla <ra...@apache.org>
Committed: Tue Oct 11 02:01:13 2016 +0530

----------------------------------------------------------------------
 .../apache/phoenix/calcite/CalciteDDLIT.java    |  34 ++++
 .../phoenix/end2end/UserDefinedFunctionsIT.java | 139 ++++++++--------
 phoenix-core/src/main/codegen/data/Parser.tdd   |  11 ++
 .../src/main/codegen/includes/parserImpls.ftl   | 158 +++++++++++++++++++
 .../org/apache/calcite/sql/ListJarsTable.java   |  92 +++++++++++
 .../calcite/sql/SqlFunctionArguementNode.java   |  68 ++++++++
 .../apache/phoenix/calcite/CalciteUtils.java    |  13 +-
 .../phoenix/calcite/PhoenixPrepareImpl.java     |  96 ++++++++++-
 .../phoenix/calcite/PhoenixScalarFunction.java  | 103 ++++++++++++
 .../apache/phoenix/calcite/PhoenixSchema.java   |  25 ++-
 .../calcite/parse/SqlCreateFunction.java        |  72 +++++++++
 .../phoenix/calcite/parse/SqlDeleteJarNode.java |  51 ++++++
 .../phoenix/calcite/parse/SqlDropFunction.java  |  54 +++++++
 .../calcite/parse/SqlUploadJarsNode.java        |  48 ++++++
 .../apache/phoenix/compile/FromCompiler.java    |  53 ++++++-
 .../apache/phoenix/jdbc/PhoenixStatement.java   |   4 +-
 .../phoenix/calcite/PhoenixSqlParserTest.java   |  74 +++++++++
 17 files changed, 1012 insertions(+), 83 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/54cc2403/phoenix-core/src/it/java/org/apache/phoenix/calcite/CalciteDDLIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/calcite/CalciteDDLIT.java b/phoenix-core/src/it/java/org/apache/phoenix/calcite/CalciteDDLIT.java
index 2a222cd..2434105 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/calcite/CalciteDDLIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/calcite/CalciteDDLIT.java
@@ -1,7 +1,16 @@
 package org.apache.phoenix.calcite;
 
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.sql.ResultSet;
+import java.sql.Statement;
 import java.util.Properties;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.phoenix.query.QueryServices;
 import org.junit.Ignore;
 import org.junit.Test;
 
@@ -51,6 +60,31 @@ public class CalciteDDLIT extends BaseCalciteIT {
         start(PROPS).sql("drop table t5").execute().close();
         start(PROPS).sql("create table t5(a bigint not null primary key, b varchar)").execute().close();
     }
+
+    @Test public void testCreateAndDropFunction() throws Exception {
+        start(PROPS).sql("create function myreverse(VARCHAR) returns VARCHAR as 'org.apache.phoenix.end2end.MyReverse' using jar 'hdfs://localhost:51573/hbase/tmpjars/myjar1.jar'").execute().close();
+        start(PROPS).sql("create or replace function myfunction(INTEGER, INTEGER CONSTANT defaultValue=10 minvalue=1 maxvalue=15) returns INTEGER as 'org.apache.phoenix.end2end.MyReverse' using jar 'hdfs://localhost:51573/hbase/tmpjars/myjar1.jar'").execute().close();
+        start(PROPS).sql("drop function if exists myfunction").execute().close();
+        start(PROPS).sql("create or replace function myfunction(VARCHAR ARRAY, INTEGER CONSTANT defaultValue=10 minvalue=1 maxvalue=15) returns INTEGER as 'org.apache.phoenix.end2end.MyReverse' using jar 'hdfs://localhost:51573/hbase/tmpjars/myjar1.jar'").execute().close();
+    }
+
+    @Test public void testUploadAndDeleteJars() throws Exception {
+        String jarName = "myjar.jar";
+        String jarPath = "." + File.separator + jarName;
+        File jarFile = new File(jarPath);
+        jarFile.createNewFile();
+        start(PROPS).sql("upload jars '"+jarFile.getAbsolutePath()+"'").execute().close();
+        Start start = start(PROPS);
+        Statement statement = start.getConnection().createStatement();
+        final String sql = "select jar_location\n"
+              + "from table(\"ListJars\"())";
+        ResultSet rs = statement.executeQuery(sql);
+        assertTrue(rs.next());
+        Configuration conf = HBaseConfiguration.create();
+        start(PROPS).sql("delete jar '"+ conf.get(QueryServices.DYNAMIC_JARS_DIR_KEY)+"/"+jarName+"'").execute().close();
+        rs = statement.executeQuery(sql);
+        assertFalse(rs.next());
+    }
     
     @Ignore
     @Test public void testUpdateStatistics() throws Exception {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/54cc2403/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 8b5a591..f8d36f6 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
@@ -27,7 +27,6 @@ import static org.apache.phoenix.util.PhoenixRuntime.PHOENIX_TEST_DRIVER_URL_PAR
 import static org.apache.phoenix.util.TestUtil.JOIN_ITEM_TABLE_FULL_NAME;
 import static org.apache.phoenix.util.TestUtil.JOIN_SUPPLIER_TABLE_FULL_NAME;
 import static org.apache.phoenix.util.TestUtil.LOCALHOST;
-import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
@@ -37,12 +36,13 @@ import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.sql.Connection;
+import java.sql.Driver;
 import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
+import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.HashSet;
-import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
 import java.util.jar.Attributes;
@@ -57,12 +57,14 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.phoenix.expression.function.UDFExpression;
+import org.apache.phoenix.jdbc.PhoenixCalciteTestDriver;
 import org.apache.phoenix.jdbc.PhoenixTestDriver;
 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.PropertiesUtil;
 import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
 import org.junit.After;
@@ -75,7 +77,7 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT {
     
     protected static final String TENANT_ID = "ZZTop";
     private static String url;
-    private static PhoenixTestDriver driver;
+    private static Driver driver;
     private static HBaseTestingUtility util;
 
     private static String STRING_REVERSE_EVALUATE_METHOD =
@@ -188,7 +190,7 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT {
     private static String MY_ARRAY_INDEX_PROGRAM = getProgram(MY_ARRAY_INDEX_CLASS_NAME, ARRAY_INDEX_EVALUATE_METHOD, "return PDataType.fromTypeId(children.get(0).getDataType().getSqlType()- PDataType.ARRAY_TYPE_BASE);");
     private static String GETX_CLASSNAME_PROGRAM = getProgram(GETX_CLASSNAME, GETX_EVALUATE_METHOD, "return PLong.INSTANCE;");
     private static String GETY_CLASSNAME_PROGRAM = getProgram(GETY_CLASSNAME, GETY_EVALUATE_METHOD, "return PInteger.INSTANCE;");
-    private static Properties EMPTY_PROPS = new Properties();
+    private static Properties UDF_PROPS = new Properties();
     
 
     @Override
@@ -258,10 +260,13 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT {
         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");
-        props.put(QueryServices.DYNAMIC_JARS_DIR_KEY,string+"/hbase/tmpjars/");
-        driver = initAndRegisterTestDriver(url, new ReadOnlyProps(props.entrySet().iterator()));
+        UDF_PROPS.put(QueryServices.ALLOW_USER_DEFINED_FUNCTIONS_ATTRIB, "true");
+        UDF_PROPS.put(QueryServices.DYNAMIC_JARS_DIR_KEY,string+"/hbase/tmpjars/");
+        PhoenixTestDriver newDriver = new PhoenixTestDriver();
+        DriverManager.registerDriver(newDriver);
+        Class.forName(PhoenixCalciteTestDriver.class.getName());
+        calciteUrl = url.replaceFirst(PhoenixRuntime.JDBC_PROTOCOL, PhoenixRuntime.JDBC_PROTOCOL_CALCITE);
+        driver = DriverManager.getDriver(calciteUrl);
         compileTestClass(MY_REVERSE_CLASS_NAME, MY_REVERSE_PROGRAM, 1);
         compileTestClass(MY_SUM_CLASS_NAME, MY_SUM_PROGRAM, 2);
         compileTestClass(MY_ARRAY_INDEX_CLASS_NAME, MY_ARRAY_INDEX_PROGRAM, 3);
@@ -272,9 +277,9 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT {
     
     @Test
     public void testListJars() throws Exception {
-        Connection conn = driver.connect(url, EMPTY_PROPS);
+        Connection conn = driver.connect(calciteUrl, UDF_PROPS);
         Statement stmt = conn.createStatement();
-        ResultSet rs = stmt.executeQuery("list jars");
+        ResultSet rs = stmt.executeQuery("select jar_location from table(\"ListJars\"())");
         assertTrue(rs.next());
         assertEquals(util.getConfiguration().get(QueryServices.DYNAMIC_JARS_DIR_KEY)+"/"+"myjar1.jar", rs.getString("jar_location"));
         assertTrue(rs.next());
@@ -290,9 +295,11 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT {
 
     @Test
     public void testDeleteJar() throws Exception {
-        Connection conn = driver.connect(url, EMPTY_PROPS);
+        Connection conn = driver.connect(calciteUrl, UDF_PROPS);
         Statement stmt = conn.createStatement();
-        ResultSet rs = stmt.executeQuery("list jars");
+        final String sql = "select jar_location\n"
+                + "from table(\"ListJars\"())";
+        ResultSet rs = stmt.executeQuery(sql);
         assertTrue(rs.next());
         assertEquals(util.getConfiguration().get(QueryServices.DYNAMIC_JARS_DIR_KEY)+"/"+"myjar1.jar", rs.getString("jar_location"));
         assertTrue(rs.next());
@@ -307,7 +314,7 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT {
         assertEquals(util.getConfiguration().get(QueryServices.DYNAMIC_JARS_DIR_KEY)+"/"+"myjar6.jar", rs.getString("jar_location"));
         assertFalse(rs.next());
         stmt.execute("delete jar '"+ util.getConfiguration().get(QueryServices.DYNAMIC_JARS_DIR_KEY)+"/"+"myjar4.jar'");
-        rs = stmt.executeQuery("list jars");
+        rs = stmt.executeQuery(sql);
         assertTrue(rs.next());
         assertEquals(util.getConfiguration().get(QueryServices.DYNAMIC_JARS_DIR_KEY)+"/"+"myjar1.jar", rs.getString("jar_location"));
         assertTrue(rs.next());
@@ -323,9 +330,9 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT {
 
     @Test
     public void testCreateFunction() throws Exception {
-        Connection conn = driver.connect(url, EMPTY_PROPS);
+        Connection conn = driver.connect(calciteUrl, UDF_PROPS);
         Statement stmt = conn.createStatement();
-        conn.createStatement().execute("create table t(k integer primary key, firstname varchar, lastname varchar)");
+        conn.createStatement().execute("create table t(k integer not null 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 "
@@ -361,7 +368,7 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT {
         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);
+        Connection conn2 = driver.connect(calciteUrl+";"+PhoenixRuntime.TENANT_ID_ATTRIB+"="+TENANT_ID, UDF_PROPS);
         try {
             conn2.createStatement().execute("upsert into t3 values(1,'foo','jock')");
             conn2.commit();
@@ -380,21 +387,18 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT {
         try {
             conn2.createStatement().execute("drop function myreverse2");
             fail("FunctionNotFoundException should be thrown");
-        } catch(FunctionNotFoundException e){
-            
+        } 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) {
-            
+        } catch(FunctionNotFoundException e){
         }
         conn.createStatement().execute("CREATE TABLE TESTTABLE10(ID VARCHAR NOT NULL, NAME VARCHAR ARRAY, CITY VARCHAR ARRAY CONSTRAINT pk PRIMARY KEY (ID) )");
         conn.createStatement().execute("create function UDF_ARRAY_ELEM(VARCHAR ARRAY, INTEGER) returns VARCHAR as 'org.apache.phoenix.end2end."+MY_ARRAY_INDEX_CLASS_NAME+"' using jar "
@@ -422,7 +426,7 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT {
 
     @Test
     public void testSameUDFWithDifferentImplementationsInDifferentTenantConnections() throws Exception {
-        Connection nonTenantConn = driver.connect(url, EMPTY_PROPS);
+        Connection nonTenantConn = driver.connect(calciteUrl, UDF_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 {
@@ -430,13 +434,12 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT {
                     + "'"+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);
+        Connection tenant1Conn = driver.connect(calciteUrl+";"+PhoenixRuntime.TENANT_ID_ATTRIB+"="+tenantId1, UDF_PROPS);
+        Connection tenant2Conn = driver.connect(calciteUrl+";"+PhoenixRuntime.TENANT_ID_ATTRIB+"="+tenantId2, UDF_PROPS);
         tenant1Conn.createStatement().execute("upsert into t7 values(1,1,'jock')");
         tenant1Conn.commit();
         tenant2Conn.createStatement().execute("upsert into t7 values(1,2,'jock')");
@@ -448,7 +451,6 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT {
                     + "'"+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 "
@@ -458,7 +460,6 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT {
                     + "'"+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");
@@ -471,7 +472,6 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT {
         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));
@@ -486,16 +486,15 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT {
 
     @Test
     public void testUDFsWithMultipleConnections() throws Exception {
-        Connection conn1 = driver.connect(url, EMPTY_PROPS);
+        Connection conn1 = driver.connect(calciteUrl, UDF_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);
+        Connection conn2 = driver.connect(calciteUrl, UDF_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')");
@@ -514,18 +513,18 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT {
         try {
             rs = conn1.createStatement().executeQuery("select MYFUNCTION(name) from t8");
             fail("FunctionNotFoundException should be thrown");
-        } catch(FunctionNotFoundException e) {
-            
+        } catch (FunctionNotFoundException e) {
         }
     }
+
     @Test
     public void testUsingUDFFunctionInDifferentQueries() throws Exception {
-        Connection conn = driver.connect(url, EMPTY_PROPS);
+        Connection conn = driver.connect(calciteUrl, UDF_PROPS);
         Statement stmt = conn.createStatement();
-        conn.createStatement().execute("create table t1(k integer primary key, firstname varchar, lastname varchar)");
+        conn.createStatement().execute("create table t1(k integer not null 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.createStatement().execute("create table t2(k integer not null 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"+"'");
@@ -556,9 +555,9 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT {
 
     @Test
     public void testVerifyCreateFunctionArguments() throws Exception {
-        Connection conn = driver.connect(url, EMPTY_PROPS);
+        Connection conn = driver.connect(calciteUrl, UDF_PROPS);
         Statement stmt = conn.createStatement();
-        conn.createStatement().execute("create table t4(k integer primary key, k1 integer, lastname varchar)");
+        conn.createStatement().execute("create table t4(k integer not null 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 "
@@ -579,9 +578,9 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT {
 
     @Test
     public void testTemporaryFunctions() throws Exception {
-        Connection conn = driver.connect(url, EMPTY_PROPS);
+        Connection conn = driver.connect(calciteUrl, UDF_PROPS);
         Statement stmt = conn.createStatement();
-        conn.createStatement().execute("create table t9(k integer primary key, k1 integer, lastname varchar)");
+        conn.createStatement().execute("create table t9(k integer not null 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 "
@@ -600,7 +599,7 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT {
             fail("FunctionNotFoundException should be thrown");
         } catch(FunctionNotFoundException e) {
         } catch(Exception e) {
-            fail("FunctionNotFoundException should be thrown");
+            fail("FunctionNotFoundException should be thrown");        
         }
         try {
             rs = stmt.executeQuery("select mysum9() from t9");
@@ -612,15 +611,13 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT {
         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){
-            
+        } catch(FunctionNotFoundException e) {
         }
     }
 
     @Test
     public void testDropFunction() throws Exception {
-        Connection conn = driver.connect(url, EMPTY_PROPS);
+        Connection conn = driver.connect(calciteUrl, UDF_PROPS);
         Statement stmt = conn.createStatement();
         String query = "select count(*) from "+ SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_FUNCTION_TABLE + "\"";
         ResultSet rs = stmt.executeQuery(query);
@@ -636,18 +633,16 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT {
         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)");
+        conn.createStatement().execute("create table t6(k integer not null 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");
@@ -666,9 +661,9 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT {
     @Test
     public void testUDFsWhenTimestampManagedAtClient() throws Exception {
         long ts = 100;
-        Properties props = new Properties();
+        Properties props = PropertiesUtil.deepCopy(UDF_PROPS);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts));
-        Connection conn = DriverManager.getConnection(url, props);
+        Connection conn = DriverManager.getConnection(calciteUrl, props);
         Statement stmt = conn.createStatement();
         String query = "select count(*) from "+ SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_FUNCTION_TABLE + "\"";
         ResultSet rs = stmt.executeQuery(query);
@@ -677,31 +672,28 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT {
         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();
+        conn = DriverManager.getConnection(calciteUrl, props);
         rs = stmt.executeQuery(query);
         rs.next();
         int numRowsAfter= rs.getInt(1);
         assertEquals(3, numRowsAfter - numRowsBefore);
-        stmt.execute("drop function mysum61");
+        conn.createStatement().execute("drop function mysum61");
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 20));
-        conn = DriverManager.getConnection(url, props);
+        conn = DriverManager.getConnection(calciteUrl, 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)");
+        stmt.execute("create table t62(k integer not null 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");
@@ -711,16 +703,16 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT {
         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);
+        conn = DriverManager.getConnection(calciteUrl, 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)");
+        conn.createStatement().execute("create table t61(k integer not null primary key, k1 integer, lastname varchar)");
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 40));
-        conn = DriverManager.getConnection(url, props);
+        conn = DriverManager.getConnection(calciteUrl, props);
         stmt = conn.createStatement();
         stmt.execute("upsert into t61 values(1,1,'jock')");
         conn.commit();
@@ -729,7 +721,7 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT {
         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);
+        conn = DriverManager.getConnection(calciteUrl, props);
         stmt = conn.createStatement();
         rs = stmt.executeQuery("select myfunction6(k,12) from t61");
         assertTrue(rs.next());
@@ -743,29 +735,28 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT {
         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);
+        conn = DriverManager.getConnection(calciteUrl, 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);
+        conn = DriverManager.getConnection(calciteUrl, 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);
+        Connection conn = driver.connect(calciteUrl, UDF_PROPS);
         Statement stmt = conn.createStatement();
-        stmt.execute("create table t5(k integer primary key, k1 integer, lastname_reverse varchar)");
+        stmt.execute("create table t5(k integer not null 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();
@@ -943,7 +934,7 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT {
         String query = "SELECT /*+ USE_SORT_MERGE_JOIN*/ item.\"item_id\", item.name, supp.\"supplier_id\", myreverse8(supp.name) FROM "
                 + JOIN_SUPPLIER_TABLE_FULL_NAME + " supp RIGHT JOIN " + JOIN_ITEM_TABLE_FULL_NAME
                 + " item ON myreverse8(item.\"supplier_id\") = myreverse8(supp.\"supplier_id\") ORDER BY \"item_id\"";
-        Connection conn = driver.connect(url, EMPTY_PROPS);
+        Connection conn = driver.connect(calciteUrl, UDF_PROPS);
         initJoinTableValues(conn);
         conn.createStatement().execute(
                 "create function myreverse8(VARCHAR) returns VARCHAR as 'org.apache.phoenix.end2end.MyReverse' using jar "
@@ -995,9 +986,9 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT {
 
     @Test
     public void testReplaceFunction() throws Exception {
-        Connection conn = driver.connect(url, EMPTY_PROPS);
+        Connection conn = driver.connect(calciteUrl, UDF_PROPS);
         Statement stmt = conn.createStatement();
-        conn.createStatement().execute("create table t10(k integer primary key, k1 integer, lastname varchar)");
+        conn.createStatement().execute("create table t10(k integer not null 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 "
@@ -1013,7 +1004,7 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT {
         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);
+        Connection conn2 = driver.connect(calciteUrl, UDF_PROPS);
         stmt = conn2.createStatement();
         rs = stmt.executeQuery("select myfunction63(k,12) from t10");
         assertTrue(rs.next());
@@ -1028,9 +1019,9 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT {
 
     @Test
     public void testUDFsWithSameChildrenInAQuery() throws Exception {
-        Connection conn = driver.connect(url, EMPTY_PROPS);
+        Connection conn = driver.connect(calciteUrl, UDF_PROPS);
         Statement stmt = conn.createStatement();
-        conn.createStatement().execute("create table t11(k varbinary primary key, k1 integer, lastname varchar)");
+        conn.createStatement().execute("create table t11(k varbinary not null primary key, k1 integer, lastname varchar)");
         String query = "UPSERT INTO t11"
                 + "(k, k1, lastname) "
                 + "VALUES(?,?,?)";
@@ -1110,9 +1101,9 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT {
             jarFos.close();
             
             assertTrue(jarFile.exists());
-            Connection conn = driver.connect(url, EMPTY_PROPS);
+            Connection conn = driver.connect(calciteUrl, UDF_PROPS);
             Statement stmt = conn.createStatement();
-            stmt.execute("add jars '"+jarFile.getAbsolutePath()+"'");
+            stmt.execute("upload jars '"+jarFile.getAbsolutePath()+"'");
         } finally {
             if (javaFile != null) javaFile.delete();
             if (classFile != null) classFile.delete();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/54cc2403/phoenix-core/src/main/codegen/data/Parser.tdd
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/codegen/data/Parser.tdd b/phoenix-core/src/main/codegen/data/Parser.tdd
index 68a0f43..61781e3 100644
--- a/phoenix-core/src/main/codegen/data/Parser.tdd
+++ b/phoenix-core/src/main/codegen/data/Parser.tdd
@@ -29,6 +29,7 @@
     "org.apache.phoenix.parse.ColumnDefInPkConstraint",
     "org.apache.calcite.sql.SqlColumnDefNode"
     "org.apache.calcite.sql.SqlColumnDefInPkConstraintNode"
+    "org.apache.calcite.sql.SqlFunctionArguementNode"
     "org.apache.calcite.sql.SqlIndexExpressionNode"
     "org.apache.calcite.sql.SqlDataTypeNode"
     "org.apache.calcite.sql.SqlOptionNode"
@@ -46,6 +47,12 @@
     "ROW_TIMESTAMP"
     "SPLIT"
     "STATISTICS"
+    "REPLACE"
+    "JAR"
+    "CONSTANT"
+    "DEFAULTVALUE"
+    "JARS"
+    "UPLOAD"
   ]
 
   # List of keywords from "keywords" section that are not reserved.
@@ -64,6 +71,10 @@
     "SqlDropIndex()",
     "SqlDropSequence()",
     "SqlUpdateStatistics()",
+    "SqlCreateFunction()"
+    "SqlDropFunction()"
+    "SqlUploadJarsNode()"
+    "SqlDeleteJarNode()"
   ]
 
   # List of methods for parsing custom literals.

http://git-wip-us.apache.org/repos/asf/phoenix/blob/54cc2403/phoenix-core/src/main/codegen/includes/parserImpls.ftl
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/codegen/includes/parserImpls.ftl b/phoenix-core/src/main/codegen/includes/parserImpls.ftl
index 58018d5..e86ef85 100644
--- a/phoenix-core/src/main/codegen/includes/parserImpls.ftl
+++ b/phoenix-core/src/main/codegen/includes/parserImpls.ftl
@@ -420,6 +420,115 @@ SqlNode SqlUpdateStatistics() :
     }
 }
 
+/**
+ * Parses statement
+ *   CREATE FUNCTION
+ */
+SqlNode SqlCreateFunction() :
+{
+    SqlParserPos pos;
+    SqlIdentifier functionName;
+    boolean replace = false;
+    boolean tempFunction = false;
+    SqlNodeList functionArguements = SqlNodeList.EMPTY;
+    SqlIdentifier returnType;
+    SqlNode className;
+    SqlNode jarPath = null;
+}
+{
+    <CREATE> { pos = getPos(); }
+    [
+    	<OR> <REPLACE> { replace = true; }
+    ]
+    [
+        <TEMPORARY> { tempFunction = true; }
+    ]   
+    <FUNCTION>
+    functionName = SimpleIdentifier()
+    <LPAREN>
+    [
+        functionArguements = FunctionArguementsList()
+    ]
+    <RPAREN>
+    <RETURNS> 
+    returnType = TypeName()
+    <AS>
+    className = StringLiteral()
+    [
+    	<USING> <JAR>
+    	jarPath = StringLiteral()
+    ] 
+    {
+        return new SqlCreateFunction(pos.plus(getPos()), functionName,
+            SqlLiteral.createBoolean(replace, SqlParserPos.ZERO),SqlLiteral.createBoolean(tempFunction, SqlParserPos.ZERO),
+            functionArguements, returnType, className,jarPath);
+    }
+}
+
+/**
+ * Parses statement
+ *   DROP FUNCTION
+ */
+SqlNode SqlDropFunction() :
+{
+    SqlParserPos pos;
+    SqlIdentifier functionName;
+    boolean ifExists;
+}
+{
+    <DROP> { pos = getPos(); } <FUNCTION>
+    (
+        <IF> <EXISTS> { ifExists = true; }
+        |
+        {
+            ifExists = false;
+        }
+    )
+    functionName = SimpleIdentifier()
+    {
+        return new  SqlDropFunction(pos.plus(getPos()), functionName,
+            SqlLiteral.createBoolean(ifExists, SqlParserPos.ZERO));
+    }
+}
+
+/**
+ * Parses statement
+ *   UPLOAD JARS
+ */
+SqlNode SqlUploadJarsNode() :
+{
+    SqlParserPos pos;
+    SqlNode jarPath;
+    List<SqlNode> jarPathsList;
+}
+{
+    <UPLOAD>  { pos = getPos(); } <JARS> 
+    jarPath = StringLiteral() { jarPathsList = startList(jarPath); }
+    (
+        <COMMA> jarPath = StringLiteral() { jarPathsList.add(jarPath); }
+    ) *
+    {
+        return new  SqlUploadJarsNode(pos.plus(getPos()), jarPathsList);
+    }
+}
+
+/**
+ * Parses statement
+ *   DELETE JAR
+ */
+SqlNode SqlDeleteJarNode() :
+{
+    SqlParserPos pos;
+    SqlNode jarPath;
+}
+{
+    <DELETE> { pos = getPos(); } <JAR>
+    jarPath = StringLiteral()
+    {
+        return new  SqlDeleteJarNode(pos.plus(getPos()), jarPath);
+    }
+}
+
 SqlNodeList ColumnDefList() :
 {
     SqlParserPos pos;
@@ -719,3 +828,52 @@ SqlIdentifier DualIdentifier() :
         return new SqlIdentifier(list, null, pos, posList);
     }
 }
+
+SqlFunctionArguementNode FunctionArguement() :
+{
+	SqlDataTypeNode typeNode;
+	boolean isConstant = false;
+	SqlNode defaultValue = null;
+	SqlNode minValue = null;
+	SqlNode maxValue = null;
+	SqlParserPos pos;
+} 
+{
+	typeNode = PhoenixDataType()
+	[
+		<CONSTANT> { isConstant = true; }
+	]
+	[
+		<DEFAULTVALUE> <EQ> 
+			defaultValue = Expression(ExprContext.ACCEPT_NONQUERY)
+	]
+	[
+		<MINVALUE> <EQ> 
+			minValue = Expression(ExprContext.ACCEPT_NONQUERY)
+	]
+	[
+		<MAXVALUE> <EQ> 
+			maxValue = Expression(ExprContext.ACCEPT_NONQUERY)
+	]
+	{
+		pos = typeNode.getParserPosition().plus(getPos());
+		return new SqlFunctionArguementNode(pos, typeNode, isConstant, defaultValue, minValue, maxValue);
+	}
+}
+
+SqlNodeList FunctionArguementsList() :
+{
+    SqlParserPos pos;
+    SqlNode e;
+    List<SqlNode> functionArguements;
+}
+{
+    { pos = getPos(); }
+    e = FunctionArguement() { functionArguements = startList(e); }
+    (
+        <COMMA> e = FunctionArguement() { functionArguements.add(e); }
+    ) *
+    {
+        return new SqlNodeList(functionArguements, pos.plus(getPos()));
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/54cc2403/phoenix-core/src/main/java/org/apache/calcite/sql/ListJarsTable.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/calcite/sql/ListJarsTable.java b/phoenix-core/src/main/java/org/apache/calcite/sql/ListJarsTable.java
new file mode 100644
index 0000000..db2b003
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/calcite/sql/ListJarsTable.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.sql;
+
+import java.io.IOException;
+import java.lang.reflect.Method;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.calcite.DataContext;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.linq4j.Linq4j;
+import org.apache.calcite.linq4j.tree.Types;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.schema.ScannableTable;
+import org.apache.calcite.schema.impl.AbstractTable;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.Smalls.MazeTable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.phoenix.calcite.PhoenixPrepareImpl;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.query.HBaseFactoryProvider;
+import org.apache.phoenix.query.QueryServices;
+
+public class ListJarsTable extends AbstractTable implements ScannableTable {
+
+    @Override
+    public RelDataType getRowType(RelDataTypeFactory typeFactory) {
+        return typeFactory.builder()
+                .add("JAR_LOCATION", SqlTypeName.VARCHAR)
+                .build();
+    }
+
+    public static final Method LIST_JARS_TABLE_METHOD =
+            Types.lookupMethod(ListJarsTable.class, "generate");
+
+    public static ScannableTable generate() {
+        return new ListJarsTable();
+    }
+    @Override
+    public Enumerable<Object[]> scan(DataContext root) {
+        List<Object[]> filePaths = new ArrayList<Object[]>(1);
+        PhoenixConnection phoenixConnection =
+                PhoenixPrepareImpl.getPhoenixConnection(root.getRootSchema());
+        String dynamicJarsDir =
+                phoenixConnection.getQueryServices().getProps()
+                        .get(QueryServices.DYNAMIC_JARS_DIR_KEY);
+        if (dynamicJarsDir == null) {
+            throw new RuntimeException(new SQLException(QueryServices.DYNAMIC_JARS_DIR_KEY
+                    + " is not configured for the listing the jars."));
+        }
+        dynamicJarsDir = dynamicJarsDir.endsWith("/") ? dynamicJarsDir : dynamicJarsDir + '/';
+        Configuration conf = HBaseFactoryProvider.getConfigurationFactory().getConfiguration();
+        Path dynamicJarsDirPath = new Path(dynamicJarsDir);
+        FileSystem fs;
+        try {
+            fs = dynamicJarsDirPath.getFileSystem(conf);
+            RemoteIterator<LocatedFileStatus> listFiles = fs.listFiles(dynamicJarsDirPath, true);
+            if (listFiles != null) {
+                while (listFiles.hasNext()) {
+                    LocatedFileStatus file = listFiles.next();
+                    filePaths.add(new Object[] { file.getPath().toString() });
+                }
+            }
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+        return Linq4j.asEnumerable(filePaths);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/54cc2403/phoenix-core/src/main/java/org/apache/calcite/sql/SqlFunctionArguementNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/calcite/sql/SqlFunctionArguementNode.java b/phoenix-core/src/main/java/org/apache/calcite/sql/SqlFunctionArguementNode.java
new file mode 100644
index 0000000..564cfec
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/calcite/sql/SqlFunctionArguementNode.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.sql;
+
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.util.SqlVisitor;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
+import org.apache.calcite.util.Litmus;
+
+public class SqlFunctionArguementNode extends SqlNode {
+
+    public final SqlDataTypeNode typeNode;
+    public final boolean isConstant;
+    public final SqlNode defaultValue;
+    public final SqlNode minValue;
+    public final SqlNode maxValue;
+
+    public SqlFunctionArguementNode(SqlParserPos pos, SqlDataTypeNode typeNode, boolean isConstant,
+            SqlNode defaultValue, SqlNode minValue, SqlNode maxValue) {
+        super(pos);
+        this.typeNode = typeNode;
+        this.isConstant = isConstant;
+        this.defaultValue = defaultValue;
+        this.minValue = minValue;
+        this.maxValue = maxValue;
+    }
+
+    @Override
+    public <R> R accept(SqlVisitor<R> arg0) {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
+    @Override
+    public boolean equalsDeep(SqlNode arg0, Litmus arg1) {
+        // TODO Auto-generated method stub
+        return false;
+    }
+
+    @Override
+    public void unparse(SqlWriter arg0, int arg1, int arg2) {
+        // TODO Auto-generated method stub
+
+    }
+
+    @Override
+    public void validate(SqlValidator arg0, SqlValidatorScope arg1) {
+        // TODO Auto-generated method stub
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/54cc2403/phoenix-core/src/main/java/org/apache/phoenix/calcite/CalciteUtils.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/calcite/CalciteUtils.java b/phoenix-core/src/main/java/org/apache/phoenix/calcite/CalciteUtils.java
index ae143f7..0668da2 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/calcite/CalciteUtils.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/CalciteUtils.java
@@ -30,6 +30,7 @@ import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.schema.Function;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.sql.SemiJoinType;
 import org.apache.calcite.sql.SqlAggFunction;
@@ -45,6 +46,7 @@ import org.apache.calcite.sql.parser.SqlParser;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.type.ArraySqlType;
 import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.validate.SqlUserDefinedFunction;
 import org.apache.calcite.tools.FrameworkConfig;
 import org.apache.calcite.tools.Frameworks;
 import org.apache.calcite.tools.Planner;
@@ -103,6 +105,7 @@ import org.apache.phoenix.expression.function.RoundTimestampExpression;
 import org.apache.phoenix.expression.function.SqrtFunction;
 import org.apache.phoenix.expression.function.SumAggregateFunction;
 import org.apache.phoenix.expression.function.TrimFunction;
+import org.apache.phoenix.expression.function.UDFExpression;
 import org.apache.phoenix.expression.function.UpperFunction;
 import org.apache.phoenix.parse.JoinTableNode.JoinType;
 import org.apache.phoenix.parse.SequenceValueParseNode;
@@ -123,6 +126,7 @@ import org.apache.phoenix.schema.types.PUnsignedTimestamp;
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
+
 import org.apache.phoenix.util.ExpressionUtil;
 
 /**
@@ -738,7 +742,14 @@ public class CalciteUtils {
                 List<Expression> children = convertChildren(call, implementor);
                 SqlOperator op = call.getOperator();
                 try {
-                    if (op == SqlStdOperatorTable.SQRT) {
+                    if (op instanceof SqlUserDefinedFunction) {
+                        SqlUserDefinedFunction udf = (SqlUserDefinedFunction) op;
+                        Function func = udf.getFunction();
+                        if (func instanceof PhoenixScalarFunction) {
+                            PhoenixScalarFunction scalarFunc = (PhoenixScalarFunction) func;
+                            return new UDFExpression(children, scalarFunc.getFunctionInfo());
+                        }
+                    } else if (op == SqlStdOperatorTable.SQRT) {
                         return new SqrtFunction(children);
                     } else if (op == SqlStdOperatorTable.POWER) {
                         return new PowerFunction(children);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/54cc2403/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixPrepareImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixPrepareImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixPrepareImpl.java
index d00161b..8ac009e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixPrepareImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixPrepareImpl.java
@@ -2,6 +2,7 @@ package org.apache.phoenix.calcite;
 
 import java.lang.reflect.Type;
 import java.sql.SQLException;
+import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -27,6 +28,7 @@ import org.apache.calcite.runtime.Hook.Closeable;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.sql.SqlColumnDefInPkConstraintNode;
 import org.apache.calcite.sql.SqlColumnDefNode;
+import org.apache.calcite.sql.SqlFunctionArguementNode;
 import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlIndexExpressionNode;
 import org.apache.calcite.sql.SqlKind;
@@ -42,13 +44,17 @@ import org.apache.calcite.tools.Programs;
 import org.apache.calcite.util.Holder;
 import org.apache.calcite.util.NlsString;
 import org.apache.hadoop.hbase.util.Pair;
+import org.apache.phoenix.calcite.parse.SqlCreateFunction;
 import org.apache.phoenix.calcite.parse.SqlCreateIndex;
 import org.apache.phoenix.calcite.parse.SqlCreateSequence;
 import org.apache.phoenix.calcite.parse.SqlCreateTable;
+import org.apache.phoenix.calcite.parse.SqlDeleteJarNode;
+import org.apache.phoenix.calcite.parse.SqlDropFunction;
 import org.apache.phoenix.calcite.parse.SqlDropIndex;
 import org.apache.phoenix.calcite.parse.SqlDropSequence;
 import org.apache.phoenix.calcite.parse.SqlDropTable;
 import org.apache.phoenix.calcite.parse.SqlUpdateStatistics;
+import org.apache.phoenix.calcite.parse.SqlUploadJarsNode;
 import org.apache.phoenix.calcite.parser.PhoenixParserImpl;
 import org.apache.phoenix.calcite.rel.PhoenixRel;
 import org.apache.phoenix.calcite.rel.PhoenixServerProject;
@@ -61,25 +67,32 @@ import org.apache.phoenix.calcite.rules.PhoenixOrderedAggregateRule;
 import org.apache.phoenix.calcite.rules.PhoenixReverseTableScanRule;
 import org.apache.phoenix.calcite.rules.PhoenixSortServerJoinTransposeRule;
 import org.apache.phoenix.calcite.rules.PhoenixTableScanColumnRefRule;
+import org.apache.phoenix.compile.BaseMutationPlan;
 import org.apache.phoenix.compile.CreateIndexCompiler;
 import org.apache.phoenix.compile.CreateSequenceCompiler;
 import org.apache.phoenix.compile.CreateTableCompiler;
 import org.apache.phoenix.compile.MutationPlan;
+import org.apache.phoenix.expression.LiteralExpression;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.jdbc.PhoenixStatement.Operation;
 import org.apache.phoenix.parse.ColumnDef;
 import org.apache.phoenix.parse.ColumnDefInPkConstraint;
 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.DropFunctionStatement;
 import org.apache.phoenix.parse.DropIndexStatement;
 import org.apache.phoenix.parse.DropSequenceStatement;
 import org.apache.phoenix.parse.DropTableStatement;
 import org.apache.phoenix.parse.IndexKeyConstraint;
+import org.apache.phoenix.parse.LiteralParseNode;
 import org.apache.phoenix.parse.NamedNode;
 import org.apache.phoenix.parse.NamedTableNode;
+import org.apache.phoenix.parse.PFunction;
+import org.apache.phoenix.parse.PFunction.FunctionArgument;
 import org.apache.phoenix.parse.ParseNode;
 import org.apache.phoenix.parse.ParseNodeFactory;
 import org.apache.phoenix.parse.PrimaryKeyConstraint;
@@ -90,6 +103,7 @@ import org.apache.phoenix.parse.UpdateStatisticsStatement;
 import org.apache.phoenix.schema.MetaDataClient;
 import org.apache.phoenix.schema.PTable.IndexType;
 import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.schema.Sequence;
 import org.apache.phoenix.schema.SortOrder;
 
 import com.google.common.base.Function;
@@ -426,6 +440,86 @@ public class PhoenixPrepareImpl extends CalcitePrepareImpl {
                     final UpdateStatisticsStatement updateStatsStmt = nodeFactory.updateStatistics(table, updateStatsNode.scope, props);
                     MetaDataClient client = new MetaDataClient(connection);
                     client.updateStatistics(updateStatsStmt);                    
+                } else if (node instanceof SqlCreateFunction) {
+                    SqlCreateFunction createFunctionNode = (SqlCreateFunction) node;
+                    short i = 0;
+                    List<FunctionArgument> functionArguements =
+                            new ArrayList<FunctionArgument>(
+                                    createFunctionNode.functionArguements.size());
+                    for (SqlNode functionArguement : createFunctionNode.functionArguements) {
+                        LiteralExpression dvExpression = null;
+                        LiteralExpression minValueExpression = null;
+                        LiteralExpression maxValueExpression = null;
+                        SqlFunctionArguementNode funArgNode =
+                                (SqlFunctionArguementNode) functionArguement;
+                        if (funArgNode.defaultValue != null) {
+                            LiteralParseNode dv =
+                                    (LiteralParseNode) convertSqlNodeToParseNode(funArgNode.defaultValue);
+                            dvExpression = LiteralExpression.newConstant(dv.getValue());
+                        }
+                        if (funArgNode.minValue != null) {
+                            LiteralParseNode minValue =
+                                    (LiteralParseNode) convertSqlNodeToParseNode(funArgNode.minValue);
+                            minValueExpression = LiteralExpression.newConstant(minValue.getValue());
+                        }
+                        if (funArgNode.maxValue != null) {
+                            LiteralParseNode maxValue =
+                                    (LiteralParseNode) convertSqlNodeToParseNode(funArgNode.maxValue);
+                            maxValueExpression = LiteralExpression.newConstant(maxValue.getValue());
+                        }
+                        functionArguements.add(new PFunction.FunctionArgument(
+                                funArgNode.typeNode.typeName, funArgNode.typeNode.isArray,
+                                funArgNode.isConstant, dvExpression, minValueExpression,
+                                maxValueExpression, i));
+                        i++;
+                    }
+
+                    final SqlLiteral className = (SqlLiteral) createFunctionNode.className;
+                    String quotedClassNameStr = ((NlsString) className.getValue()).toString();
+                    String classNameStr = quotedClassNameStr.substring(1, quotedClassNameStr.length() - 1);
+                    String jarPathStr = null;
+                    if (createFunctionNode.jarPath != null) {
+                        final SqlLiteral jarPath = (SqlLiteral) createFunctionNode.jarPath;
+                        String quotedJarPathStr = ((NlsString) jarPath.getValue()).toString();
+                        jarPathStr = quotedJarPathStr.substring(1, quotedJarPathStr.length() - 1);
+                    }
+                    PFunction function =
+                            new PFunction(createFunctionNode.functionName.getSimple(),
+                                    functionArguements, createFunctionNode.returnType.getSimple(),
+                                    classNameStr, jarPathStr);
+                    CreateFunctionStatement createFunction =
+                            nodeFactory.createFunction(function,
+                                createFunctionNode.tempFunction.booleanValue(),
+                                createFunctionNode.replace.booleanValue());
+                    MetaDataClient client = new MetaDataClient(connection);
+                    client.createFunction(createFunction);
+                } else if (node instanceof SqlDropFunction) {
+                    SqlDropFunction dropFunctionNode = (SqlDropFunction) node;
+                    DropFunctionStatement dropFunctionStmt =
+                            new DropFunctionStatement(dropFunctionNode.functionName.getSimple(),
+                                    dropFunctionNode.ifExists.booleanValue());
+                    MetaDataClient client = new MetaDataClient(connection);
+                    client.dropFunction(dropFunctionStmt);
+                } else if (node instanceof SqlUploadJarsNode) {
+                    PhoenixStatement phoenixStatement = new PhoenixStatement(connection);
+                    List<SqlNode> operandList = ((SqlUploadJarsNode) node).getOperandList();
+                    List<LiteralParseNode> jarsPaths = new ArrayList<LiteralParseNode>();
+                    for (SqlNode jarPath : operandList) {
+                        jarsPaths.add((LiteralParseNode) convertSqlNodeToParseNode(jarPath));
+                    }
+                    MutationPlan compilePlan =
+                            new PhoenixStatement.ExecutableAddJarsStatement(jarsPaths).compilePlan(phoenixStatement,
+                                Sequence.ValueOp.VALIDATE_SEQUENCE);
+                    ((BaseMutationPlan) compilePlan).execute();
+                } else if (node instanceof SqlDeleteJarNode) {
+                    PhoenixStatement phoenixStatement = new PhoenixStatement(connection);
+                    List<SqlNode> operandList = ((SqlDeleteJarNode) node).getOperandList();
+                    LiteralParseNode jarPath =
+                            (LiteralParseNode) convertSqlNodeToParseNode(operandList.get(0));
+                    MutationPlan compilePlan =
+                            new PhoenixStatement.ExecutableDeleteJarStatement(jarPath).compilePlan(phoenixStatement,
+                                Sequence.ValueOp.VALIDATE_SEQUENCE);
+                    ((BaseMutationPlan) compilePlan).execute();
                 } else {
                     throw new AssertionError("unknown DDL node " + node.getClass());                    
                 }
@@ -482,7 +576,7 @@ public class PhoenixPrepareImpl extends CalcitePrepareImpl {
         return splits;
     }
     
-    private static PhoenixConnection getPhoenixConnection(SchemaPlus rootSchema) {
+    public static PhoenixConnection getPhoenixConnection(SchemaPlus rootSchema) {
         for (String subSchemaName : rootSchema.getSubSchemaNames()) {               
             try {
                 PhoenixSchema phoenixSchema = rootSchema

http://git-wip-us.apache.org/repos/asf/phoenix/blob/54cc2403/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixScalarFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixScalarFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixScalarFunction.java
new file mode 100644
index 0000000..a65f851
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixScalarFunction.java
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.calcite;
+
+import java.util.List;
+
+import org.apache.calcite.adapter.enumerable.CallImplementor;
+import org.apache.calcite.adapter.enumerable.RexToLixTranslator;
+import org.apache.calcite.adapter.enumerable.RexImpTable.NullAs;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.schema.FunctionParameter;
+import org.apache.calcite.schema.ImplementableFunction;
+import org.apache.calcite.schema.ScalarFunction;
+import org.apache.phoenix.parse.PFunction;
+import org.apache.phoenix.parse.PFunction.FunctionArgument;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.util.SchemaUtil;
+
+import com.google.common.collect.Lists;
+
+public class PhoenixScalarFunction implements ScalarFunction, ImplementableFunction {
+    private final PFunction functionInfo;
+    private final PDataType returnType;
+    private final List<FunctionParameter> parameters;
+    
+    public PhoenixScalarFunction(PFunction functionInfo) {
+        this.functionInfo = functionInfo;
+        this.returnType =
+                PDataType.fromSqlTypeName(SchemaUtil.normalizeIdentifier(functionInfo.getReturnType()));
+        this.parameters = Lists.newArrayListWithExpectedSize(functionInfo.getFunctionArguments().size());
+        for (final FunctionArgument arg : functionInfo.getFunctionArguments()) {
+            parameters.add(
+                    new FunctionParameter() {
+                        public int getOrdinal() {
+                            return arg.getArgPosition();
+                        }
+
+                        public String getName() {
+                            return getArgumentName(arg.getArgPosition());
+                        }
+
+                        public RelDataType getType(RelDataTypeFactory typeFactory) {
+                            PDataType dataType =
+                                    arg.isArrayType() ? PDataType.fromTypeId(PDataType.sqlArrayType(SchemaUtil
+                                            .normalizeIdentifier(SchemaUtil.normalizeIdentifier(arg
+                                                    .getArgumentType())))) : PDataType.fromSqlTypeName(SchemaUtil
+                                            .normalizeIdentifier(arg.getArgumentType()));
+                            return typeFactory.createJavaType(dataType.getJavaClass());
+                        }
+
+                        public boolean isOptional() {
+                            return arg.getDefaultValue() != null;
+                        }
+                    });
+        }
+    }
+
+    @Override
+    public RelDataType getReturnType(RelDataTypeFactory typeFactory) {
+        return typeFactory.createJavaType(returnType.getJavaClass());
+    }
+
+    @Override
+    public List<FunctionParameter> getParameters() {
+        return parameters;
+    }
+    
+    public PFunction getFunctionInfo() {
+        return functionInfo;
+    }
+
+    private static String getArgumentName(int ordinal) {
+        return "arg" + ordinal;
+    }
+
+    @Override
+    public CallImplementor getImplementor() {
+        return new CallImplementor() {
+            public Expression implement(RexToLixTranslator translator, RexCall call, NullAs nullAs) {
+                return Expressions.constant(null);
+            }
+        };
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/54cc2403/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixSchema.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixSchema.java b/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixSchema.java
index 3f2d6ea..ac918b7 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixSchema.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixSchema.java
@@ -9,13 +9,17 @@ import org.apache.calcite.jdbc.CalciteSchema;
 import org.apache.calcite.linq4j.tree.Expression;
 import org.apache.calcite.materialize.MaterializationService;
 import org.apache.calcite.schema.*;
+import org.apache.calcite.schema.impl.TableFunctionImpl;
 import org.apache.calcite.schema.impl.ViewTable;
+import org.apache.calcite.sql.ListJarsTable;
 import org.apache.phoenix.compile.ColumnResolver;
 import org.apache.phoenix.compile.FromCompiler;
+import org.apache.phoenix.expression.function.UDFExpression;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.parse.ColumnDef;
 import org.apache.phoenix.parse.NamedTableNode;
+import org.apache.phoenix.parse.PFunction;
 import org.apache.phoenix.parse.TableName;
 import org.apache.phoenix.schema.MetaDataClient;
 import org.apache.phoenix.schema.PColumn;
@@ -32,6 +36,7 @@ import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.ResultSet;
 import java.sql.SQLException;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
@@ -62,6 +67,9 @@ public class PhoenixSchema implements Schema {
     protected final Map<String, Table> tables;
     protected final Map<String, Function> views;
     protected final Set<TableRef> viewTables;
+    protected final UDFExpression exp = new UDFExpression();
+    private final static Function listJarsFunction = TableFunctionImpl
+            .create(ListJarsTable.LIST_JARS_TABLE_METHOD);
     
     protected PhoenixSchema(String name, String schemaName,
             SchemaPlus parentSchema, PhoenixConnection pc) {
@@ -73,7 +81,9 @@ public class PhoenixSchema implements Schema {
         this.subSchemas = Maps.newHashMap();
         this.tables = Maps.newHashMap();
         this.views = Maps.newHashMap();
+        this.views.put("ListJars", listJarsFunction);
         this.viewTables = Sets.newHashSet();
+
     }
 
     private static Schema create(SchemaPlus parentSchema,
@@ -143,7 +153,19 @@ public class PhoenixSchema implements Schema {
         if (func != null) {
             return ImmutableList.of(func);
         }
-        
+        try {
+            List<String> functionNames = new ArrayList<String>(1);
+            functionNames.add(name);
+            ColumnResolver resolver = FromCompiler.getResolverForFunction(pc, functionNames);
+            List<PFunction> pFunctions = resolver.getFunctions();
+            assert !pFunctions.isEmpty();
+            List<Function> funcs = new ArrayList<Function>(pFunctions.size());
+            for (PFunction pFunction : pFunctions) {
+                funcs.add(new PhoenixScalarFunction(pFunction));
+            }
+            return ImmutableList.copyOf(funcs);
+        } catch (SQLException e) {
+        }
         try {
             ColumnResolver x = FromCompiler.getResolver(
                     NamedTableNode.create(
@@ -222,6 +244,7 @@ public class PhoenixSchema implements Schema {
     public void clear() {
         tables.clear();
         views.clear();
+        this.views.put("ListJars", listJarsFunction);
         viewTables.clear();
     }
     

http://git-wip-us.apache.org/repos/asf/phoenix/blob/54cc2403/phoenix-core/src/main/java/org/apache/phoenix/calcite/parse/SqlCreateFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/calcite/parse/SqlCreateFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/calcite/parse/SqlCreateFunction.java
new file mode 100644
index 0000000..8de4e0e
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/parse/SqlCreateFunction.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.calcite.parse;
+
+import java.util.List;
+
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.parser.SqlParserPos;
+
+import com.google.common.collect.ImmutableList;
+
+public class SqlCreateFunction extends SqlCall {
+    public static final SqlOperator OPERATOR = new SqlDdlOperator("CREATE FUNCTION",
+            SqlKind.OTHER_DDL);
+    public final SqlIdentifier functionName;
+    public final SqlLiteral replace;
+    public final SqlLiteral tempFunction;
+    public final SqlNodeList functionArguements;
+    public final SqlIdentifier returnType;
+    public final SqlNode className;
+    public final SqlNode jarPath;
+
+    public SqlCreateFunction(SqlParserPos pos, SqlIdentifier functionName, SqlLiteral replace,
+            SqlLiteral tempFunction, SqlNodeList functionArguements, SqlIdentifier returnType,
+            SqlNode className, SqlNode jarPath) {
+        super(pos);
+        this.functionName = functionName;
+        this.replace = replace;
+        this.tempFunction = tempFunction;
+        this.functionArguements = functionArguements;
+        this.returnType = returnType;
+        this.className = className;
+        this.jarPath = jarPath;
+    }
+
+    public List<SqlNode> getOperandList() {
+        return ImmutableList.of(functionName, replace, tempFunction, functionArguements,
+            returnType, className, jarPath);
+    }
+
+    @Override
+    public SqlOperator getOperator() {
+        return OPERATOR;
+    }
+
+    @Override
+    public void unparse(SqlWriter arg0, int arg1, int arg2) {
+        // TODO Auto-generated method stub
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/54cc2403/phoenix-core/src/main/java/org/apache/phoenix/calcite/parse/SqlDeleteJarNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/calcite/parse/SqlDeleteJarNode.java b/phoenix-core/src/main/java/org/apache/phoenix/calcite/parse/SqlDeleteJarNode.java
new file mode 100644
index 0000000..78809c7
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/parse/SqlDeleteJarNode.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.calcite.parse;
+
+import java.util.List;
+
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.parser.SqlParserPos;
+
+import com.google.common.collect.ImmutableList;
+
+public class SqlDeleteJarNode extends SqlCall {
+
+    public static final SqlOperator OPERATOR = new SqlDdlOperator("DELETE JAR",
+        SqlKind.OTHER_DDL);
+    public final SqlNode jarPath;
+
+    public SqlDeleteJarNode(SqlParserPos pos, SqlNode jarPath) {
+        super(pos);
+        this.jarPath = jarPath;
+    }
+
+    @Override
+    public SqlOperator getOperator() {
+        return OPERATOR;
+    }
+
+    @Override
+    public List<SqlNode> getOperandList() {
+        return ImmutableList.of(jarPath);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/54cc2403/phoenix-core/src/main/java/org/apache/phoenix/calcite/parse/SqlDropFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/calcite/parse/SqlDropFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/calcite/parse/SqlDropFunction.java
new file mode 100644
index 0000000..14eb73e
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/parse/SqlDropFunction.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.calcite.parse;
+
+import java.util.List;
+
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.parser.SqlParserPos;
+
+import com.google.common.collect.ImmutableList;
+
+public class SqlDropFunction extends SqlCall {
+    public static final SqlOperator OPERATOR = new SqlDdlOperator("DROP FUNCTION",
+            SqlKind.OTHER_DDL);
+    public final SqlIdentifier functionName;
+    public final SqlLiteral ifExists;
+
+    public SqlDropFunction(SqlParserPos pos, SqlIdentifier functionName, SqlLiteral ifExists) {
+        super(pos);
+        this.functionName = functionName;
+        this.ifExists = ifExists;
+    }
+
+    @Override
+    public SqlOperator getOperator() {
+        return OPERATOR;
+    }
+
+    @Override
+    public List<SqlNode> getOperandList() {
+        return ImmutableList.of(functionName, ifExists);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/54cc2403/phoenix-core/src/main/java/org/apache/phoenix/calcite/parse/SqlUploadJarsNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/calcite/parse/SqlUploadJarsNode.java b/phoenix-core/src/main/java/org/apache/phoenix/calcite/parse/SqlUploadJarsNode.java
new file mode 100644
index 0000000..feb210b
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/parse/SqlUploadJarsNode.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.calcite.parse;
+
+import java.util.List;
+
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.parser.SqlParserPos;
+
+public class SqlUploadJarsNode extends SqlCall {
+    public static final SqlOperator OPERATOR = new SqlDdlOperator("UPLOAD JARS",
+        SqlKind.OTHER_DDL);
+    public final List<SqlNode> jarPaths;
+
+    public SqlUploadJarsNode(SqlParserPos pos, List<SqlNode> jarPaths) {
+        super(pos);
+        this.jarPaths = jarPaths;
+    }
+
+    @Override
+    public SqlOperator getOperator() {
+        return OPERATOR;
+    }
+
+    @Override
+    public List<SqlNode> getOperandList() {
+        return jarPaths;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/54cc2403/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 8e4d9aa..232bb95 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
@@ -226,6 +226,10 @@ public class FromCompiler {
         return new SchemaResolver(connection, SchemaUtil.normalizeIdentifier(schema), true);
     }
 
+    public static ColumnResolver getResolverForFunction(PhoenixConnection connection, List<String> functionNames) throws SQLException {
+        return new FunctionResolver(connection, functionNames);
+    }
+
     public static ColumnResolver getResolver(NamedTableNode tableNode, PhoenixConnection connection) throws SQLException {
         SingleTableColumnResolver visitor = new SingleTableColumnResolver(connection, tableNode, true);
         return visitor;
@@ -463,6 +467,39 @@ public class FromCompiler {
         }
     }
 
+    private static class FunctionResolver extends BaseColumnResolver {
+        
+        public FunctionResolver(PhoenixConnection connection, List<String> functionNames) throws SQLException {
+            super(connection, 0, true, functionNames);
+        }
+        
+        @Override
+        public List<TableRef> getTables() {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public TableRef resolveTable(String schemaName, String tableName) throws SQLException {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public ColumnRef resolveColumn(String schemaName, String tableName, String colName)
+                throws SQLException {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public PSchema resolveSchema(String schemaName) throws SQLException {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public List<PSchema> getSchemas() {
+            throw new UnsupportedOperationException();
+        }
+    }
+
     private static abstract class BaseColumnResolver implements ColumnResolver {
         protected final PhoenixConnection connection;
         protected final MetaDataClient client;
@@ -481,15 +518,23 @@ public class FromCompiler {
             this.functions = Collections.<PFunction>emptyList();
         }
 
-        private BaseColumnResolver(PhoenixConnection connection, int tsAddition, boolean updateCacheImmediately, Map<String, UDFParseNode> udfParseNodes) throws SQLException {
-        	this.connection = connection;
+        private BaseColumnResolver(PhoenixConnection connection, int tsAddition,
+                boolean updateCacheImmediately, Map<String, UDFParseNode> udfParseNodes)
+                throws SQLException {
+            this(connection, tsAddition, updateCacheImmediately,
+                    udfParseNodes.isEmpty() ? Collections.<String> emptyList()
+                            : new ArrayList<String>(udfParseNodes.keySet()));
+        }
+
+        private BaseColumnResolver(PhoenixConnection connection, int tsAddition, boolean updateCacheImmediately, List<String> functionNames) 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()) {
+            if (functionNames.isEmpty()) {
                 functions = Collections.<PFunction> emptyList();
             } else {
-                functions = createFunctionRef(new ArrayList<String>(udfParseNodes.keySet()), updateCacheImmediately);
+                functions = createFunctionRef(functionNames, updateCacheImmediately);
                 for (PFunction function : functions) {
                     functionMap.put(function.getFunctionName(), function);
                 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/54cc2403/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 2bbe595..24cda3b 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
@@ -710,7 +710,7 @@ public class PhoenixStatement implements Statement, SQLCloseable {
         }
     }
     
-    private static class ExecutableAddJarsStatement extends AddJarsStatement implements CompilableStatement {
+    public static class ExecutableAddJarsStatement extends AddJarsStatement implements CompilableStatement {
 
         public ExecutableAddJarsStatement(List<LiteralParseNode> jarPaths) {
             super(jarPaths);
@@ -768,7 +768,7 @@ public class PhoenixStatement implements Statement, SQLCloseable {
         }
     }
 
-    private static class ExecutableDeleteJarStatement extends DeleteJarStatement implements CompilableStatement {
+    public static class ExecutableDeleteJarStatement extends DeleteJarStatement implements CompilableStatement {
 
         public ExecutableDeleteJarStatement(LiteralParseNode jarPath) {
             super(jarPath);