You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by jh...@apache.org on 2018/11/17 19:14:04 UTC

calcite git commit: [CALCITE-2663] Add CREATE and DROP FUNCTION (ambition119)

Repository: calcite
Updated Branches:
  refs/heads/master 614bc7b2b -> bd956458f


[CALCITE-2663] Add CREATE and DROP FUNCTION (ambition119)

Syntax is similar to Apache Hive's function syntax.

Close apache/calcite#908
Close apache/calcite#920
Close apache/calcite#913


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

Branch: refs/heads/master
Commit: bd956458f9dd321c3cc90daaf5c13f194b8feb03
Parents: 614bc7b
Author: ambition119 <1269223860>
Authored: Thu Nov 8 20:28:12 2018 +0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Fri Nov 16 20:10:33 2018 -0800

----------------------------------------------------------------------
 .../java/org/apache/calcite/sql/SqlKind.java    |  6 ++
 .../calcite/sql/validate/SqlValidatorUtil.java  |  3 +-
 .../main/java/org/apache/calcite/util/Util.java | 11 +++
 .../calcite/sql/parser/SqlParserTest.java       | 17 ++--
 .../calcite/sql/parser/SqlUnParserTest.java     |  4 +
 .../java/org/apache/calcite/util/UtilTest.java  | 13 ++-
 server/src/main/codegen/config.fmpp             | 10 ++
 .../src/main/codegen/includes/parserImpls.ftl   | 59 ++++++++++++
 .../calcite/sql/ddl/SqlCreateFunction.java      | 97 ++++++++++++++++++++
 .../org/apache/calcite/sql/ddl/SqlDdlNodes.java | 21 +++++
 .../apache/calcite/sql/ddl/SqlDropFunction.java | 39 ++++++++
 .../apache/calcite/sql/ddl/SqlDropObject.java   |  1 +
 .../apache/calcite/test/ServerParserTest.java   | 35 +++++++
 .../apache/calcite/test/ServerUnParserTest.java | 40 ++++++++
 site/_docs/reference.md                         | 16 ++++
 15 files changed, 363 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/bd956458/core/src/main/java/org/apache/calcite/sql/SqlKind.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlKind.java b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
index e56b066..fa8c2f0 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlKind.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
@@ -1059,6 +1059,12 @@ public enum SqlKind {
   /** {@code DROP TYPE} DDL statement. */
   DROP_TYPE,
 
+  /** {@code CREATE FUNCTION} DDL statement. */
+  CREATE_FUNCTION,
+
+  /** {@code DROP FUNCTION} DDL statement. */
+  DROP_FUNCTION,
+
   /** DDL statement not handled above.
    *
    * <p><b>Note to other projects</b>: If you are extending Calcite's SQL parser

http://git-wip-us.apache.org/repos/asf/calcite/blob/bd956458/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java
index 99aa23e..de3bcf6 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java
@@ -168,8 +168,7 @@ public class SqlValidatorUtil {
       SqlNodeList extendedColumns) {
     final List list = extendedColumns.getList();
     //noinspection unchecked
-    return Pair.zip(Util.quotientList(list, 2, 0),
-        Util.quotientList(list, 2, 1));
+    return Util.pairs(list);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/bd956458/core/src/main/java/org/apache/calcite/util/Util.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/Util.java b/core/src/main/java/org/apache/calcite/util/Util.java
index 1dcb567..1a5a429 100644
--- a/core/src/main/java/org/apache/calcite/util/Util.java
+++ b/core/src/main/java/org/apache/calcite/util/Util.java
@@ -1848,6 +1848,17 @@ public class Util {
     };
   }
 
+  /** Given a list with N elements
+   * [e<sub>0</sub>, e<sub>1</sub>, ..., e<sub>N-1</sub>]
+   * (where N is even), returns a list of the N / 2 elements
+   * [ (e<sub>0</sub>, e<sub>1</sub>),
+   * (e<sub>2</sub>, e<sub>3</sub>), ... ]. */
+  public static <E> List<Pair<E, E>> pairs(final List<E> list) {
+    //noinspection unchecked
+    return Pair.zip(quotientList(list, 2, 0),
+        quotientList(list, 2, 1));
+  }
+
   /** Returns the first value if it is not null,
    * otherwise the second value.
    *

http://git-wip-us.apache.org/repos/asf/calcite/blob/bd956458/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java b/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
index ad2856a..6d5d1dd 100644
--- a/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
@@ -62,6 +62,7 @@ import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.CoreMatchers.not;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
+import static org.junit.Assume.assumeFalse;
 import static org.junit.Assume.assumeTrue;
 
 /**
@@ -1057,16 +1058,20 @@ public class SqlParserTest {
     final String whereRow2 = "select 1 from t2 where ^(x, y)^ < (a, b)";
     conformance = SqlConformanceEnum.DEFAULT;
     sql(whereRow2).sansCarets().ok(whereExpected);
-    if (this instanceof SqlUnParserTest) {
-      // After this point, SqlUnparserTest has problems.
-      // We generate ROW in a dialect that does not allow ROW in all contexts.
-      // So bail out.
-      return;
-    }
+
+    // After this point, SqlUnparserTest has problems.
+    // We generate ROW in a dialect that does not allow ROW in all contexts.
+    // So bail out.
+    assumeFalse(isUnparserTest());
     conformance = SqlConformanceEnum.SQL_SERVER_2008;
     sql(whereRow2).sansCarets().ok(whereExpected);
   }
 
+  /** Whether this is a sub-class that tests un-parsing as well as parsing. */
+  protected boolean isUnparserTest() {
+    return false;
+  }
+
   @Test public void testPeriod() {
     // We don't have a PERIOD constructor currently;
     // ROW constructor is sufficient for now.

http://git-wip-us.apache.org/repos/asf/calcite/blob/bd956458/core/src/test/java/org/apache/calcite/sql/parser/SqlUnParserTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/parser/SqlUnParserTest.java b/core/src/test/java/org/apache/calcite/sql/parser/SqlUnParserTest.java
index a067e49..1126333 100644
--- a/core/src/test/java/org/apache/calcite/sql/parser/SqlUnParserTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/parser/SqlUnParserTest.java
@@ -31,6 +31,10 @@ public class SqlUnParserTest extends SqlParserTest {
   @Override protected Tester getTester() {
     return new UnparsingTesterImpl();
   }
+
+  @Override protected boolean isUnparserTest() {
+    return true;
+  }
 }
 
 // End SqlUnParserTest.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/bd956458/core/src/test/java/org/apache/calcite/util/UtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/util/UtilTest.java b/core/src/test/java/org/apache/calcite/util/UtilTest.java
index f7a71ca..2063fe2 100644
--- a/core/src/test/java/org/apache/calcite/util/UtilTest.java
+++ b/core/src/test/java/org/apache/calcite/util/UtilTest.java
@@ -969,7 +969,8 @@ public class UtilTest {
   }
 
   /**
-   * Unit test for {@link Util#quotientList(java.util.List, int, int)}.
+   * Unit test for {@link Util#quotientList(java.util.List, int, int)}
+   * and {@link Util#pairs(List)}.
    */
   @Test public void testQuotientList() {
     List<String> beatles = Arrays.asList("john", "paul", "george", "ringo");
@@ -1017,6 +1018,16 @@ public class UtilTest {
 
     final List list5 = Util.quotientList(beatles, 10, 5);
     assertEquals(0, list5.size());
+
+    final List<Pair<String, String>> list6 = Util.pairs(beatles);
+    assertThat(list6.size(), is(2));
+    assertThat(list6.get(0).left, is("john"));
+    assertThat(list6.get(0).right, is("paul"));
+    assertThat(list6.get(1).left, is("george"));
+    assertThat(list6.get(1).right, is("ringo"));
+
+    final List<Pair<String, String>> list7 = Util.pairs(empty);
+    assertThat(list7.size(), is(0));
   }
 
   @Test public void testImmutableIntList() {

http://git-wip-us.apache.org/repos/asf/calcite/blob/bd956458/server/src/main/codegen/config.fmpp
----------------------------------------------------------------------
diff --git a/server/src/main/codegen/config.fmpp b/server/src/main/codegen/config.fmpp
index c9f938f..30c3031 100644
--- a/server/src/main/codegen/config.fmpp
+++ b/server/src/main/codegen/config.fmpp
@@ -25,6 +25,8 @@ data: {
         "org.apache.calcite.sql.SqlCreate"
         "org.apache.calcite.sql.SqlDrop"
         "org.apache.calcite.sql.ddl.SqlDdlNodes"
+        "java.util.Map"
+        "java.util.HashMap"
       ]
 
       # List of keywords.
@@ -33,6 +35,9 @@ data: {
         "MATERIALIZED"
         "STORED"
         "VIRTUAL"
+        "JAR"
+        "FILE"
+        "ARCHIVE"
       ]
 
       # List of keywords from "keywords" section that are not reserved.
@@ -344,6 +349,9 @@ data: {
         "MATERIALIZED"
         "STORED"
         "VIRTUAL"
+        "JAR"
+        "FILE"
+        "ARCHIVE"
       ]
 
       # List of additional join types. Each is a method with no arguments.
@@ -378,6 +386,7 @@ data: {
         "SqlCreateTable"
         "SqlCreateType"
         "SqlCreateView"
+        "SqlCreateFunction"
       ]
 
       # List of methods for parsing extensions to "DROP" calls.
@@ -388,6 +397,7 @@ data: {
         "SqlDropTable"
         "SqlDropType"
         "SqlDropView"
+        "SqlDropFunction"
       ]
 
       # List of files in @includes directory that have parser method

http://git-wip-us.apache.org/repos/asf/calcite/blob/bd956458/server/src/main/codegen/includes/parserImpls.ftl
----------------------------------------------------------------------
diff --git a/server/src/main/codegen/includes/parserImpls.ftl b/server/src/main/codegen/includes/parserImpls.ftl
index 82210a0..51ea3b4 100644
--- a/server/src/main/codegen/includes/parserImpls.ftl
+++ b/server/src/main/codegen/includes/parserImpls.ftl
@@ -303,6 +303,53 @@ SqlCreate SqlCreateMaterializedView(Span s, boolean replace) :
     }
 }
 
+private void FunctionJarDef(SqlNodeList usingList) :
+{
+    final SqlDdlNodes.FileType fileType;
+    final SqlNode uri;
+}
+{
+    (
+        <ARCHIVE> { fileType = SqlDdlNodes.FileType.ARCHIVE; }
+    |
+        <FILE> { fileType = SqlDdlNodes.FileType.FILE; }
+    |
+        <JAR> { fileType = SqlDdlNodes.FileType.JAR; }
+    ) {
+        usingList.add(SqlLiteral.createSymbol(fileType, getPos()));
+    }
+    uri = StringLiteral() {
+        usingList.add(uri);
+    }
+}
+
+SqlCreate SqlCreateFunction(Span s, boolean replace) :
+{
+    final boolean ifNotExists;
+    final SqlIdentifier id;
+    final SqlNode className;
+    SqlNodeList usingList = SqlNodeList.EMPTY;
+}
+{
+    <FUNCTION> ifNotExists = IfNotExistsOpt()
+    id = CompoundIdentifier()
+    <AS>
+    className = StringLiteral()
+    [
+        <USING> {
+            usingList = new SqlNodeList(getPos());
+        }
+        FunctionJarDef(usingList)
+        (
+            <COMMA>
+            FunctionJarDef(usingList)
+        )*
+    ] {
+        return SqlDdlNodes.createFunction(s.end(this), replace, ifNotExists,
+            id, className, usingList);
+    }
+}
+
 SqlDrop SqlDropSchema(Span s, boolean replace) :
 {
     final boolean ifExists;
@@ -364,4 +411,16 @@ SqlDrop SqlDropMaterializedView(Span s, boolean replace) :
     }
 }
 
+SqlDrop SqlDropFunction(Span s, boolean replace) :
+{
+    final boolean ifExists;
+    final SqlIdentifier id;
+}
+{
+    <FUNCTION> ifExists = IfExistsOpt()
+    id = CompoundIdentifier() {
+        return SqlDdlNodes.dropFunction(s.end(this), ifExists, id);
+    }
+}
+
 // End parserImpls.ftl

http://git-wip-us.apache.org/repos/asf/calcite/blob/bd956458/server/src/main/java/org/apache/calcite/sql/ddl/SqlCreateFunction.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/calcite/sql/ddl/SqlCreateFunction.java b/server/src/main/java/org/apache/calcite/sql/ddl/SqlCreateFunction.java
new file mode 100644
index 0000000..29ccc6b
--- /dev/null
+++ b/server/src/main/java/org/apache/calcite/sql/ddl/SqlCreateFunction.java
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.ddl;
+
+import org.apache.calcite.sql.SqlCreate;
+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.SqlSpecialOperator;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
+
+import com.google.common.base.Preconditions;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Objects;
+
+/**
+ * Parse tree for {@code CREATE FUNCTION} statement.
+ */
+public class SqlCreateFunction extends SqlCreate {
+  private final SqlIdentifier name;
+  private final SqlNode className;
+  private final SqlNodeList usingList;
+
+  private static final SqlSpecialOperator OPERATOR =
+      new SqlSpecialOperator("CREATE FUNCTION", SqlKind.CREATE_FUNCTION);
+
+  /** Creates a SqlCreateFunction. */
+  public SqlCreateFunction(SqlParserPos pos, boolean replace,
+      boolean ifNotExists, SqlIdentifier name,
+      SqlNode className, SqlNodeList usingList) {
+    super(OPERATOR, pos, replace, ifNotExists);
+    this.name = Objects.requireNonNull(name);
+    this.className = className;
+    this.usingList = Objects.requireNonNull(usingList);
+    Preconditions.checkArgument(usingList.size() % 2 == 0);
+  }
+
+  @Override public void unparse(SqlWriter writer, int leftPrec,
+      int rightPrec) {
+    writer.keyword(getReplace() ? "CREATE OR REPLACE" : "CREATE");
+    writer.keyword("FUNCTION");
+    if (ifNotExists) {
+      writer.keyword("IF NOT EXISTS");
+    }
+    name.unparse(writer, 0, 0);
+    writer.keyword("AS");
+    className.unparse(writer, 0, 0);
+    if (usingList.size() > 0) {
+      writer.keyword("USING");
+      final SqlWriter.Frame frame =
+          writer.startList(SqlWriter.FrameTypeEnum.SIMPLE);
+      for (Pair<SqlLiteral, SqlLiteral> using : pairs()) {
+        writer.sep(",");
+        using.left.unparse(writer, 0, 0); // FILE, URL or ARCHIVE
+        using.right.unparse(writer, 0, 0); // e.g. 'file:foo/bar.jar'
+      }
+      writer.endList(frame);
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  private List<Pair<SqlLiteral, SqlLiteral>> pairs() {
+    return Util.pairs((List) usingList.getList());
+  }
+
+  @Override public SqlOperator getOperator() {
+    return OPERATOR;
+  }
+
+  @Override public List<SqlNode> getOperandList() {
+    return Arrays.asList(name, className, usingList);
+  }
+}
+
+// End SqlCreateFunction.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/bd956458/server/src/main/java/org/apache/calcite/sql/ddl/SqlDdlNodes.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/calcite/sql/ddl/SqlDdlNodes.java b/server/src/main/java/org/apache/calcite/sql/ddl/SqlDdlNodes.java
index d35e7cc..f370436 100644
--- a/server/src/main/java/org/apache/calcite/sql/ddl/SqlDdlNodes.java
+++ b/server/src/main/java/org/apache/calcite/sql/ddl/SqlDdlNodes.java
@@ -99,6 +99,14 @@ public class SqlDdlNodes {
         columnList, query);
   }
 
+  /** Creates a CREATE FUNCTION. */
+  public static SqlCreateFunction createFunction(
+      SqlParserPos pos, boolean replace, boolean ifNotExists,
+      SqlIdentifier name, SqlNode className, SqlNodeList usingList) {
+    return new SqlCreateFunction(pos, replace, ifNotExists, name,
+        className, usingList);
+  }
+
   /** Creates a DROP [ FOREIGN ] SCHEMA. */
   public static SqlDropSchema dropSchema(SqlParserPos pos, boolean foreign,
       boolean ifExists, SqlIdentifier name) {
@@ -129,6 +137,12 @@ public class SqlDdlNodes {
     return new SqlDropMaterializedView(pos, ifExists, name);
   }
 
+  /** Creates a DROP FUNCTION. */
+  public static SqlDrop dropFunction(SqlParserPos pos,
+      boolean ifExists, SqlIdentifier name) {
+    return new SqlDropFunction(pos, ifExists, name);
+  }
+
   /** Creates a column declaration. */
   public static SqlNode column(SqlParserPos pos, SqlIdentifier name,
       SqlDataTypeSpec dataType, SqlNode expression, ColumnStrategy strategy) {
@@ -236,6 +250,13 @@ public class SqlDdlNodes {
       throw new RuntimeException(e);
     }
   }
+
+  /** File type for CREATE FUNCTION. */
+  public enum FileType {
+    FILE,
+    JAR,
+    ARCHIVE
+  }
 }
 
 // End SqlDdlNodes.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/bd956458/server/src/main/java/org/apache/calcite/sql/ddl/SqlDropFunction.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/calcite/sql/ddl/SqlDropFunction.java b/server/src/main/java/org/apache/calcite/sql/ddl/SqlDropFunction.java
new file mode 100644
index 0000000..bd6c5bd
--- /dev/null
+++ b/server/src/main/java/org/apache/calcite/sql/ddl/SqlDropFunction.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.sql.ddl;
+
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlSpecialOperator;
+import org.apache.calcite.sql.parser.SqlParserPos;
+
+/**
+ * Parse tree for {@code DROP FUNCTION} statement.
+ */
+public class SqlDropFunction extends SqlDropObject {
+  private static final SqlOperator OPERATOR =
+      new SqlSpecialOperator("DROP FUNCTION", SqlKind.DROP_FUNCTION);
+
+  /** Creates a SqlDropFunction. */
+  public SqlDropFunction(SqlParserPos pos, boolean ifExists,
+      SqlIdentifier name) {
+    super(OPERATOR, pos, ifExists, name);
+  }
+}
+
+// End SqlDropFunction.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/bd956458/server/src/main/java/org/apache/calcite/sql/ddl/SqlDropObject.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/calcite/sql/ddl/SqlDropObject.java b/server/src/main/java/org/apache/calcite/sql/ddl/SqlDropObject.java
index dacc093..ed6e6d4 100644
--- a/server/src/main/java/org/apache/calcite/sql/ddl/SqlDropObject.java
+++ b/server/src/main/java/org/apache/calcite/sql/ddl/SqlDropObject.java
@@ -91,6 +91,7 @@ abstract class SqlDropObject extends SqlDrop
             RESOURCE.typeNotFound(name.getSimple()));
       }
       break;
+    case OTHER_DDL:
     default:
       throw new AssertionError(getKind());
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/bd956458/server/src/test/java/org/apache/calcite/test/ServerParserTest.java
----------------------------------------------------------------------
diff --git a/server/src/test/java/org/apache/calcite/test/ServerParserTest.java b/server/src/test/java/org/apache/calcite/test/ServerParserTest.java
index 3bb7c39..d9628d3 100644
--- a/server/src/test/java/org/apache/calcite/test/ServerParserTest.java
+++ b/server/src/test/java/org/apache/calcite/test/ServerParserTest.java
@@ -210,6 +210,29 @@ public class ServerParserTest extends SqlParserTest {
     sql(sql).ok(expected);
   }
 
+  @Test public void testCreateOrReplaceFunction() {
+    final String sql = "create or replace function if not exists x.udf\n"
+        + " as 'org.apache.calcite.udf.TableFun.demoUdf'\n"
+        + "using jar 'file:/path/udf/udf-0.0.1-SNAPSHOT.jar',\n"
+        + " jar 'file:/path/udf/udf2-0.0.1-SNAPSHOT.jar',\n"
+        + " file 'file:/path/udf/logback.xml'";
+    final String expected = "CREATE OR REPLACE FUNCTION"
+        + " IF NOT EXISTS `X`.`UDF`"
+        + " AS 'org.apache.calcite.udf.TableFun.demoUdf'"
+        + " USING JAR 'file:/path/udf/udf-0.0.1-SNAPSHOT.jar',"
+        + " JAR 'file:/path/udf/udf2-0.0.1-SNAPSHOT.jar',"
+        + " FILE 'file:/path/udf/logback.xml'";
+    sql(sql).ok(expected);
+  }
+
+  @Test public void testCreateOrReplaceFunction2() {
+    final String sql = "create function \"my Udf\"\n"
+        + " as 'org.apache.calcite.udf.TableFun.demoUdf'";
+    final String expected = "CREATE FUNCTION `my Udf`"
+        + " AS 'org.apache.calcite.udf.TableFun.demoUdf'";
+    sql(sql).ok(expected);
+  }
+
   @Test public void testDropSchema() {
     sql("drop schema x")
         .ok("DROP SCHEMA `X`");
@@ -270,6 +293,18 @@ public class ServerParserTest extends SqlParserTest {
         .ok("DROP MATERIALIZED VIEW IF EXISTS `X`");
   }
 
+  @Test public void testDropFunction() {
+    final String sql = "drop function x.udf";
+    final String expected = "DROP FUNCTION `X`.`UDF`";
+    sql(sql).ok(expected);
+  }
+
+  @Test public void testDropFunctionIfExists() {
+    final String sql = "drop function if exists \"my udf\"";
+    final String expected = "DROP FUNCTION IF EXISTS `my udf`";
+    sql(sql).ok(expected);
+  }
+
 }
 
 // End ServerParserTest.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/bd956458/server/src/test/java/org/apache/calcite/test/ServerUnParserTest.java
----------------------------------------------------------------------
diff --git a/server/src/test/java/org/apache/calcite/test/ServerUnParserTest.java b/server/src/test/java/org/apache/calcite/test/ServerUnParserTest.java
new file mode 100644
index 0000000..debf5c4
--- /dev/null
+++ b/server/src/test/java/org/apache/calcite/test/ServerUnParserTest.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.test;
+
+/**
+ * Extension to {@link ServerParserTest} that ensures that every expression can
+ * un-parse successfully.
+ */
+public class ServerUnParserTest extends ServerParserTest {
+  //~ Constructors -----------------------------------------------------------
+
+  public ServerUnParserTest() {
+  }
+
+  //~ Methods ----------------------------------------------------------------
+
+  @Override protected Tester getTester() {
+    return new UnparsingTesterImpl();
+  }
+
+  @Override protected boolean isUnparserTest() {
+    return true;
+  }
+}
+
+// End ServerUnParserTest.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/bd956458/site/_docs/reference.md
----------------------------------------------------------------------
diff --git a/site/_docs/reference.md b/site/_docs/reference.md
index 261c1f1..e43e255 100644
--- a/site/_docs/reference.md
+++ b/site/_docs/reference.md
@@ -2196,12 +2196,14 @@ ddlStatement:
   |   createViewStatement
   |   createMaterializedViewStatement
   |   createTypeStatement
+  |   createFunctionStatement
   |   dropSchemaStatement
   |   dropForeignSchemaStatement
   |   dropTableStatement
   |   dropViewStatement
   |   dropMaterializedViewStatement
   |   dropTypeStatement
+  |   dropFunctionStatement
 
 createSchemaStatement:
       CREATE [ OR REPLACE ] SCHEMA [ IF NOT EXISTS ] name
@@ -2267,6 +2269,14 @@ createMaterializedViewStatement:
       [ '(' columnName [, columnName ]* ')' ]
       AS query
 
+createFunctionStatement:
+      CREATE [ OR REPLACE ] FUNCTION [ IF NOT EXISTS ] name
+      AS classNameLiteral
+      [ USING  usingFile [, usingFile ]* ]
+
+usingFile:
+      ( JAR | FILE | ARCHIVE ) filePathLiteral
+
 dropSchemaStatement:
       DROP SCHEMA [ IF EXISTS ] name
 
@@ -2284,6 +2294,9 @@ dropMaterializedViewStatement:
 
 dropTypeStatement:
       DROP TYPE [ IF EXISTS ] name
+
+dropFunctionStatement:
+      DROP FUNCTION [ IF EXISTS ] name
 {% endhighlight %}
 
 In *createTableStatement*, if you specify *AS query*, you may omit the list of
@@ -2292,3 +2305,6 @@ case it just renames the underlying column.
 
 In *columnGenerator*, if you do not specify `VIRTUAL` or `STORED` for a
 generated column, `VIRTUAL` is the default.
+
+In *createFunctionStatement* and *usingFile*, *classNameLiteral*
+and *filePathLiteral* are character literals.