You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by yhuai <gi...@git.apache.org> on 2016/04/01 23:56:33 UTC

[GitHub] spark pull request: [WIP] [SQL] Handle CreateFunction/DropFunction

GitHub user yhuai opened a pull request:

    https://github.com/apache/spark/pull/12117

    [WIP] [SQL] Handle CreateFunction/DropFunction

    ## What changes were proposed in this pull request?
    TODO
    
    ## How was this patch tested?
    TODO
    
    ## TODO
    [ ] A detailed description
    [ ] Self-review
    [ ] Cleanup
    [ ] More tests for create/drop functions (we need to more tests for permanent functions).
    [ ] File JIRAs for all TODOs

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/yhuai/spark function

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/12117.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #12117
    
----
commit 7d001843f6a161298702aaf20079f1611b18141b
Author: Liang-Chi Hsieh <si...@tw.ibm.com>
Date:   2016-03-27T13:15:54Z

    init import.

commit 9af70af9cd2dfdac0fa36b2a1440dc14b88d9ddc
Author: Liang-Chi Hsieh <si...@tw.ibm.com>
Date:   2016-03-29T04:14:40Z

    Merge remote-tracking branch 'upstream/master' into native-ddl-function

commit 35ad7ae14ebff693a8e15dd231eb69be7061402f
Author: Liang-Chi Hsieh <si...@tw.ibm.com>
Date:   2016-03-29T12:10:19Z

    Fix hive temp function feature.

commit cb29f0fb9d66b3bf3774dd9f76a82af365c39deb
Author: Liang-Chi Hsieh <si...@tw.ibm.com>
Date:   2016-03-29T13:08:37Z

    Fix scala style.

commit 77848c910a78165416e7c069b25eaaee1b9c5a93
Author: Liang-Chi Hsieh <si...@tw.ibm.com>
Date:   2016-03-29T14:02:27Z

    Use HiveFunctionRegistry to find Generator function in HiveSqlParser.

commit b8dda845dde32202c60db668270e73a0d3513309
Author: Liang-Chi Hsieh <si...@tw.ibm.com>
Date:   2016-03-29T15:34:50Z

    Fix describe function problem.

commit ee957db463300c0b3f9e5194d9400d598b29509d
Author: Liang-Chi Hsieh <si...@tw.ibm.com>
Date:   2016-03-29T23:10:54Z

    Fix scala style.

commit 133ce1a35d5d7f0f7461be723fbc8bf6db73a5f2
Author: Liang-Chi Hsieh <si...@tw.ibm.com>
Date:   2016-03-30T02:34:56Z

    Fix test.

commit 6b769801108d64711fad899e48d08b53903a5f3f
Author: Liang-Chi Hsieh <si...@tw.ibm.com>
Date:   2016-03-30T02:41:36Z

    Merge remote-tracking branch 'upstream/master' into native-ddl-function
    
    Conflicts:
    	sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala

commit e05b108f9aeedf5d8e3fb5c7145940619811a858
Author: Liang-Chi Hsieh <si...@tw.ibm.com>
Date:   2016-03-30T03:21:25Z

    Address some comments.

commit 314c4db851da5e832fb995f5538e66d4c1ddc89a
Author: Liang-Chi Hsieh <si...@tw.ibm.com>
Date:   2016-03-30T04:00:24Z

    Add resources to CatalogFunction.

commit c370c479b388606226b91e19da791f328014559e
Author: Liang-Chi Hsieh <si...@tw.ibm.com>
Date:   2016-03-30T09:51:47Z

    Let Spark load Hive permanent function and create expression.

commit acf9299a79684dbb063efa035323726c753c682d
Author: Liang-Chi Hsieh <si...@tw.ibm.com>
Date:   2016-03-30T10:05:34Z

    A little refactoring.

commit 2cab41cdc254764ed976842475ce2c2547b0ddcc
Author: Liang-Chi Hsieh <si...@tw.ibm.com>
Date:   2016-03-31T09:32:07Z

    Refactoring.

commit 65d9dbdfbb26c0a1e917767aae08f73df4f9763c
Author: Liang-Chi Hsieh <si...@tw.ibm.com>
Date:   2016-03-31T10:22:31Z

    Fix compilation error.

commit 05709f09cfa5fb171e926c839929135b1cda112e
Author: Liang-Chi Hsieh <si...@tw.ibm.com>
Date:   2016-03-31T23:48:22Z

    Merge remote-tracking branch 'upstream/master' into native-ddl-function
    
    Conflicts:
    	sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala

commit 67df04f70d2f6edb9a670aaa6cf749c8c86f6695
Author: Yin Huai <yh...@databricks.com>
Date:   2016-04-01T02:35:06Z

    Rename alias to className

commit b67c4448b23aff65d6d153c9f4fdcf1bc6a26712
Author: Yin Huai <yh...@databricks.com>
Date:   2016-04-01T03:26:37Z

    Make SessionCatalog handle makeFunctionBuilder and remove HiveFunctionRegistry

commit 1753cac2a09da47fd5ac439656d0f49f8866f841
Author: Yin Huai <yh...@databricks.com>
Date:   2016-04-01T03:27:28Z

    functionClassName => className

commit a2d588ffed2dda19f48270d9b42999ea0585ceb5
Author: Yin Huai <yh...@databricks.com>
Date:   2016-04-01T04:21:59Z

    Put loadFunctionResources to SessionCatalog

commit 51b72dd028a73276ad374b9560223e27c60cbbd5
Author: Yin Huai <yh...@databricks.com>
Date:   2016-04-01T05:25:02Z

    SessionCatalog loads permanent function instead of letting function registry loads permanent function.

commit 66c5261a5638dd05322b5b1a45af9b591a328233
Author: Yin Huai <yh...@databricks.com>
Date:   2016-04-01T17:02:15Z

    Take care UnresolvedGenerator and cleanup.

commit 2aa3725dd42c3e62e7326ba87ea411be0d308569
Author: Yin Huai <yh...@databricks.com>
Date:   2016-04-01T20:25:26Z

    If we cannot find a function in Spark's FunctionRegistry. We will also try Hive's.

commit e34308761cb773b8bf63b75e691a6d39ee797bbe
Author: Yin Huai <yh...@databricks.com>
Date:   2016-04-01T20:26:22Z

    Merge remote-tracking branch 'upstream/master' into function

commit 81543596860c8ec45fc9fd54154501c8690e2cfd
Author: Yin Huai <yh...@databricks.com>
Date:   2016-04-01T21:52:38Z

    Fix tests and pre-load two Hive's builtin functions.

commit e0570cdacda5b3cc81efa3b317e75882186f2d97
Author: Yin Huai <yh...@databricks.com>
Date:   2016-04-01T21:53:15Z

    Merge remote-tracking branch 'upstream/master' into function

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58472800
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -524,7 +537,39 @@ class SessionCatalog(
        * Note: This is currently only used for temporary functions.
        */
       def lookupFunction(name: String, children: Seq[Expression]): Expression = {
    -    functionRegistry.lookupFunction(name, children)
    +    // TODO: Right now, the name can be qualified or not qualified.
    +    // It will be better to get a FunctionIdentifier.
    +    // TODO: Right now, we assume that name is not qualified!
    +    val qualifiedName = FunctionIdentifier(name, Some(currentDb)).unquotedString
    +    if (functionRegistry.functionExists(name)) {
    +      // This function has been already loaded into the function registry.
    +      functionRegistry.lookupFunction(name, children)
    +    } else if (functionRegistry.functionExists(qualifiedName)) {
    +      // This function has been already loaded into the function registry.
    +      // Unlike the above block, we find this function by using the qualified name.
    +      functionRegistry.lookupFunction(qualifiedName, children)
    +    } else {
    +      // The function has not been loaded to the function registry, which means
    +      // that the function is a permanent function (if it actually has been registered
    +      // in the metastore). We need to first put the function in FunctionRegistry.
    +      val catalogFunction = try {
    +        externalCatalog.getFunction(currentDb, name)
    +      } catch {
    +        case e: AnalysisException => failFunctionLookup(name)
    +        case e: NoSuchFunctionException => failFunctionLookup(name)
    +      }
    +      loadFunctionResources(catalogFunction.resources)
    +      // Please note that qualifiedName is provided by the user. However,
    +      // catalogFunction.identifier.unquotedString is returned by the underlying
    +      // catalog. So, it is possible that qualifiedName is not exactly the same as
    +      // catalogFunction.identifier.unquotedString (difference is on case-sensitivity).
    +      // At here, we preserve the input from the user.
    +      val info = new ExpressionInfo(catalogFunction.className, qualifiedName)
    +      val builder = makeFunctionBuilder(qualifiedName, catalogFunction.className)
    +      createTempFunction(qualifiedName, info, builder, ignoreIfExists = false)
    --- End diff --
    
    Updated the doc.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SQL] Handle CreateFunction/DropFunction

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-204585532
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58602203
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala ---
    @@ -85,24 +91,23 @@ case class DropFunction(
       extends RunnableCommand {
     
       override def run(sqlContext: SQLContext): Seq[Row] = {
    +    val catalog = sqlContext.sessionState.catalog
         if (isTemp) {
           if (databaseName.isDefined) {
             throw new AnalysisException(
               s"It is not allowed to provide database name when dropping a temporary function. " +
                 s"However, database name ${databaseName.get} is provided.")
           }
    -      sqlContext.sessionState.catalog.dropTempFunction(functionName, ifExists)
    +      catalog.dropTempFunction(functionName, ifExists)
         } else {
    -      // We are dropping a permanent.
    -      val dbName = databaseName.getOrElse(sqlContext.sessionState.catalog.getCurrentDatabase)
    +      // We are dropping a permanent function.
    +      val dbName = databaseName.getOrElse(catalog.getCurrentDatabase)
           val func = FunctionIdentifier(functionName, Some(dbName))
    -      if (!ifExists) {
    -        if (!sqlContext.sessionState.catalog.functionExists(func)) {
    -          throw new AnalysisException(
    -            s"Function '$functionName' does not exist in database '$dbName'.")
    -        }
    +      if (!ifExists && !catalog.functionExists(func)) {
    +        throw new AnalysisException(
    +          s"Function '$functionName' does not exist in database '$dbName'.")
    --- End diff --
    
    https://issues.apache.org/jira/browse/SPARK-14410


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58472254
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -428,53 +432,85 @@ class SessionCatalog(
        */
       def dropFunction(name: FunctionIdentifier): Unit = {
         val db = name.database.getOrElse(currentDb)
    +    val qualified = name.copy(database = Some(db)).unquotedString
    +    if (functionRegistry.functionExists(qualified)) {
    +      // If we have loaded this function into FunctionRegistry,
    +      // also drop it from there.
    +      functionRegistry.dropFunction(qualified)
    --- End diff --
    
    Done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58474075
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala ---
    @@ -0,0 +1,109 @@
    +/*
    + * 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.spark.sql.execution.command
    +
    +import org.apache.spark.sql.{AnalysisException, Row, SQLContext}
    +import org.apache.spark.sql.catalyst.FunctionIdentifier
    +import org.apache.spark.sql.catalyst.catalog.CatalogFunction
    +import org.apache.spark.sql.catalyst.expressions.ExpressionInfo
    +
    +
    +/**
    + * The DDL command that creates a function.
    + * alias: the class name that implements the created function.
    + * resources: Jars, files, or archives which need to be added to the environment when the function
    + *            is referenced for the first time by a session.
    + * isTemp: indicates if it is a temporary function.
    + */
    +// TODO: Use Seq[FunctionResource] instead of Seq[(String, String)] for resources.
    +case class CreateFunction(
    +    databaseName: Option[String],
    +    functionName: String,
    +    className: String,
    +    resources: Seq[(String, String)],
    +    isTemp: Boolean)
    +  extends RunnableCommand {
    +
    +  override def run(sqlContext: SQLContext): Seq[Row] = {
    +    if (isTemp) {
    +      if (databaseName.isDefined) {
    +        throw new AnalysisException(
    +          s"It is not allowed to provide database name when defining a temporary function. " +
    +            s"However, database name ${databaseName.get} is provided.")
    +      }
    +      // We first load resources and then put the builder in the function registry.
    +      // Please note that it is allowed to overwrite an existing temp function.
    +      sqlContext.sessionState.catalog.loadFunctionResources(resources)
    +      val info = new ExpressionInfo(className, functionName)
    +      val builder =
    +        sqlContext.sessionState.catalog.makeFunctionBuilder(functionName, className)
    +      sqlContext.sessionState.catalog.createTempFunction(
    +        functionName, info, builder, ignoreIfExists = false)
    +    } else {
    +      val dbName = databaseName.getOrElse(sqlContext.sessionState.catalog.getCurrentDatabase)
    +      val func = FunctionIdentifier(functionName, Some(dbName))
    +      val catalogFunc = CatalogFunction(func, className, resources)
    +      // We are creating a permanent function. First, we want to check if this function
    +      // has already been created.
    +      // Check if the function to create is already existing. If so, throw exception.
    --- End diff --
    
    Deleted


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SQL] Handle CreateFunction/Drop...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-205150803
  
    **[Test build #54830 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/54830/consoleFull)** for PR 12117 at commit [`0572acc`](https://github.com/apache/spark/commit/0572acce35f93961ac5f56bd215c736da00d4aea).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SQL] Handle CreateFunction/DropFunction

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58282759
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -453,28 +464,80 @@ class SessionCatalog(
        * If a database is specified in `name`, this will return the function in that database.
        * If no database is specified, this will return the function in the current database.
        */
    +  // TODO: have a better name. This method is actually for fetching the metadata of a function.
    --- End diff --
    
    if we change this (which we should do separately) then we should rename `getTable` too.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/spark/pull/12117


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SQL] Handle CreateFunction/DropFunction

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-204608452
  
    **[Test build #54726 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/54726/consoleFull)** for PR 12117 at commit [`04a5926`](https://github.com/apache/spark/commit/04a59263664df573565e287381c8fd1dcad59197).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58456814
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala ---
    @@ -208,6 +208,22 @@ class SQLContext private[sql](
         sparkContext.addJar(path)
       }
     
    +  /** Returns a [[FunctionResourceLoader]] that can be used in SessionCatalog. */
    +  protected[sql] def functionResourceLoader: FunctionResourceLoader = {
    --- End diff --
    
    This needs to be a val! Otherwise you great a new one every time you call it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58600078
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala ---
    @@ -85,24 +91,23 @@ case class DropFunction(
       extends RunnableCommand {
     
       override def run(sqlContext: SQLContext): Seq[Row] = {
    +    val catalog = sqlContext.sessionState.catalog
         if (isTemp) {
           if (databaseName.isDefined) {
             throw new AnalysisException(
               s"It is not allowed to provide database name when dropping a temporary function. " +
                 s"However, database name ${databaseName.get} is provided.")
           }
    -      sqlContext.sessionState.catalog.dropTempFunction(functionName, ifExists)
    +      catalog.dropTempFunction(functionName, ifExists)
         } else {
    -      // We are dropping a permanent.
    -      val dbName = databaseName.getOrElse(sqlContext.sessionState.catalog.getCurrentDatabase)
    +      // We are dropping a permanent function.
    +      val dbName = databaseName.getOrElse(catalog.getCurrentDatabase)
           val func = FunctionIdentifier(functionName, Some(dbName))
    -      if (!ifExists) {
    -        if (!sqlContext.sessionState.catalog.functionExists(func)) {
    -          throw new AnalysisException(
    -            s"Function '$functionName' does not exist in database '$dbName'.")
    -        }
    +      if (!ifExists && !catalog.functionExists(func)) {
    +        throw new AnalysisException(
    +          s"Function '$functionName' does not exist in database '$dbName'.")
    --- End diff --
    
    I still think this should be handled within `dropFunction` itself (by passing the `ifExists` flag), but not a big deal


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [WIP] [SQL] Handle CreateFunctio...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58301771
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala ---
    @@ -112,4 +124,121 @@ class HiveSessionCatalog(
         metastoreCatalog.cachedDataSourceTables.getIfPresent(key)
       }
     
    +  override def makeFunctionBuilder(funcName: String, className: String): FunctionBuilder = {
    +    makeFunctionBuilder(funcName, Utils.classForName(className))
    +  }
    +
    +  /**
    +   * Construct a [[FunctionBuilder]] based on the provided class that represents a function.
    +   */
    +  private def makeFunctionBuilder(name: String, clazz: Class[_]): FunctionBuilder = {
    +    // When we instantiate hive UDF wrapper class, we may throw exception if the input
    +    // expressions don't satisfy the hive UDF, such as type mismatch, input number
    +    // mismatch, etc. Here we catch the exception and throw AnalysisException instead.
    +    (children: Seq[Expression]) => {
    +      try {
    +        if (classOf[UDF].isAssignableFrom(clazz)) {
    +          val udf = HiveSimpleUDF(name, new HiveFunctionWrapper(clazz.getName), children)
    +          udf.dataType // Force it to check input data types.
    +          udf
    +        } else if (classOf[GenericUDF].isAssignableFrom(clazz)) {
    +          val udf = HiveGenericUDF(name, new HiveFunctionWrapper(clazz.getName), children)
    +          udf.dataType // Force it to check input data types.
    +          udf
    +        } else if (classOf[AbstractGenericUDAFResolver].isAssignableFrom(clazz)) {
    +          val udaf = HiveUDAFFunction(name, new HiveFunctionWrapper(clazz.getName), children)
    +          udaf.dataType // Force it to check input data types.
    +          udaf
    +        } else if (classOf[UDAF].isAssignableFrom(clazz)) {
    +          val udaf = HiveUDAFFunction(
    +            name,
    +            new HiveFunctionWrapper(clazz.getName),
    +            children,
    +            isUDAFBridgeRequired = true)
    +          udaf.dataType  // Force it to check input data types.
    +          udaf
    +        } else if (classOf[GenericUDTF].isAssignableFrom(clazz)) {
    +          val udtf = HiveGenericUDTF(name, new HiveFunctionWrapper(clazz.getName), children)
    +          udtf.elementTypes // Force it to check input data types.
    +          udtf
    +        } else {
    +          throw new AnalysisException(s"No handler for Hive UDF '${clazz.getCanonicalName}'")
    +        }
    +      } catch {
    +        case ae: AnalysisException =>
    +          throw ae
    +        case NonFatal(e) =>
    +          val analysisException =
    +            new AnalysisException(s"No handler for Hive UDF '${clazz.getCanonicalName}': $e")
    +          analysisException.setStackTrace(e.getStackTrace)
    +          throw analysisException
    +      }
    +    }
    +  }
    +
    +  // We have a list of Hive built-in functions that we do not support. So, we will check
    +  // Hive's function registry and lazily load needed functions into our own function registry.
    +  // Those Hive built-in functions are
    +  // assert_true, collect_list, collect_set, compute_stats, context_ngrams, create_union,
    +  // current_user ,elt, ewah_bitmap, ewah_bitmap_and, ewah_bitmap_empty, ewah_bitmap_or, field,
    +  // histogram_numeric, in_file, index, inline, java_method, map_keys, map_values,
    +  // matchpath, ngrams, noop, noopstreaming, noopwithmap, noopwithmapstreaming,
    +  // parse_url, parse_url_tuple, percentile, percentile_approx, posexplode, reflect, reflect2,
    +  // regexp, sentences, stack, std, str_to_map, windowingtablefunction, xpath, xpath_boolean,
    +  // xpath_double, xpath_float, xpath_int, xpath_long, xpath_number,
    +  // xpath_short, and xpath_string.
    +  override def lookupFunction(name: String, children: Seq[Expression]): Expression = {
    +    Try(super.lookupFunction(name, children)) match {
    +      case Success(expr) => expr
    +      case Failure(error) =>
    +        if (functionRegistry.functionExists(name)) {
    +          // If the function actually exists in functionRegistry, it means that there is an
    +          // error when we create the Expression using the given children.
    +          // We need to throw the original exception.
    +          throw error
    --- End diff --
    
    When there is a builder in the function registry, `super.lookupFunction(name, children)` can still fail (for example, when we try to create an expression for a hive builtin function but arguments are not valid).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [WIP] [SQL] Handle CreateFunctio...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58317039
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/functionResources.scala ---
    @@ -0,0 +1,56 @@
    +/*
    + * 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.spark.sql.catalyst.catalog
    +
    +import org.apache.spark.sql.AnalysisException
    +
    +/** An trait that represents the type of a resourced needed by a function. */
    +sealed trait FunctionResourceType
    +
    +object JarResource extends FunctionResourceType
    +
    +object FileResource extends FunctionResourceType
    +
    +// We do nto allow users to specify a archive because it is YARN specific.
    +// When loading resources, we will throw an exception and ask users to
    +// use --archive with spark submit.
    +object ArchiveResource extends FunctionResourceType
    +
    +object FunctionResourceType {
    +  def fromString(resourceType: String): FunctionResourceType = {
    +    resourceType.toLowerCase match {
    +      case "jar" => JarResource
    +      case "file" => FileResource
    +      case "archive" => ArchiveResource
    +      case other =>
    +        throw new AnalysisException(s"Resource Type '$resourceType' is not supported.")
    +    }
    +  }
    +}
    +
    +case class FunctionResource(resourceType: FunctionResourceType, uri: String)
    +
    +trait FunctionResourceLoader {
    +  def loadResource(resource: FunctionResource): Unit
    +}
    +
    +class DummyFunctionResourceLoader extends FunctionResourceLoader {
    --- End diff --
    
    object?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58471886
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala ---
    @@ -49,7 +49,6 @@ object SimpleAnalyzer
     class SimpleAnalyzer(functionRegistry: FunctionRegistry, conf: CatalystConf)
       extends Analyzer(
         new SessionCatalog(new InMemoryCatalog, functionRegistry, conf),
    -    functionRegistry,
         conf)
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SQL] Handle CreateFunction/DropFunction

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-204585522
  
    **[Test build #54724 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/54724/consoleFull)** for PR 12117 at commit [`0a1866c`](https://github.com/apache/spark/commit/0a1866c95bbe4ef260206360f18d521a99f5f482).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58466640
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala ---
    @@ -112,4 +124,129 @@ class HiveSessionCatalog(
         metastoreCatalog.cachedDataSourceTables.getIfPresent(key)
       }
     
    +  override def makeFunctionBuilder(funcName: String, className: String): FunctionBuilder = {
    +    makeFunctionBuilder(funcName, Utils.classForName(className))
    +  }
    +
    +  /**
    +   * Construct a [[FunctionBuilder]] based on the provided class that represents a function.
    +   */
    +  private def makeFunctionBuilder(name: String, clazz: Class[_]): FunctionBuilder = {
    +    // When we instantiate hive UDF wrapper class, we may throw exception if the input
    +    // expressions don't satisfy the hive UDF, such as type mismatch, input number
    +    // mismatch, etc. Here we catch the exception and throw AnalysisException instead.
    +    (children: Seq[Expression]) => {
    +      try {
    +        if (classOf[UDF].isAssignableFrom(clazz)) {
    +          val udf = HiveSimpleUDF(name, new HiveFunctionWrapper(clazz.getName), children)
    +          udf.dataType // Force it to check input data types.
    +          udf
    +        } else if (classOf[GenericUDF].isAssignableFrom(clazz)) {
    +          val udf = HiveGenericUDF(name, new HiveFunctionWrapper(clazz.getName), children)
    +          udf.dataType // Force it to check input data types.
    +          udf
    +        } else if (classOf[AbstractGenericUDAFResolver].isAssignableFrom(clazz)) {
    +          val udaf = HiveUDAFFunction(name, new HiveFunctionWrapper(clazz.getName), children)
    +          udaf.dataType // Force it to check input data types.
    +          udaf
    +        } else if (classOf[UDAF].isAssignableFrom(clazz)) {
    +          val udaf = HiveUDAFFunction(
    +            name,
    +            new HiveFunctionWrapper(clazz.getName),
    +            children,
    +            isUDAFBridgeRequired = true)
    +          udaf.dataType  // Force it to check input data types.
    +          udaf
    +        } else if (classOf[GenericUDTF].isAssignableFrom(clazz)) {
    +          val udtf = HiveGenericUDTF(name, new HiveFunctionWrapper(clazz.getName), children)
    +          udtf.elementTypes // Force it to check input data types.
    +          udtf
    +        } else {
    +          throw new AnalysisException(s"No handler for Hive UDF '${clazz.getCanonicalName}'")
    +        }
    +      } catch {
    +        case ae: AnalysisException =>
    +          throw ae
    +        case NonFatal(e) =>
    +          val analysisException =
    +            new AnalysisException(s"No handler for Hive UDF '${clazz.getCanonicalName}': $e")
    +          analysisException.setStackTrace(e.getStackTrace)
    +          throw analysisException
    +      }
    +    }
    +  }
    +
    +  // We have a list of Hive built-in functions that we do not support. So, we will check
    +  // Hive's function registry and lazily load needed functions into our own function registry.
    +  // Those Hive built-in functions are
    +  // assert_true, collect_list, collect_set, compute_stats, context_ngrams, create_union,
    +  // current_user ,elt, ewah_bitmap, ewah_bitmap_and, ewah_bitmap_empty, ewah_bitmap_or, field,
    +  // histogram_numeric, in_file, index, inline, java_method, map_keys, map_values,
    +  // matchpath, ngrams, noop, noopstreaming, noopwithmap, noopwithmapstreaming,
    +  // parse_url, parse_url_tuple, percentile, percentile_approx, posexplode, reflect, reflect2,
    +  // regexp, sentences, stack, std, str_to_map, windowingtablefunction, xpath, xpath_boolean,
    +  // xpath_double, xpath_float, xpath_int, xpath_long, xpath_number,
    +  // xpath_short, and xpath_string.
    +  override def lookupFunction(name: String, children: Seq[Expression]): Expression = {
    +    // TODO: Once lookupFunction accepts a TableIdentifier, we should refactor this method to
    --- End diff --
    
    FunctionIdentifier* :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SQL] Handle CreateFunction/Drop...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58413245
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala ---
    @@ -175,13 +175,54 @@ case class DescribeDatabase(
       }
     }
     
    +/**
    + * The DDL command that creates a function.
    + * alias: the class name that implements the created function.
    + * resources: Jars, files, or archives which need to be added to the environment when the function
    + *            is referenced for the first time by a session.
    + * isTemp: indicates if it is a temporary function.
    + */
    +// TODO: Use Seq[FunctionResource] instead of Seq[(String, String)] for resources.
     case class CreateFunction(
    --- End diff --
    
    +1


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SQL] Handle CreateFunction/DropFunction

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-204584182
  
    **[Test build #54723 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/54723/consoleFull)** for PR 12117 at commit [`e0570cd`](https://github.com/apache/spark/commit/e0570cdacda5b3cc81efa3b317e75882186f2d97).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58472808
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -534,17 +579,11 @@ class SessionCatalog(
         val dbFunctions =
           externalCatalog.listFunctions(db, pattern).map { f => FunctionIdentifier(f, Some(db)) }
         val regex = pattern.replaceAll("\\*", ".*").r
    --- End diff --
    
    Will keep this part as is.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58473408
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala ---
    @@ -396,18 +400,16 @@ case class ShowFunctions(db: Option[String], pattern: Option[String]) extends Ru
         schema.toAttributes
       }
     
    -  override def run(sqlContext: SQLContext): Seq[Row] = pattern match {
    -    case Some(p) =>
    -      try {
    -        val regex = java.util.regex.Pattern.compile(p)
    -        sqlContext.sessionState.functionRegistry.listFunction()
    -          .filter(regex.matcher(_).matches()).map(Row(_))
    -      } catch {
    -        // probably will failed in the regex that user provided, then returns empty row.
    -        case _: Throwable => Seq.empty[Row]
    -      }
    -    case None =>
    -      sqlContext.sessionState.functionRegistry.listFunction().map(Row(_))
    +  override def run(sqlContext: SQLContext): Seq[Row] = {
    +    val dbName = db.getOrElse(sqlContext.sessionState.catalog.getCurrentDatabase)
    +    val functionNames =
    +      sqlContext.sessionState.catalog
    +        .listFunctions(dbName, pattern.getOrElse(".*"))
    +        .map(_.unquotedString)
    +    // We use distinct at here because SessionCatalog's listFunctions will return
    +    // duplicated entries for UDFs that have already been loaded into the
    --- End diff --
    
    Good!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [WIP] [SQL] Handle CreateFunctio...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-205057831
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58467158
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala ---
    @@ -0,0 +1,109 @@
    +/*
    + * 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.spark.sql.execution.command
    +
    +import org.apache.spark.sql.{AnalysisException, Row, SQLContext}
    +import org.apache.spark.sql.catalyst.FunctionIdentifier
    +import org.apache.spark.sql.catalyst.catalog.CatalogFunction
    +import org.apache.spark.sql.catalyst.expressions.ExpressionInfo
    +
    +
    +/**
    + * The DDL command that creates a function.
    + * alias: the class name that implements the created function.
    + * resources: Jars, files, or archives which need to be added to the environment when the function
    + *            is referenced for the first time by a session.
    + * isTemp: indicates if it is a temporary function.
    + */
    +// TODO: Use Seq[FunctionResource] instead of Seq[(String, String)] for resources.
    +case class CreateFunction(
    +    databaseName: Option[String],
    +    functionName: String,
    +    className: String,
    +    resources: Seq[(String, String)],
    +    isTemp: Boolean)
    +  extends RunnableCommand {
    +
    +  override def run(sqlContext: SQLContext): Seq[Row] = {
    +    if (isTemp) {
    +      if (databaseName.isDefined) {
    +        throw new AnalysisException(
    +          s"It is not allowed to provide database name when defining a temporary function. " +
    +            s"However, database name ${databaseName.get} is provided.")
    +      }
    +      // We first load resources and then put the builder in the function registry.
    +      // Please note that it is allowed to overwrite an existing temp function.
    +      sqlContext.sessionState.catalog.loadFunctionResources(resources)
    --- End diff --
    
    can you add a comment in the `else` case to explain the divergence in behavior?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [WIP] [SQL] Handle CreateFunctio...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-205070488
  
    **[Test build #54813 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/54813/consoleFull)** for PR 12117 at commit [`776c09a`](https://github.com/apache/spark/commit/776c09afd8f448c78b063de510b14434764d72cd).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SQL] Handle CreateFunction/Drop...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58454715
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -428,53 +432,85 @@ class SessionCatalog(
        */
       def dropFunction(name: FunctionIdentifier): Unit = {
         val db = name.database.getOrElse(currentDb)
    +    val qualified = name.copy(database = Some(db)).unquotedString
    +    if (functionRegistry.functionExists(qualified)) {
    +      // If we have loaded this function into FunctionRegistry,
    +      // also drop it from there.
    +      functionRegistry.dropFunction(qualified)
    --- End diff --
    
    I don't understand why we have to do this. Isn't the function registry only for temporary functions? We only put things in it in `createTempFunction`, so we just only remove things from it in `dropTempFunction`. This method here is for persistent functions only.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SQL] Handle CreateFunction/Drop...

Posted by bomeng <gi...@git.apache.org>.
Github user bomeng commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58454116
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -534,17 +579,11 @@ class SessionCatalog(
         val dbFunctions =
           externalCatalog.listFunctions(db, pattern).map { f => FunctionIdentifier(f, Some(db)) }
         val regex = pattern.replaceAll("\\*", ".*").r
    --- End diff --
    
    I may create another PR for put all the pattern stuff in one place. Right now, it was scattered all over with replaceAll(). Please keep your code as it is for now, I will do it later. Thanks.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [WIP] [SQL] Handle CreateFunctio...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58315200
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala ---
    @@ -133,6 +133,33 @@ object UnresolvedAttribute {
       }
     }
     
    +/**
    + * Represents an unresolved generator, which will be created by the parser for
    + * the [[org.apache.spark.sql.catalyst.plans.logical.Generate]] operator.
    + * The analyzer will resolve this generator.
    + */
    +case class UnresolvedGenerator(name: String, children: Seq[Expression]) extends Generator {
    --- End diff --
    
    Yay!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58475394
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveSqlParser.scala ---
    @@ -21,19 +21,19 @@ import scala.collection.JavaConverters._
     import org.antlr.v4.runtime.{ParserRuleContext, Token}
     import org.apache.hadoop.hive.conf.HiveConf
     import org.apache.hadoop.hive.conf.HiveConf.ConfVars
    -import org.apache.hadoop.hive.ql.exec.FunctionRegistry
     import org.apache.hadoop.hive.ql.parse.EximUtil
     import org.apache.hadoop.hive.ql.session.SessionState
     import org.apache.hadoop.hive.serde.serdeConstants
     import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe
     
    -import org.apache.spark.sql.catalyst.catalog.{CatalogColumn, CatalogStorageFormat, CatalogTable, CatalogTableType}
    +import org.apache.spark.sql.catalyst.analysis.UnresolvedGenerator
    +import org.apache.spark.sql.catalyst.catalog._
     import org.apache.spark.sql.catalyst.expressions._
     import org.apache.spark.sql.catalyst.parser._
     import org.apache.spark.sql.catalyst.parser.SqlBaseParser._
     import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
     import org.apache.spark.sql.execution.SparkSqlAstBuilder
    -import org.apache.spark.sql.hive.{CreateTableAsSelect => CTAS, CreateViewAsSelect => CreateView}
    +import org.apache.spark.sql.hive.{CreateTableAsSelect => CTAS, CreateViewAsSelect => CreateView, HiveSessionState}
    --- End diff --
    
    Removed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-205652627
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/54948/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SQL] Handle CreateFunction/DropFunction

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-204585536
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/54724/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SQL] Handle CreateFunction/DropFunction

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58281932
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -453,28 +464,80 @@ class SessionCatalog(
        * If a database is specified in `name`, this will return the function in that database.
        * If no database is specified, this will return the function in the current database.
        */
    +  // TODO: have a better name. This method is actually for fetching the metadata of a function.
       def getFunction(name: FunctionIdentifier): CatalogFunction = {
         val db = name.database.getOrElse(currentDb)
         externalCatalog.getFunction(db, name.funcName)
       }
     
    +  /**
    +   * Check if a function is already existing.
    +   *
    +   */
    +  def functionExists(name: FunctionIdentifier): Boolean = {
    +    if (functionRegistry.functionExists(name.unquotedString)) {
    +      // This function exists in the FunctionRegistry.
    +      true
    +    } else {
    +      // Need to check if this function exists in the metastore.
    +      try {
    +        getFunction(name) != null
    +      } catch {
    +        case _: NoSuchFunctionException => false
    +        case _: AnalysisException => false // HiveExternalCatalog wraps all exceptions with it.
    +      }
    +    }
    +  }
     
       // ----------------------------------------------------------------
       // | Methods that interact with temporary and metastore functions |
       // ----------------------------------------------------------------
     
    +
    +  /**
    +   * Return a temporary function. For testing only.
    +   */
    +  private[catalog] def getTempFunction(name: String): Option[FunctionBuilder] = {
    +    // TODO: Why do we need this?
    --- End diff --
    
    we probably don't


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58466595
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala ---
    @@ -17,27 +17,39 @@
     
     package org.apache.spark.sql.hive
     
    +import scala.util.{Failure, Success, Try}
    +import scala.util.control.NonFatal
    +
     import org.apache.hadoop.fs.Path
     import org.apache.hadoop.hive.conf.HiveConf
    +import org.apache.hadoop.hive.ql.exec.{UDAF, UDF}
    +import org.apache.hadoop.hive.ql.exec.{FunctionRegistry => HiveFunctionRegistry}
    +import org.apache.hadoop.hive.ql.udf.generic.{AbstractGenericUDAFResolver, GenericUDF, GenericUDTF}
     
     import org.apache.spark.sql.catalyst.TableIdentifier
     import org.apache.spark.sql.catalyst.analysis.FunctionRegistry
    -import org.apache.spark.sql.catalyst.catalog.SessionCatalog
    +import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder
    +import org.apache.spark.sql.catalyst.catalog.{FunctionResourceLoader, SessionCatalog}
    +import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionInfo}
     import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
     import org.apache.spark.sql.catalyst.rules.Rule
    +import org.apache.spark.sql.AnalysisException
     import org.apache.spark.sql.execution.datasources.BucketSpec
    +import org.apache.spark.sql.hive.HiveShim.HiveFunctionWrapper
     import org.apache.spark.sql.hive.client.HiveClient
     import org.apache.spark.sql.internal.SQLConf
     import org.apache.spark.sql.types.StructType
    +import org.apache.spark.util.Utils
     
     
    -class HiveSessionCatalog(
    +private[sql] class HiveSessionCatalog(
    --- End diff --
    
    oops, good change!!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-205600013
  
    **[Test build #54931 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/54931/consoleFull)** for PR 12117 at commit [`88fd93c`](https://github.com/apache/spark/commit/88fd93c54cf8b6e70edab25526be93cd91e2563a).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [WIP] [SQL] Handle CreateFunctio...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58317032
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/functionResources.scala ---
    @@ -0,0 +1,56 @@
    +/*
    + * 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.spark.sql.catalyst.catalog
    +
    +import org.apache.spark.sql.AnalysisException
    +
    +/** An trait that represents the type of a resourced needed by a function. */
    +sealed trait FunctionResourceType
    +
    +object JarResource extends FunctionResourceType
    +
    +object FileResource extends FunctionResourceType
    +
    +// We do nto allow users to specify a archive because it is YARN specific.
    +// When loading resources, we will throw an exception and ask users to
    +// use --archive with spark submit.
    +object ArchiveResource extends FunctionResourceType
    +
    +object FunctionResourceType {
    +  def fromString(resourceType: String): FunctionResourceType = {
    --- End diff --
    
    Couple of questions.
    
    1. Is `FunctionResourceType` not a broader concept, i.e. `ResourceType`? Are there any other places we should use this (not to be done in this PR!).
    2. Shouldn't the resource specified for function by pairs of `FunctionResourceType` and `String`?
    3. The `SparkSqlParser` currently does something similar, see: https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala#L303-L309 We could integrate this.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58466523
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala ---
    @@ -0,0 +1,109 @@
    +/*
    + * 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.spark.sql.execution.command
    +
    +import org.apache.spark.sql.{AnalysisException, Row, SQLContext}
    +import org.apache.spark.sql.catalyst.FunctionIdentifier
    +import org.apache.spark.sql.catalyst.catalog.CatalogFunction
    +import org.apache.spark.sql.catalyst.expressions.ExpressionInfo
    +
    +
    +/**
    + * The DDL command that creates a function.
    + * alias: the class name that implements the created function.
    + * resources: Jars, files, or archives which need to be added to the environment when the function
    + *            is referenced for the first time by a session.
    + * isTemp: indicates if it is a temporary function.
    + */
    +// TODO: Use Seq[FunctionResource] instead of Seq[(String, String)] for resources.
    +case class CreateFunction(
    +    databaseName: Option[String],
    +    functionName: String,
    +    className: String,
    +    resources: Seq[(String, String)],
    +    isTemp: Boolean)
    +  extends RunnableCommand {
    +
    +  override def run(sqlContext: SQLContext): Seq[Row] = {
    +    if (isTemp) {
    +      if (databaseName.isDefined) {
    +        throw new AnalysisException(
    +          s"It is not allowed to provide database name when defining a temporary function. " +
    +            s"However, database name ${databaseName.get} is provided.")
    +      }
    +      // We first load resources and then put the builder in the function registry.
    +      // Please note that it is allowed to overwrite an existing temp function.
    +      sqlContext.sessionState.catalog.loadFunctionResources(resources)
    +      val info = new ExpressionInfo(className, functionName)
    +      val builder =
    +        sqlContext.sessionState.catalog.makeFunctionBuilder(functionName, className)
    +      sqlContext.sessionState.catalog.createTempFunction(
    +        functionName, info, builder, ignoreIfExists = false)
    +    } else {
    +      val dbName = databaseName.getOrElse(sqlContext.sessionState.catalog.getCurrentDatabase)
    +      val func = FunctionIdentifier(functionName, Some(dbName))
    +      val catalogFunc = CatalogFunction(func, className, resources)
    +      // We are creating a permanent function. First, we want to check if this function
    +      // has already been created.
    +      // Check if the function to create is already existing. If so, throw exception.
    +      if (sqlContext.sessionState.catalog.functionExists(func)) {
    +        throw new AnalysisException(
    +          s"Function '$functionName' already exists in database '$dbName'.")
    +      }
    +      // This function will be loaded into the FunctionRegistry when a query uses it.
    +      sqlContext.sessionState.catalog.createFunction(catalogFunc)
    +    }
    +    Seq.empty[Row]
    +  }
    +}
    +
    +/**
    + * The DDL command that drops a function.
    + * ifExists: returns an error if the function doesn't exist, unless this is true.
    + * isTemp: indicates if it is a temporary function.
    + */
    +case class DropFunction(
    +    databaseName: Option[String],
    +    functionName: String,
    +    ifExists: Boolean,
    +    isTemp: Boolean)
    +  extends RunnableCommand {
    +
    +  override def run(sqlContext: SQLContext): Seq[Row] = {
    +    if (isTemp) {
    +      if (databaseName.isDefined) {
    +        throw new AnalysisException(
    +          s"It is not allowed to provide database name when dropping a temporary function. " +
    +            s"However, database name ${databaseName.get} is provided.")
    +      }
    +      sqlContext.sessionState.catalog.dropTempFunction(functionName, ifExists)
    +    } else {
    +      // We are dropping a permanent.
    +      val dbName = databaseName.getOrElse(sqlContext.sessionState.catalog.getCurrentDatabase)
    +      val func = FunctionIdentifier(functionName, Some(dbName))
    +      if (!ifExists) {
    +        if (!sqlContext.sessionState.catalog.functionExists(func)) {
    --- End diff --
    
    we duplicate `sqlContext.sessionState.catalog` in many lines. Just assign it to `val catalog` at the top.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [WIP] [SQL] Handle CreateFunctio...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58301096
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala ---
    @@ -67,9 +70,14 @@ class SimpleFunctionRegistry extends FunctionRegistry {
       }
     
       override def lookupFunction(name: String, children: Seq[Expression]): Expression = {
    +    val builder = functionBuilders.get(name)
    +    if (builder.isEmpty) {
    +      throw new AnalysisException(s"undefined function $name")
    +    }
         val func = synchronized {
    -      functionBuilders.get(name).map(_._2).getOrElse {
    -        throw new AnalysisException(s"undefined function $name")
    +      Try(builder.map(_._2)) match {
    --- End diff --
    
    Will revert them.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [WIP] [SQL] Handle CreateFunctio...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-205057830
  
    **[Test build #54810 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/54810/consoleFull)** for PR 12117 at commit [`ae359fb`](https://github.com/apache/spark/commit/ae359fb60dcbb34514487553ac3c7fb24de3d4c0).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [WIP] [SQL] Handle CreateFunctio...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-205070732
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/54813/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SQL] Handle CreateFunction/Drop...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58454910
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -428,53 +432,85 @@ class SessionCatalog(
        */
       def dropFunction(name: FunctionIdentifier): Unit = {
         val db = name.database.getOrElse(currentDb)
    +    val qualified = name.copy(database = Some(db)).unquotedString
    +    if (functionRegistry.functionExists(qualified)) {
    +      // If we have loaded this function into FunctionRegistry,
    +      // also drop it from there.
    +      functionRegistry.dropFunction(qualified)
    +    }
         externalCatalog.dropFunction(db, name.funcName)
       }
     
       /**
    -   * Alter a metastore function whose name that matches the one specified in `funcDefinition`.
    -   *
    -   * If no database is specified in `funcDefinition`, assume the function is in the
    -   * current database.
    -   *
    -   * Note: If the underlying implementation does not support altering a certain field,
    -   * this becomes a no-op.
    -   */
    -  def alterFunction(funcDefinition: CatalogFunction): Unit = {
    -    val db = funcDefinition.identifier.database.getOrElse(currentDb)
    -    val newFuncDefinition = funcDefinition.copy(
    -      identifier = FunctionIdentifier(funcDefinition.identifier.funcName, Some(db)))
    -    externalCatalog.alterFunction(db, newFuncDefinition)
    -  }
    -
    -  /**
        * Retrieve the metadata of a metastore function.
        *
        * If a database is specified in `name`, this will return the function in that database.
        * If no database is specified, this will return the function in the current database.
        */
    +  // TODO: have a better name. This method is actually for fetching the metadata of a function.
       def getFunction(name: FunctionIdentifier): CatalogFunction = {
         val db = name.database.getOrElse(currentDb)
         externalCatalog.getFunction(db, name.funcName)
       }
     
    +  /**
    +   * Check if a function is already existing.
    +   *
    +   */
    +  def functionExists(name: FunctionIdentifier): Boolean = {
    +    if (functionRegistry.functionExists(name.unquotedString)) {
    +      // This function exists in the FunctionRegistry.
    +      true
    +    } else {
    +      // Need to check if this function exists in the metastore.
    +      try {
    +        getFunction(name) != null
    +      } catch {
    +        case _: NoSuchFunctionException => false
    +        case _: AnalysisException => false // HiveExternalCatalog wraps all exceptions with it.
    --- End diff --
    
    This use of `try` is kind of hacky. The right thing to do here is to implement `functionExists` in the external catalog as well. That means we also need to implement it in `HiveClientImpl`. We can do that later if you prefer.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [WIP] [SQL] Handle CreateFunctio...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58286920
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala ---
    @@ -211,10 +241,11 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton {
         checkExistence(sql("describe functioN abcadf"), true,
           "Function: abcadf not found.")
     
    -    checkExistence(sql("describe functioN  `~`"), true,
    -      "Function: ~",
    -      "Class: org.apache.hadoop.hive.ql.udf.UDFOPBitNot",
    -      "Usage: ~ n - Bitwise not")
    +    // TODO: Re-enable this test after we fix SPARK-14335.
    --- End diff --
    
    Why this? I remember #12036 passes this test.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58469061
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -428,53 +432,85 @@ class SessionCatalog(
        */
       def dropFunction(name: FunctionIdentifier): Unit = {
         val db = name.database.getOrElse(currentDb)
    +    val qualified = name.copy(database = Some(db)).unquotedString
    +    if (functionRegistry.functionExists(qualified)) {
    +      // If we have loaded this function into FunctionRegistry,
    +      // also drop it from there.
    +      functionRegistry.dropFunction(qualified)
    +    }
         externalCatalog.dropFunction(db, name.funcName)
       }
     
       /**
    -   * Alter a metastore function whose name that matches the one specified in `funcDefinition`.
    -   *
    -   * If no database is specified in `funcDefinition`, assume the function is in the
    -   * current database.
    -   *
    -   * Note: If the underlying implementation does not support altering a certain field,
    -   * this becomes a no-op.
    -   */
    -  def alterFunction(funcDefinition: CatalogFunction): Unit = {
    -    val db = funcDefinition.identifier.database.getOrElse(currentDb)
    -    val newFuncDefinition = funcDefinition.copy(
    -      identifier = FunctionIdentifier(funcDefinition.identifier.funcName, Some(db)))
    -    externalCatalog.alterFunction(db, newFuncDefinition)
    -  }
    -
    -  /**
        * Retrieve the metadata of a metastore function.
        *
        * If a database is specified in `name`, this will return the function in that database.
        * If no database is specified, this will return the function in the current database.
        */
    +  // TODO: have a better name. This method is actually for fetching the metadata of a function.
       def getFunction(name: FunctionIdentifier): CatalogFunction = {
         val db = name.database.getOrElse(currentDb)
         externalCatalog.getFunction(db, name.funcName)
       }
     
    +  /**
    +   * Check if a function is already existing.
    +   *
    +   */
    +  def functionExists(name: FunctionIdentifier): Boolean = {
    +    if (functionRegistry.functionExists(name.unquotedString)) {
    +      // This function exists in the FunctionRegistry.
    +      true
    +    } else {
    +      // Need to check if this function exists in the metastore.
    +      try {
    +        getFunction(name) != null
    +      } catch {
    +        case _: NoSuchFunctionException => false
    +        case _: AnalysisException => false // HiveExternalCatalog wraps all exceptions with it.
    --- End diff --
    
    yeah, I think it is not consistent that getFunction could return a null or an exception when it can't find the function. I did this part before is just to make think work first. Better to refactor it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SQL] Handle CreateFunction/Drop...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58329423
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala ---
    @@ -183,11 +203,12 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton {
         checkAnswer(sql("SHOW functions abc.abs"), Row("abs"))
         checkAnswer(sql("SHOW functions `abc`.`abs`"), Row("abs"))
         checkAnswer(sql("SHOW functions `abc`.`abs`"), Row("abs"))
    -    checkAnswer(sql("SHOW functions `~`"), Row("~"))
    +    // TODO: Re-enable this test after we fix SPARK-14335.
    +    // checkAnswer(sql("SHOW functions `~`"), Row("~"))
         checkAnswer(sql("SHOW functions `a function doens't exist`"), Nil)
    -    checkAnswer(sql("SHOW functions `weekofyea.*`"), Row("weekofyear"))
    +    checkAnswer(sql("SHOW functions `weekofyea*`"), Row("weekofyear"))
         // this probably will failed if we add more function with `sha` prefixing.
    -    checkAnswer(sql("SHOW functions `sha.*`"), Row("sha") :: Row("sha1") :: Row("sha2") :: Nil)
    +    checkAnswer(sql("SHOW functions `sha*`"), Row("sha") :: Row("sha1") :: Row("sha2") :: Nil)
    --- End diff --
    
    `*` is the wildcard for any character(s) not `.*`. So, I changed this test.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SQL] Handle CreateFunction/Drop...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-205449437
  
    **[Test build #54877 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/54877/consoleFull)** for PR 12117 at commit [`1f77973`](https://github.com/apache/spark/commit/1f779736c3c7f0b778fab4739fb7a1b4fd909d6a).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [WIP] [SQL] Handle CreateFunctio...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-204845983
  
    **[Test build #54785 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/54785/consoleFull)** for PR 12117 at commit [`3718e61`](https://github.com/apache/spark/commit/3718e613498cbf9a996f52bc3f215f1051f2ae51).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `case class UnresolvedGenerator(name: String, children: Seq[Expression]) extends Generator `


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SQL] Handle CreateFunction/Drop...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58455582
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -428,53 +432,85 @@ class SessionCatalog(
        */
       def dropFunction(name: FunctionIdentifier): Unit = {
         val db = name.database.getOrElse(currentDb)
    +    val qualified = name.copy(database = Some(db)).unquotedString
    +    if (functionRegistry.functionExists(qualified)) {
    +      // If we have loaded this function into FunctionRegistry,
    +      // also drop it from there.
    +      functionRegistry.dropFunction(qualified)
    --- End diff --
    
    I see... it's because in `lookupFunction` we actually load a persistent function into the registry as a temporary function...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [WIP] [SQL] Handle CreateFunctio...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58301749
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala ---
    @@ -211,10 +241,11 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton {
         checkExistence(sql("describe functioN abcadf"), true,
           "Function: abcadf not found.")
     
    -    checkExistence(sql("describe functioN  `~`"), true,
    -      "Function: ~",
    -      "Class: org.apache.hadoop.hive.ql.udf.UDFOPBitNot",
    -      "Usage: ~ n - Bitwise not")
    +    // TODO: Re-enable this test after we fix SPARK-14335.
    --- End diff --
    
    The output of this test shows that we are loading a hive function. But, we actually have implementation of `~`. With this PR, we will lazily load hive's built-in functions and `describe` command will not trigger the load. So, this test will fail with a message like `Undefined function`. SPARK-14335 will fix this problem.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-205616654
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SQL] Handle CreateFunction/Drop...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-205401300
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/54857/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58466044
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala ---
    @@ -0,0 +1,109 @@
    +/*
    + * 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.spark.sql.execution.command
    +
    +import org.apache.spark.sql.{AnalysisException, Row, SQLContext}
    +import org.apache.spark.sql.catalyst.FunctionIdentifier
    +import org.apache.spark.sql.catalyst.catalog.CatalogFunction
    +import org.apache.spark.sql.catalyst.expressions.ExpressionInfo
    +
    +
    +/**
    + * The DDL command that creates a function.
    + * alias: the class name that implements the created function.
    + * resources: Jars, files, or archives which need to be added to the environment when the function
    + *            is referenced for the first time by a session.
    + * isTemp: indicates if it is a temporary function.
    --- End diff --
    
    can you follow the javadoc format of the other commands, something like:
    ```
      * ...
      * The syntax of using this command in SQL is:
      * {{{
      *    SHOW FUNCTIONS [LIKE pattern]
      * }}}
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [WIP] [SQL] Handle CreateFunctio...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58301786
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -428,53 +432,86 @@ class SessionCatalog(
        */
       def dropFunction(name: FunctionIdentifier): Unit = {
         val db = name.database.getOrElse(currentDb)
    +    val qualified = name.copy(database = Some(db)).unquotedString
    +    if (functionRegistry.functionExists(qualified)) {
    +      // If we have loaded this function into FunctionRegistry,
    +      // also drop it from there.
    +      functionRegistry.dropFunction(qualified)
    +    }
         externalCatalog.dropFunction(db, name.funcName)
       }
     
       /**
    -   * Alter a metastore function whose name that matches the one specified in `funcDefinition`.
    -   *
    -   * If no database is specified in `funcDefinition`, assume the function is in the
    -   * current database.
    -   *
    -   * Note: If the underlying implementation does not support altering a certain field,
    -   * this becomes a no-op.
    -   */
    -  def alterFunction(funcDefinition: CatalogFunction): Unit = {
    -    val db = funcDefinition.identifier.database.getOrElse(currentDb)
    -    val newFuncDefinition = funcDefinition.copy(
    -      identifier = FunctionIdentifier(funcDefinition.identifier.funcName, Some(db)))
    -    externalCatalog.alterFunction(db, newFuncDefinition)
    -  }
    --- End diff --
    
    Users cannot alter functions (no API exposed). So, I just delete it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58472411
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -428,53 +432,85 @@ class SessionCatalog(
        */
       def dropFunction(name: FunctionIdentifier): Unit = {
         val db = name.database.getOrElse(currentDb)
    +    val qualified = name.copy(database = Some(db)).unquotedString
    +    if (functionRegistry.functionExists(qualified)) {
    +      // If we have loaded this function into FunctionRegistry,
    +      // also drop it from there.
    +      functionRegistry.dropFunction(qualified)
    +    }
         externalCatalog.dropFunction(db, name.funcName)
       }
     
       /**
    -   * Alter a metastore function whose name that matches the one specified in `funcDefinition`.
    -   *
    -   * If no database is specified in `funcDefinition`, assume the function is in the
    -   * current database.
    -   *
    -   * Note: If the underlying implementation does not support altering a certain field,
    -   * this becomes a no-op.
    -   */
    -  def alterFunction(funcDefinition: CatalogFunction): Unit = {
    -    val db = funcDefinition.identifier.database.getOrElse(currentDb)
    -    val newFuncDefinition = funcDefinition.copy(
    -      identifier = FunctionIdentifier(funcDefinition.identifier.funcName, Some(db)))
    -    externalCatalog.alterFunction(db, newFuncDefinition)
    -  }
    -
    -  /**
        * Retrieve the metadata of a metastore function.
        *
        * If a database is specified in `name`, this will return the function in that database.
        * If no database is specified, this will return the function in the current database.
        */
    +  // TODO: have a better name. This method is actually for fetching the metadata of a function.
       def getFunction(name: FunctionIdentifier): CatalogFunction = {
         val db = name.database.getOrElse(currentDb)
         externalCatalog.getFunction(db, name.funcName)
       }
     
    +  /**
    +   * Check if a function is already existing.
    +   *
    +   */
    +  def functionExists(name: FunctionIdentifier): Boolean = {
    +    if (functionRegistry.functionExists(name.unquotedString)) {
    +      // This function exists in the FunctionRegistry.
    +      true
    +    } else {
    +      // Need to check if this function exists in the metastore.
    +      try {
    +        getFunction(name) != null
    +      } catch {
    +        case _: NoSuchFunctionException => false
    +        case _: AnalysisException => false // HiveExternalCatalog wraps all exceptions with it.
    --- End diff --
    
    Added a TODO.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58477709
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala ---
    @@ -208,6 +261,118 @@ class HiveSparkSubmitSuite
       }
     }
     
    +// This application is used to test defining a new Hive UDF (with an associated jar)
    --- End diff --
    
    Yea. This is good!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SQL] Handle CreateFunction/Drop...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58329324
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala ---
    @@ -112,4 +124,121 @@ class HiveSessionCatalog(
         metastoreCatalog.cachedDataSourceTables.getIfPresent(key)
       }
     
    +  override def makeFunctionBuilder(funcName: String, className: String): FunctionBuilder = {
    +    makeFunctionBuilder(funcName, Utils.classForName(className))
    +  }
    +
    +  /**
    +   * Construct a [[FunctionBuilder]] based on the provided class that represents a function.
    +   */
    +  private def makeFunctionBuilder(name: String, clazz: Class[_]): FunctionBuilder = {
    +    // When we instantiate hive UDF wrapper class, we may throw exception if the input
    +    // expressions don't satisfy the hive UDF, such as type mismatch, input number
    +    // mismatch, etc. Here we catch the exception and throw AnalysisException instead.
    +    (children: Seq[Expression]) => {
    +      try {
    +        if (classOf[UDF].isAssignableFrom(clazz)) {
    +          val udf = HiveSimpleUDF(name, new HiveFunctionWrapper(clazz.getName), children)
    +          udf.dataType // Force it to check input data types.
    +          udf
    +        } else if (classOf[GenericUDF].isAssignableFrom(clazz)) {
    +          val udf = HiveGenericUDF(name, new HiveFunctionWrapper(clazz.getName), children)
    +          udf.dataType // Force it to check input data types.
    +          udf
    +        } else if (classOf[AbstractGenericUDAFResolver].isAssignableFrom(clazz)) {
    +          val udaf = HiveUDAFFunction(name, new HiveFunctionWrapper(clazz.getName), children)
    +          udaf.dataType // Force it to check input data types.
    +          udaf
    +        } else if (classOf[UDAF].isAssignableFrom(clazz)) {
    +          val udaf = HiveUDAFFunction(
    +            name,
    +            new HiveFunctionWrapper(clazz.getName),
    +            children,
    +            isUDAFBridgeRequired = true)
    +          udaf.dataType  // Force it to check input data types.
    +          udaf
    +        } else if (classOf[GenericUDTF].isAssignableFrom(clazz)) {
    +          val udtf = HiveGenericUDTF(name, new HiveFunctionWrapper(clazz.getName), children)
    +          udtf.elementTypes // Force it to check input data types.
    +          udtf
    +        } else {
    +          throw new AnalysisException(s"No handler for Hive UDF '${clazz.getCanonicalName}'")
    +        }
    +      } catch {
    +        case ae: AnalysisException =>
    +          throw ae
    +        case NonFatal(e) =>
    +          val analysisException =
    +            new AnalysisException(s"No handler for Hive UDF '${clazz.getCanonicalName}': $e")
    +          analysisException.setStackTrace(e.getStackTrace)
    +          throw analysisException
    +      }
    +    }
    +  }
    +
    +  // We have a list of Hive built-in functions that we do not support. So, we will check
    +  // Hive's function registry and lazily load needed functions into our own function registry.
    +  // Those Hive built-in functions are
    +  // assert_true, collect_list, collect_set, compute_stats, context_ngrams, create_union,
    +  // current_user ,elt, ewah_bitmap, ewah_bitmap_and, ewah_bitmap_empty, ewah_bitmap_or, field,
    +  // histogram_numeric, in_file, index, inline, java_method, map_keys, map_values,
    +  // matchpath, ngrams, noop, noopstreaming, noopwithmap, noopwithmapstreaming,
    +  // parse_url, parse_url_tuple, percentile, percentile_approx, posexplode, reflect, reflect2,
    +  // regexp, sentences, stack, std, str_to_map, windowingtablefunction, xpath, xpath_boolean,
    +  // xpath_double, xpath_float, xpath_int, xpath_long, xpath_number,
    +  // xpath_short, and xpath_string.
    +  override def lookupFunction(name: String, children: Seq[Expression]): Expression = {
    +    Try(super.lookupFunction(name, children)) match {
    +      case Success(expr) => expr
    +      case Failure(error) =>
    +        if (functionRegistry.functionExists(name)) {
    +          // If the function actually exists in functionRegistry, it means that there is an
    +          // error when we create the Expression using the given children.
    +          // We need to throw the original exception.
    +          throw error
    +        } else {
    +          // This function is not in functionRegistry, let's try to load it as a Hive's
    +          // built-in function.
    +          val functionName = name.toLowerCase
    +          // TODO: This may not really work for current_user because current_user is not evaluated
    +          // with session info.
    +          // We do not need to use executionHive at here because we only load
    +          // Hive's builtin functions, which do not need current db.
    +          val functionInfo = {
    +            try {
    +              Option(HiveFunctionRegistry.getFunctionInfo(functionName)).getOrElse(
    +                throw new AnalysisException(s"Undefined Hive UDF: $name"))
    +            } catch {
    +              // If HiveFunctionRegistry.getFunctionInfo throws an exception,
    +              // we are failing to load a Hive builtin function, which means that
    +              // the given function is not a Hive builtin function.
    +              case NonFatal(e) => throw new AnalysisException(s"Undefined Hive UDF: $name")
    +            }
    +          }
    +          val className = functionInfo.getFunctionClass.getName
    +          val builder = makeFunctionBuilder(functionName, className)
    +          // Put this Hive built-in function to our function registry.
    +          val info = new ExpressionInfo(className, functionName)
    +          createTempFunction(functionName, info, builder, ignoreIfExists = false)
    +          // Now, we need to create the Expression.
    +          functionRegistry.lookupFunction(functionName, children)
    +        }
    +    }
    +  }
    +
    +  // Pre-load a few commonly used Hive built-in functions.
    +  HiveSessionCatalog.preloadedHiveBuiltinFunctions.foreach {
    --- End diff --
    
    I'd like to keep it close to `lookupFunction` and the object defined below since they are related to functions.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SQL] Handle CreateFunction/Drop...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-205143016
  
    **[Test build #54824 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/54824/consoleFull)** for PR 12117 at commit [`9d39a83`](https://github.com/apache/spark/commit/9d39a835a25ea2a509fa490099a7923b70e8bcbd).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SQL] Handle CreateFunction/Drop...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58329395
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala ---
    @@ -61,8 +61,12 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext {
             .filter(regex.matcher(_).matches()).map(Row(_))
         }
         checkAnswer(sql("SHOW functions"), getFunctions(".*"))
    -    Seq("^c.*", ".*e$", "log.*", ".*date.*").foreach { pattern =>
    -      checkAnswer(sql(s"SHOW FUNCTIONS '$pattern'"), getFunctions(pattern))
    +    Seq("^c*", "*e$", "log*", "*date*").foreach { pattern =>
    --- End diff --
    
    `*` is for any character(s) not `.*`. So, I changed this test.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SQL] Handle CreateFunction/Drop...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-205183927
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58474474
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala ---
    @@ -0,0 +1,109 @@
    +/*
    + * 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.spark.sql.execution.command
    +
    +import org.apache.spark.sql.{AnalysisException, Row, SQLContext}
    +import org.apache.spark.sql.catalyst.FunctionIdentifier
    +import org.apache.spark.sql.catalyst.catalog.CatalogFunction
    +import org.apache.spark.sql.catalyst.expressions.ExpressionInfo
    +
    +
    +/**
    + * The DDL command that creates a function.
    + * alias: the class name that implements the created function.
    + * resources: Jars, files, or archives which need to be added to the environment when the function
    + *            is referenced for the first time by a session.
    + * isTemp: indicates if it is a temporary function.
    + */
    +// TODO: Use Seq[FunctionResource] instead of Seq[(String, String)] for resources.
    +case class CreateFunction(
    +    databaseName: Option[String],
    +    functionName: String,
    +    className: String,
    +    resources: Seq[(String, String)],
    +    isTemp: Boolean)
    +  extends RunnableCommand {
    +
    +  override def run(sqlContext: SQLContext): Seq[Row] = {
    +    if (isTemp) {
    +      if (databaseName.isDefined) {
    +        throw new AnalysisException(
    +          s"It is not allowed to provide database name when defining a temporary function. " +
    +            s"However, database name ${databaseName.get} is provided.")
    +      }
    +      // We first load resources and then put the builder in the function registry.
    +      // Please note that it is allowed to overwrite an existing temp function.
    +      sqlContext.sessionState.catalog.loadFunctionResources(resources)
    +      val info = new ExpressionInfo(className, functionName)
    +      val builder =
    +        sqlContext.sessionState.catalog.makeFunctionBuilder(functionName, className)
    +      sqlContext.sessionState.catalog.createTempFunction(
    +        functionName, info, builder, ignoreIfExists = false)
    +    } else {
    +      val dbName = databaseName.getOrElse(sqlContext.sessionState.catalog.getCurrentDatabase)
    +      val func = FunctionIdentifier(functionName, Some(dbName))
    +      val catalogFunc = CatalogFunction(func, className, resources)
    +      // We are creating a permanent function. First, we want to check if this function
    +      // has already been created.
    +      // Check if the function to create is already existing. If so, throw exception.
    +      if (sqlContext.sessionState.catalog.functionExists(func)) {
    +        throw new AnalysisException(
    +          s"Function '$functionName' already exists in database '$dbName'.")
    +      }
    +      // This function will be loaded into the FunctionRegistry when a query uses it.
    +      sqlContext.sessionState.catalog.createFunction(catalogFunc)
    +    }
    +    Seq.empty[Row]
    +  }
    +}
    +
    +/**
    + * The DDL command that drops a function.
    + * ifExists: returns an error if the function doesn't exist, unless this is true.
    + * isTemp: indicates if it is a temporary function.
    + */
    +case class DropFunction(
    +    databaseName: Option[String],
    +    functionName: String,
    +    ifExists: Boolean,
    +    isTemp: Boolean)
    +  extends RunnableCommand {
    +
    +  override def run(sqlContext: SQLContext): Seq[Row] = {
    +    if (isTemp) {
    +      if (databaseName.isDefined) {
    +        throw new AnalysisException(
    +          s"It is not allowed to provide database name when dropping a temporary function. " +
    +            s"However, database name ${databaseName.get} is provided.")
    +      }
    +      sqlContext.sessionState.catalog.dropTempFunction(functionName, ifExists)
    +    } else {
    +      // We are dropping a permanent.
    +      val dbName = databaseName.getOrElse(sqlContext.sessionState.catalog.getCurrentDatabase)
    +      val func = FunctionIdentifier(functionName, Some(dbName))
    --- End diff --
    
    Yea, I am trying to get the actual database name for the error message.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SQL] Handle CreateFunction/DropFunction

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58282614
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala ---
    @@ -67,9 +70,14 @@ class SimpleFunctionRegistry extends FunctionRegistry {
       }
     
       override def lookupFunction(name: String, children: Seq[Expression]): Expression = {
    +    val builder = functionBuilders.get(name)
    +    if (builder.isEmpty) {
    +      throw new AnalysisException(s"undefined function $name")
    +    }
         val func = synchronized {
    -      functionBuilders.get(name).map(_._2).getOrElse {
    -        throw new AnalysisException(s"undefined function $name")
    +      Try(builder.map(_._2)) match {
    --- End diff --
    
    I still don't understand why this needs to be changed. This is just calling `Option.map` and then getting the second field, which is what the old code does as well.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [WIP] [SQL] Handle CreateFunctio...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58301583
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -453,28 +464,80 @@ class SessionCatalog(
        * If a database is specified in `name`, this will return the function in that database.
        * If no database is specified, this will return the function in the current database.
        */
    +  // TODO: have a better name. This method is actually for fetching the metadata of a function.
       def getFunction(name: FunctionIdentifier): CatalogFunction = {
         val db = name.database.getOrElse(currentDb)
         externalCatalog.getFunction(db, name.funcName)
       }
     
    +  /**
    +   * Check if a function is already existing.
    +   *
    +   */
    +  def functionExists(name: FunctionIdentifier): Boolean = {
    +    if (functionRegistry.functionExists(name.unquotedString)) {
    +      // This function exists in the FunctionRegistry.
    +      true
    +    } else {
    +      // Need to check if this function exists in the metastore.
    +      try {
    +        getFunction(name) != null
    +      } catch {
    +        case _: NoSuchFunctionException => false
    +        case _: AnalysisException => false // HiveExternalCatalog wraps all exceptions with it.
    +      }
    +    }
    +  }
     
       // ----------------------------------------------------------------
       // | Methods that interact with temporary and metastore functions |
       // ----------------------------------------------------------------
     
    +
    +  /**
    +   * Return a temporary function. For testing only.
    +   */
    +  private[catalog] def getTempFunction(name: String): Option[FunctionBuilder] = {
    +    // TODO: Why do we need this?
    --- End diff --
    
    Will delete it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SQL] Handle CreateFunction/Drop...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58455003
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -428,53 +432,85 @@ class SessionCatalog(
        */
       def dropFunction(name: FunctionIdentifier): Unit = {
         val db = name.database.getOrElse(currentDb)
    +    val qualified = name.copy(database = Some(db)).unquotedString
    +    if (functionRegistry.functionExists(qualified)) {
    +      // If we have loaded this function into FunctionRegistry,
    +      // also drop it from there.
    +      functionRegistry.dropFunction(qualified)
    +    }
         externalCatalog.dropFunction(db, name.funcName)
       }
     
       /**
    -   * Alter a metastore function whose name that matches the one specified in `funcDefinition`.
    -   *
    -   * If no database is specified in `funcDefinition`, assume the function is in the
    -   * current database.
    -   *
    -   * Note: If the underlying implementation does not support altering a certain field,
    -   * this becomes a no-op.
    -   */
    -  def alterFunction(funcDefinition: CatalogFunction): Unit = {
    -    val db = funcDefinition.identifier.database.getOrElse(currentDb)
    -    val newFuncDefinition = funcDefinition.copy(
    -      identifier = FunctionIdentifier(funcDefinition.identifier.funcName, Some(db)))
    -    externalCatalog.alterFunction(db, newFuncDefinition)
    -  }
    -
    -  /**
        * Retrieve the metadata of a metastore function.
        *
        * If a database is specified in `name`, this will return the function in that database.
        * If no database is specified, this will return the function in the current database.
        */
    +  // TODO: have a better name. This method is actually for fetching the metadata of a function.
       def getFunction(name: FunctionIdentifier): CatalogFunction = {
         val db = name.database.getOrElse(currentDb)
         externalCatalog.getFunction(db, name.funcName)
       }
     
    +  /**
    +   * Check if a function is already existing.
    +   *
    --- End diff --
    
    also the java doc should be `Check if the specified function exists.`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58465188
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala ---
    @@ -396,18 +400,16 @@ case class ShowFunctions(db: Option[String], pattern: Option[String]) extends Ru
         schema.toAttributes
       }
     
    -  override def run(sqlContext: SQLContext): Seq[Row] = pattern match {
    -    case Some(p) =>
    -      try {
    -        val regex = java.util.regex.Pattern.compile(p)
    -        sqlContext.sessionState.functionRegistry.listFunction()
    -          .filter(regex.matcher(_).matches()).map(Row(_))
    -      } catch {
    -        // probably will failed in the regex that user provided, then returns empty row.
    -        case _: Throwable => Seq.empty[Row]
    -      }
    -    case None =>
    -      sqlContext.sessionState.functionRegistry.listFunction().map(Row(_))
    +  override def run(sqlContext: SQLContext): Seq[Row] = {
    +    val dbName = db.getOrElse(sqlContext.sessionState.catalog.getCurrentDatabase)
    +    val functionNames =
    +      sqlContext.sessionState.catalog
    +        .listFunctions(dbName, pattern.getOrElse(".*"))
    +        .map(_.unquotedString)
    +    // We use distinct at here because SessionCatalog's listFunctions will return
    +    // duplicated entries for UDFs that have already been loaded into the
    --- End diff --
    
    I think more concisely you can say:
    ```
    // The session catalog caches some persistent functions in the temporary registry
    // so there can be duplicates.
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58466759
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveSqlParser.scala ---
    @@ -21,19 +21,19 @@ import scala.collection.JavaConverters._
     import org.antlr.v4.runtime.{ParserRuleContext, Token}
     import org.apache.hadoop.hive.conf.HiveConf
     import org.apache.hadoop.hive.conf.HiveConf.ConfVars
    -import org.apache.hadoop.hive.ql.exec.FunctionRegistry
     import org.apache.hadoop.hive.ql.parse.EximUtil
     import org.apache.hadoop.hive.ql.session.SessionState
     import org.apache.hadoop.hive.serde.serdeConstants
     import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe
     
    -import org.apache.spark.sql.catalyst.catalog.{CatalogColumn, CatalogStorageFormat, CatalogTable, CatalogTableType}
    +import org.apache.spark.sql.catalyst.analysis.UnresolvedGenerator
    +import org.apache.spark.sql.catalyst.catalog._
     import org.apache.spark.sql.catalyst.expressions._
     import org.apache.spark.sql.catalyst.parser._
     import org.apache.spark.sql.catalyst.parser.SqlBaseParser._
     import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
     import org.apache.spark.sql.execution.SparkSqlAstBuilder
    -import org.apache.spark.sql.hive.{CreateTableAsSelect => CTAS, CreateViewAsSelect => CreateView}
    +import org.apache.spark.sql.hive.{CreateTableAsSelect => CTAS, CreateViewAsSelect => CreateView, HiveSessionState}
    --- End diff --
    
    not used?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SQL] Handle CreateFunction/Drop...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58454405
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala ---
    @@ -146,6 +146,7 @@ abstract class ExternalCatalog {
     
       def createFunction(db: String, funcDefinition: CatalogFunction): Unit
     
    +  // TODO: dropFunction should take a FunctionIdentifier.
    --- End diff --
    
    I don't think so... it's largely by design that we just pass in the strings here. This is the external catalog which shouldn't know anything about current databases, so the effective database should already be resolved at this point.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SQL] Handle CreateFunction/DropFunction

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58282708
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala ---
    @@ -133,6 +133,30 @@ object UnresolvedAttribute {
       }
     }
     
    +case class UnresolvedGenerator(
    --- End diff --
    
    can you add a short java doc to explain what this is for?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-205625938
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/54945/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-205623750
  
    **[Test build #54945 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/54945/consoleFull)** for PR 12117 at commit [`8a41f6d`](https://github.com/apache/spark/commit/8a41f6df56cede48107e81f50df2111ddaf5e18d).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SQL] Handle CreateFunction/Drop...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58413487
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala ---
    @@ -183,11 +203,12 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton {
         checkAnswer(sql("SHOW functions abc.abs"), Row("abs"))
         checkAnswer(sql("SHOW functions `abc`.`abs`"), Row("abs"))
         checkAnswer(sql("SHOW functions `abc`.`abs`"), Row("abs"))
    -    checkAnswer(sql("SHOW functions `~`"), Row("~"))
    +    // TODO: Re-enable this test after we fix SPARK-14335.
    +    // checkAnswer(sql("SHOW functions `~`"), Row("~"))
         checkAnswer(sql("SHOW functions `a function doens't exist`"), Nil)
    -    checkAnswer(sql("SHOW functions `weekofyea.*`"), Row("weekofyear"))
    +    checkAnswer(sql("SHOW functions `weekofyea*`"), Row("weekofyear"))
         // this probably will failed if we add more function with `sha` prefixing.
    -    checkAnswer(sql("SHOW functions `sha.*`"), Row("sha") :: Row("sha1") :: Row("sha2") :: Nil)
    +    checkAnswer(sql("SHOW functions `sha*`"), Row("sha") :: Row("sha1") :: Row("sha2") :: Nil)
    --- End diff --
    
    this will conflict with #12104


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58471970
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala ---
    @@ -341,9 +343,14 @@ object FunctionRegistry {
       /** See usage above. */
       def expression[T <: Expression](name: String)
           (implicit tag: ClassTag[T]): (String, (ExpressionInfo, FunctionBuilder)) = {
    +    expression(name, tag.runtimeClass.asInstanceOf[Class[T]])
    +  }
     
    +  def expression[T <: Expression](
    +      name: String,
    +      clazz: Class[T]): (String, (ExpressionInfo, FunctionBuilder)) = {
    --- End diff --
    
    Done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58601677
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala ---
    @@ -85,24 +91,23 @@ case class DropFunction(
       extends RunnableCommand {
     
       override def run(sqlContext: SQLContext): Seq[Row] = {
    +    val catalog = sqlContext.sessionState.catalog
         if (isTemp) {
           if (databaseName.isDefined) {
             throw new AnalysisException(
               s"It is not allowed to provide database name when dropping a temporary function. " +
                 s"However, database name ${databaseName.get} is provided.")
           }
    -      sqlContext.sessionState.catalog.dropTempFunction(functionName, ifExists)
    +      catalog.dropTempFunction(functionName, ifExists)
         } else {
    -      // We are dropping a permanent.
    -      val dbName = databaseName.getOrElse(sqlContext.sessionState.catalog.getCurrentDatabase)
    +      // We are dropping a permanent function.
    +      val dbName = databaseName.getOrElse(catalog.getCurrentDatabase)
           val func = FunctionIdentifier(functionName, Some(dbName))
    -      if (!ifExists) {
    -        if (!sqlContext.sessionState.catalog.functionExists(func)) {
    -          throw new AnalysisException(
    -            s"Function '$functionName' does not exist in database '$dbName'.")
    -        }
    +      if (!ifExists && !catalog.functionExists(func)) {
    +        throw new AnalysisException(
    +          s"Function '$functionName' does not exist in database '$dbName'.")
    --- End diff --
    
    oh yes. I totally agree. We should do it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58466485
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala ---
    @@ -0,0 +1,109 @@
    +/*
    + * 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.spark.sql.execution.command
    +
    +import org.apache.spark.sql.{AnalysisException, Row, SQLContext}
    +import org.apache.spark.sql.catalyst.FunctionIdentifier
    +import org.apache.spark.sql.catalyst.catalog.CatalogFunction
    +import org.apache.spark.sql.catalyst.expressions.ExpressionInfo
    +
    +
    +/**
    + * The DDL command that creates a function.
    + * alias: the class name that implements the created function.
    + * resources: Jars, files, or archives which need to be added to the environment when the function
    + *            is referenced for the first time by a session.
    + * isTemp: indicates if it is a temporary function.
    + */
    +// TODO: Use Seq[FunctionResource] instead of Seq[(String, String)] for resources.
    +case class CreateFunction(
    +    databaseName: Option[String],
    +    functionName: String,
    +    className: String,
    +    resources: Seq[(String, String)],
    +    isTemp: Boolean)
    +  extends RunnableCommand {
    +
    +  override def run(sqlContext: SQLContext): Seq[Row] = {
    +    if (isTemp) {
    +      if (databaseName.isDefined) {
    +        throw new AnalysisException(
    +          s"It is not allowed to provide database name when defining a temporary function. " +
    +            s"However, database name ${databaseName.get} is provided.")
    +      }
    +      // We first load resources and then put the builder in the function registry.
    +      // Please note that it is allowed to overwrite an existing temp function.
    +      sqlContext.sessionState.catalog.loadFunctionResources(resources)
    +      val info = new ExpressionInfo(className, functionName)
    +      val builder =
    +        sqlContext.sessionState.catalog.makeFunctionBuilder(functionName, className)
    +      sqlContext.sessionState.catalog.createTempFunction(
    +        functionName, info, builder, ignoreIfExists = false)
    +    } else {
    +      val dbName = databaseName.getOrElse(sqlContext.sessionState.catalog.getCurrentDatabase)
    +      val func = FunctionIdentifier(functionName, Some(dbName))
    +      val catalogFunc = CatalogFunction(func, className, resources)
    +      // We are creating a permanent function. First, we want to check if this function
    +      // has already been created.
    +      // Check if the function to create is already existing. If so, throw exception.
    +      if (sqlContext.sessionState.catalog.functionExists(func)) {
    +        throw new AnalysisException(
    +          s"Function '$functionName' already exists in database '$dbName'.")
    +      }
    +      // This function will be loaded into the FunctionRegistry when a query uses it.
    +      sqlContext.sessionState.catalog.createFunction(catalogFunc)
    +    }
    +    Seq.empty[Row]
    +  }
    +}
    +
    +/**
    + * The DDL command that drops a function.
    + * ifExists: returns an error if the function doesn't exist, unless this is true.
    + * isTemp: indicates if it is a temporary function.
    + */
    +case class DropFunction(
    +    databaseName: Option[String],
    +    functionName: String,
    +    ifExists: Boolean,
    +    isTemp: Boolean)
    +  extends RunnableCommand {
    +
    +  override def run(sqlContext: SQLContext): Seq[Row] = {
    +    if (isTemp) {
    +      if (databaseName.isDefined) {
    +        throw new AnalysisException(
    +          s"It is not allowed to provide database name when dropping a temporary function. " +
    +            s"However, database name ${databaseName.get} is provided.")
    +      }
    +      sqlContext.sessionState.catalog.dropTempFunction(functionName, ifExists)
    +    } else {
    +      // We are dropping a permanent.
    +      val dbName = databaseName.getOrElse(sqlContext.sessionState.catalog.getCurrentDatabase)
    +      val func = FunctionIdentifier(functionName, Some(dbName))
    +      if (!ifExists) {
    +        if (!sqlContext.sessionState.catalog.functionExists(func)) {
    --- End diff --
    
    `if (!ifExists && !catalog.functionExists(func))`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [WIP] [SQL] Handle CreateFunctio...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-204846060
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/54785/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [WIP] [SQL] Handle CreateFunctio...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58317251
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/functionResources.scala ---
    @@ -0,0 +1,56 @@
    +/*
    + * 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.spark.sql.catalyst.catalog
    +
    +import org.apache.spark.sql.AnalysisException
    +
    +/** An trait that represents the type of a resourced needed by a function. */
    +sealed trait FunctionResourceType
    +
    +object JarResource extends FunctionResourceType
    +
    +object FileResource extends FunctionResourceType
    +
    +// We do nto allow users to specify a archive because it is YARN specific.
    +// When loading resources, we will throw an exception and ask users to
    +// use --archive with spark submit.
    +object ArchiveResource extends FunctionResourceType
    +
    +object FunctionResourceType {
    +  def fromString(resourceType: String): FunctionResourceType = {
    --- End diff --
    
    Q1: I am not sure about it. Right now, I think only add jar needs this concept. But, add jar command can just call SQLContext to load the jar. So, looks like we do not need to expand the scope of this concept for now.
    Q2: My only concern is that `(FunctionResourceType, String)` does not really tell what is the meaning of the `String`. But, I do not have a strong opinion on it. What do you think?
    Q3: Yea. I am thinking about doing that after this PR.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SQL] Handle CreateFunction/Drop...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58454950
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -428,53 +432,85 @@ class SessionCatalog(
        */
       def dropFunction(name: FunctionIdentifier): Unit = {
         val db = name.database.getOrElse(currentDb)
    +    val qualified = name.copy(database = Some(db)).unquotedString
    +    if (functionRegistry.functionExists(qualified)) {
    +      // If we have loaded this function into FunctionRegistry,
    +      // also drop it from there.
    +      functionRegistry.dropFunction(qualified)
    +    }
         externalCatalog.dropFunction(db, name.funcName)
       }
     
       /**
    -   * Alter a metastore function whose name that matches the one specified in `funcDefinition`.
    -   *
    -   * If no database is specified in `funcDefinition`, assume the function is in the
    -   * current database.
    -   *
    -   * Note: If the underlying implementation does not support altering a certain field,
    -   * this becomes a no-op.
    -   */
    -  def alterFunction(funcDefinition: CatalogFunction): Unit = {
    -    val db = funcDefinition.identifier.database.getOrElse(currentDb)
    -    val newFuncDefinition = funcDefinition.copy(
    -      identifier = FunctionIdentifier(funcDefinition.identifier.funcName, Some(db)))
    -    externalCatalog.alterFunction(db, newFuncDefinition)
    -  }
    -
    -  /**
        * Retrieve the metadata of a metastore function.
        *
        * If a database is specified in `name`, this will return the function in that database.
        * If no database is specified, this will return the function in the current database.
        */
    +  // TODO: have a better name. This method is actually for fetching the metadata of a function.
       def getFunction(name: FunctionIdentifier): CatalogFunction = {
         val db = name.database.getOrElse(currentDb)
         externalCatalog.getFunction(db, name.funcName)
       }
     
    +  /**
    +   * Check if a function is already existing.
    +   *
    --- End diff --
    
    remove blank line


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [WIP] [SQL] Handle CreateFunctio...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58301098
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala ---
    @@ -334,17 +342,18 @@ object FunctionRegistry {
     
       val builtin: SimpleFunctionRegistry = {
         val fr = new SimpleFunctionRegistry
    -    expressions.foreach { case (name, (info, builder)) => fr.registerFunction(name, info, builder) }
    +    expressions.foreach {
    +      case (name, (info, builder)) => fr.registerFunction(name, info, builder)
    --- End diff --
    
    Will revert most of them.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SQL] Handle CreateFunction/DropFunction

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58281993
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -453,28 +464,80 @@ class SessionCatalog(
        * If a database is specified in `name`, this will return the function in that database.
        * If no database is specified, this will return the function in the current database.
        */
    +  // TODO: have a better name. This method is actually for fetching the metadata of a function.
       def getFunction(name: FunctionIdentifier): CatalogFunction = {
         val db = name.database.getOrElse(currentDb)
         externalCatalog.getFunction(db, name.funcName)
       }
     
    +  /**
    +   * Check if a function is already existing.
    +   *
    +   */
    +  def functionExists(name: FunctionIdentifier): Boolean = {
    +    if (functionRegistry.functionExists(name.unquotedString)) {
    +      // This function exists in the FunctionRegistry.
    +      true
    +    } else {
    +      // Need to check if this function exists in the metastore.
    +      try {
    +        getFunction(name) != null
    +      } catch {
    +        case _: NoSuchFunctionException => false
    +        case _: AnalysisException => false // HiveExternalCatalog wraps all exceptions with it.
    +      }
    +    }
    +  }
     
       // ----------------------------------------------------------------
       // | Methods that interact with temporary and metastore functions |
       // ----------------------------------------------------------------
     
    +
    +  /**
    +   * Return a temporary function. For testing only.
    +   */
    +  private[catalog] def getTempFunction(name: String): Option[FunctionBuilder] = {
    +    // TODO: Why do we need this?
    +    functionRegistry.lookupFunctionBuilder(name)
    +  }
    +
    +  /**
    +   * Construct a [[FunctionBuilder]] based on the provided class that represents a function.
    +   *
    +   * This performs reflection to decide what type of [[Expression]] to return in the builder.
    +   * This is useful for creating temporary functions.
    +   */
    +  private[sql] def makeFunctionBuilder(name: String, functionClassName: String): FunctionBuilder = {
    +    // TODO: at least support UDAFs here
    +    throw new UnsupportedOperationException("Use sqlContext.udf.register(...) instead.")
    +  }
    +
    +  /**
    +   * Loads resources such as JARs and Files to SQLContext.
    +   */
    +  def loadFunctionResources(resources: Seq[(String, String)]): Unit = {
    +    resources.map(r => (r._1.toLowerCase, r._2)).foreach {
    --- End diff --
    
    just do
    ```
    resources.foreach { case (resourceType, uri) =>
      ...
    }
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [WIP] [SQL] Handle CreateFunctio...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-204655381
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/54764/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SQL] Handle CreateFunction/DropFunction

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58282402
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala ---
    @@ -104,5 +102,4 @@ private[hive] class HiveSessionState(ctx: HiveContext) extends SessionState(ctx)
           }
         }
       }
    -
    --- End diff --
    
    nit: this blank line is intentional. Please do not remove!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SQL] Handle CreateFunction/DropFunction

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58282424
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala ---
    @@ -41,118 +42,6 @@ import org.apache.spark.sql.hive.HiveShim._
     import org.apache.spark.sql.hive.client.HiveClientImpl
     import org.apache.spark.sql.types._
     
    -
    -private[hive] class HiveFunctionRegistry(
    --- End diff --
    
    Great!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [WIP] [SQL] Handle CreateFunctio...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-205057256
  
    **[Test build #54810 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/54810/consoleFull)** for PR 12117 at commit [`ae359fb`](https://github.com/apache/spark/commit/ae359fb60dcbb34514487553ac3c7fb24de3d4c0).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-205878599
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SQL] Handle CreateFunction/DropFunction

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58282680
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala ---
    @@ -334,17 +342,18 @@ object FunctionRegistry {
     
       val builtin: SimpleFunctionRegistry = {
         val fr = new SimpleFunctionRegistry
    -    expressions.foreach { case (name, (info, builder)) => fr.registerFunction(name, info, builder) }
    +    expressions.foreach {
    +      case (name, (info, builder)) => fr.registerFunction(name, info, builder)
    --- End diff --
    
    almost all the changes in this file are not related


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58466538
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala ---
    @@ -0,0 +1,109 @@
    +/*
    + * 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.spark.sql.execution.command
    +
    +import org.apache.spark.sql.{AnalysisException, Row, SQLContext}
    +import org.apache.spark.sql.catalyst.FunctionIdentifier
    +import org.apache.spark.sql.catalyst.catalog.CatalogFunction
    +import org.apache.spark.sql.catalyst.expressions.ExpressionInfo
    +
    +
    +/**
    + * The DDL command that creates a function.
    + * alias: the class name that implements the created function.
    + * resources: Jars, files, or archives which need to be added to the environment when the function
    + *            is referenced for the first time by a session.
    + * isTemp: indicates if it is a temporary function.
    + */
    +// TODO: Use Seq[FunctionResource] instead of Seq[(String, String)] for resources.
    +case class CreateFunction(
    +    databaseName: Option[String],
    +    functionName: String,
    +    className: String,
    +    resources: Seq[(String, String)],
    +    isTemp: Boolean)
    +  extends RunnableCommand {
    +
    +  override def run(sqlContext: SQLContext): Seq[Row] = {
    +    if (isTemp) {
    +      if (databaseName.isDefined) {
    +        throw new AnalysisException(
    +          s"It is not allowed to provide database name when defining a temporary function. " +
    +            s"However, database name ${databaseName.get} is provided.")
    +      }
    +      // We first load resources and then put the builder in the function registry.
    +      // Please note that it is allowed to overwrite an existing temp function.
    +      sqlContext.sessionState.catalog.loadFunctionResources(resources)
    --- End diff --
    
    When we first connect to a metastore, we do not load all existing metastore functions. So, at here, we just match that behavior.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58473121
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala ---
    @@ -159,6 +160,7 @@ abstract class ExternalCatalog {
        */
       def alterFunction(db: String, funcDefinition: CatalogFunction): Unit
    --- End diff --
    
    Done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SQL] Handle CreateFunction/Drop...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-205142877
  
    @andrewor14 @viirya @hvanhovell This one is ready for review.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58465066
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala ---
    @@ -396,18 +400,16 @@ case class ShowFunctions(db: Option[String], pattern: Option[String]) extends Ru
         schema.toAttributes
       }
     
    -  override def run(sqlContext: SQLContext): Seq[Row] = pattern match {
    -    case Some(p) =>
    -      try {
    -        val regex = java.util.regex.Pattern.compile(p)
    -        sqlContext.sessionState.functionRegistry.listFunction()
    -          .filter(regex.matcher(_).matches()).map(Row(_))
    -      } catch {
    -        // probably will failed in the regex that user provided, then returns empty row.
    -        case _: Throwable => Seq.empty[Row]
    -      }
    -    case None =>
    -      sqlContext.sessionState.functionRegistry.listFunction().map(Row(_))
    +  override def run(sqlContext: SQLContext): Seq[Row] = {
    +    val dbName = db.getOrElse(sqlContext.sessionState.catalog.getCurrentDatabase)
    +    val functionNames =
    +      sqlContext.sessionState.catalog
    +        .listFunctions(dbName, pattern.getOrElse(".*"))
    --- End diff --
    
    it's not a regex right? shouldn't this be `*`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SQL] Handle CreateFunction/Drop...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-205493522
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SQL] Handle CreateFunction/Drop...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-205401293
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SQL] Handle CreateFunction/DropFunction

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-204614914
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SQL] Handle CreateFunction/Drop...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-205400376
  
    **[Test build #54857 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/54857/consoleFull)** for PR 12117 at commit [`a16395a`](https://github.com/apache/spark/commit/a16395a343e024cee8c641ee1ccd23161be1d447).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58482346
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala ---
    @@ -201,8 +201,13 @@ class TestHiveContext private[hive](
         }
     
         override lazy val functionRegistry = {
    -      new TestHiveFunctionRegistry(
    -        org.apache.spark.sql.catalyst.analysis.FunctionRegistry.builtin.copy(), self.executionHive)
    +      // TestHiveFunctionRegistry tracks removed functions. So, we cannot simply use
    --- End diff --
    
    Updated


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SQL] Handle CreateFunction/Drop...

Posted by bomeng <gi...@git.apache.org>.
Github user bomeng commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58420071
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -534,17 +579,11 @@ class SessionCatalog(
         val dbFunctions =
           externalCatalog.listFunctions(db, pattern).map { f => FunctionIdentifier(f, Some(db)) }
         val regex = pattern.replaceAll("\\*", ".*").r
    --- End diff --
    
    Please include my fix for that, or create an utility for all the occurrence.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-205652624
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [WIP] [SQL] Handle CreateFunctio...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58287012
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala ---
    @@ -112,4 +124,121 @@ class HiveSessionCatalog(
         metastoreCatalog.cachedDataSourceTables.getIfPresent(key)
       }
     
    +  override def makeFunctionBuilder(funcName: String, className: String): FunctionBuilder = {
    +    makeFunctionBuilder(funcName, Utils.classForName(className))
    +  }
    +
    +  /**
    +   * Construct a [[FunctionBuilder]] based on the provided class that represents a function.
    +   */
    +  private def makeFunctionBuilder(name: String, clazz: Class[_]): FunctionBuilder = {
    +    // When we instantiate hive UDF wrapper class, we may throw exception if the input
    +    // expressions don't satisfy the hive UDF, such as type mismatch, input number
    +    // mismatch, etc. Here we catch the exception and throw AnalysisException instead.
    +    (children: Seq[Expression]) => {
    +      try {
    +        if (classOf[UDF].isAssignableFrom(clazz)) {
    +          val udf = HiveSimpleUDF(name, new HiveFunctionWrapper(clazz.getName), children)
    +          udf.dataType // Force it to check input data types.
    +          udf
    +        } else if (classOf[GenericUDF].isAssignableFrom(clazz)) {
    +          val udf = HiveGenericUDF(name, new HiveFunctionWrapper(clazz.getName), children)
    +          udf.dataType // Force it to check input data types.
    +          udf
    +        } else if (classOf[AbstractGenericUDAFResolver].isAssignableFrom(clazz)) {
    +          val udaf = HiveUDAFFunction(name, new HiveFunctionWrapper(clazz.getName), children)
    +          udaf.dataType // Force it to check input data types.
    +          udaf
    +        } else if (classOf[UDAF].isAssignableFrom(clazz)) {
    +          val udaf = HiveUDAFFunction(
    +            name,
    +            new HiveFunctionWrapper(clazz.getName),
    +            children,
    +            isUDAFBridgeRequired = true)
    +          udaf.dataType  // Force it to check input data types.
    +          udaf
    +        } else if (classOf[GenericUDTF].isAssignableFrom(clazz)) {
    +          val udtf = HiveGenericUDTF(name, new HiveFunctionWrapper(clazz.getName), children)
    +          udtf.elementTypes // Force it to check input data types.
    +          udtf
    +        } else {
    +          throw new AnalysisException(s"No handler for Hive UDF '${clazz.getCanonicalName}'")
    +        }
    +      } catch {
    +        case ae: AnalysisException =>
    +          throw ae
    +        case NonFatal(e) =>
    +          val analysisException =
    +            new AnalysisException(s"No handler for Hive UDF '${clazz.getCanonicalName}': $e")
    +          analysisException.setStackTrace(e.getStackTrace)
    +          throw analysisException
    +      }
    +    }
    +  }
    +
    +  // We have a list of Hive built-in functions that we do not support. So, we will check
    +  // Hive's function registry and lazily load needed functions into our own function registry.
    +  // Those Hive built-in functions are
    +  // assert_true, collect_list, collect_set, compute_stats, context_ngrams, create_union,
    +  // current_user ,elt, ewah_bitmap, ewah_bitmap_and, ewah_bitmap_empty, ewah_bitmap_or, field,
    +  // histogram_numeric, in_file, index, inline, java_method, map_keys, map_values,
    +  // matchpath, ngrams, noop, noopstreaming, noopwithmap, noopwithmapstreaming,
    +  // parse_url, parse_url_tuple, percentile, percentile_approx, posexplode, reflect, reflect2,
    +  // regexp, sentences, stack, std, str_to_map, windowingtablefunction, xpath, xpath_boolean,
    +  // xpath_double, xpath_float, xpath_int, xpath_long, xpath_number,
    +  // xpath_short, and xpath_string.
    +  override def lookupFunction(name: String, children: Seq[Expression]): Expression = {
    +    Try(super.lookupFunction(name, children)) match {
    +      case Success(expr) => expr
    +      case Failure(error) =>
    +        if (functionRegistry.functionExists(name)) {
    +          // If the function actually exists in functionRegistry, it means that there is an
    +          // error when we create the Expression using the given children.
    +          // We need to throw the original exception.
    +          throw error
    +        } else {
    +          // This function is not in functionRegistry, let's try to load it as a Hive's
    +          // built-in function.
    +          val functionName = name.toLowerCase
    +          // TODO: This may not really work for current_user because current_user is not evaluated
    +          // with session info.
    +          // We do not need to use executionHive at here because we only load
    +          // Hive's builtin functions, which do not need current db.
    +          val functionInfo = {
    +            try {
    +              Option(HiveFunctionRegistry.getFunctionInfo(functionName)).getOrElse(
    +                throw new AnalysisException(s"Undefined Hive UDF: $name"))
    +            } catch {
    +              // If HiveFunctionRegistry.getFunctionInfo throws an exception,
    +              // we are failing to load a Hive builtin function, which means that
    +              // the given function is not a Hive builtin function.
    --- End diff --
    
    So you don't allow to load permanent function created with `CREATE FUNCTION` ddl command?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [WIP] [SQL] Handle CreateFunctio...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-205070725
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SQL] Handle CreateFunction/Drop...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58455071
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -428,53 +432,85 @@ class SessionCatalog(
        */
       def dropFunction(name: FunctionIdentifier): Unit = {
         val db = name.database.getOrElse(currentDb)
    +    val qualified = name.copy(database = Some(db)).unquotedString
    +    if (functionRegistry.functionExists(qualified)) {
    +      // If we have loaded this function into FunctionRegistry,
    +      // also drop it from there.
    +      functionRegistry.dropFunction(qualified)
    +    }
         externalCatalog.dropFunction(db, name.funcName)
       }
     
       /**
    -   * Alter a metastore function whose name that matches the one specified in `funcDefinition`.
    -   *
    -   * If no database is specified in `funcDefinition`, assume the function is in the
    -   * current database.
    -   *
    -   * Note: If the underlying implementation does not support altering a certain field,
    -   * this becomes a no-op.
    -   */
    -  def alterFunction(funcDefinition: CatalogFunction): Unit = {
    -    val db = funcDefinition.identifier.database.getOrElse(currentDb)
    -    val newFuncDefinition = funcDefinition.copy(
    -      identifier = FunctionIdentifier(funcDefinition.identifier.funcName, Some(db)))
    -    externalCatalog.alterFunction(db, newFuncDefinition)
    -  }
    -
    -  /**
        * Retrieve the metadata of a metastore function.
        *
        * If a database is specified in `name`, this will return the function in that database.
        * If no database is specified, this will return the function in the current database.
        */
    +  // TODO: have a better name. This method is actually for fetching the metadata of a function.
       def getFunction(name: FunctionIdentifier): CatalogFunction = {
         val db = name.database.getOrElse(currentDb)
         externalCatalog.getFunction(db, name.funcName)
       }
     
    +  /**
    +   * Check if a function is already existing.
    +   *
    +   */
    +  def functionExists(name: FunctionIdentifier): Boolean = {
    +    if (functionRegistry.functionExists(name.unquotedString)) {
    +      // This function exists in the FunctionRegistry.
    +      true
    +    } else {
    +      // Need to check if this function exists in the metastore.
    +      try {
    +        getFunction(name) != null
    +      } catch {
    +        case _: NoSuchFunctionException => false
    +        case _: AnalysisException => false // HiveExternalCatalog wraps all exceptions with it.
    +      }
    +    }
    +  }
     
       // ----------------------------------------------------------------
       // | Methods that interact with temporary and metastore functions |
       // ----------------------------------------------------------------
     
       /**
    +   * Construct a [[FunctionBuilder]] based on the provided class that represents a function.
    +   *
    +   * This performs reflection to decide what type of [[Expression]] to return in the builder.
    +   */
    +  private[sql] def makeFunctionBuilder(name: String, functionClassName: String): FunctionBuilder = {
    +    // TODO: at least support UDAFs here
    +    throw new UnsupportedOperationException("Use sqlContext.udf.register(...) instead.")
    +  }
    +
    +  /**
    +   * Loads resources such as JARs and Files for a function.
    +   */
    +  def loadFunctionResources(resources: Seq[(String, String)]): Unit = {
    +    resources.foreach {
    +      case (resourceType, uri) =>
    --- End diff --
    
    style nit, can you put these on the previous line


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58466809
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala ---
    @@ -0,0 +1,109 @@
    +/*
    + * 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.spark.sql.execution.command
    +
    +import org.apache.spark.sql.{AnalysisException, Row, SQLContext}
    +import org.apache.spark.sql.catalyst.FunctionIdentifier
    +import org.apache.spark.sql.catalyst.catalog.CatalogFunction
    +import org.apache.spark.sql.catalyst.expressions.ExpressionInfo
    +
    +
    +/**
    + * The DDL command that creates a function.
    + * alias: the class name that implements the created function.
    + * resources: Jars, files, or archives which need to be added to the environment when the function
    + *            is referenced for the first time by a session.
    + * isTemp: indicates if it is a temporary function.
    + */
    +// TODO: Use Seq[FunctionResource] instead of Seq[(String, String)] for resources.
    +case class CreateFunction(
    +    databaseName: Option[String],
    +    functionName: String,
    +    className: String,
    +    resources: Seq[(String, String)],
    +    isTemp: Boolean)
    +  extends RunnableCommand {
    +
    +  override def run(sqlContext: SQLContext): Seq[Row] = {
    +    if (isTemp) {
    +      if (databaseName.isDefined) {
    +        throw new AnalysisException(
    +          s"It is not allowed to provide database name when defining a temporary function. " +
    +            s"However, database name ${databaseName.get} is provided.")
    +      }
    +      // We first load resources and then put the builder in the function registry.
    +      // Please note that it is allowed to overwrite an existing temp function.
    +      sqlContext.sessionState.catalog.loadFunctionResources(resources)
    +      val info = new ExpressionInfo(className, functionName)
    +      val builder =
    +        sqlContext.sessionState.catalog.makeFunctionBuilder(functionName, className)
    +      sqlContext.sessionState.catalog.createTempFunction(
    +        functionName, info, builder, ignoreIfExists = false)
    +    } else {
    +      val dbName = databaseName.getOrElse(sqlContext.sessionState.catalog.getCurrentDatabase)
    +      val func = FunctionIdentifier(functionName, Some(dbName))
    +      val catalogFunc = CatalogFunction(func, className, resources)
    +      // We are creating a permanent function. First, we want to check if this function
    +      // has already been created.
    +      // Check if the function to create is already existing. If so, throw exception.
    +      if (sqlContext.sessionState.catalog.functionExists(func)) {
    +        throw new AnalysisException(
    +          s"Function '$functionName' already exists in database '$dbName'.")
    +      }
    +      // This function will be loaded into the FunctionRegistry when a query uses it.
    +      sqlContext.sessionState.catalog.createFunction(catalogFunc)
    +    }
    +    Seq.empty[Row]
    +  }
    +}
    +
    +/**
    + * The DDL command that drops a function.
    + * ifExists: returns an error if the function doesn't exist, unless this is true.
    + * isTemp: indicates if it is a temporary function.
    + */
    +case class DropFunction(
    +    databaseName: Option[String],
    +    functionName: String,
    +    ifExists: Boolean,
    +    isTemp: Boolean)
    +  extends RunnableCommand {
    +
    +  override def run(sqlContext: SQLContext): Seq[Row] = {
    +    if (isTemp) {
    +      if (databaseName.isDefined) {
    +        throw new AnalysisException(
    +          s"It is not allowed to provide database name when dropping a temporary function. " +
    +            s"However, database name ${databaseName.get} is provided.")
    +      }
    +      sqlContext.sessionState.catalog.dropTempFunction(functionName, ifExists)
    +    } else {
    +      // We are dropping a permanent.
    --- End diff --
    
    Oh, I need to finish this comment.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SQL] Handle CreateFunction/DropFunction

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58282048
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/functionResources.scala ---
    @@ -0,0 +1,52 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.catalyst.catalog
    +
    +import org.apache.spark.sql.AnalysisException
    +
    +sealed trait FunctionResourceType
    +
    +object JarResource extends FunctionResourceType
    +
    +object FileResource extends FunctionResourceType
    +
    +object ArchiveResource extends FunctionResourceType
    +
    +object FunctionResourceType {
    +  def fromString(resourceType: String): FunctionResourceType = {
    +    resourceType.toLowerCase match {
    +      case "jar" => JarResource
    +      case "file" => FileResource
    +      case "archive" => ArchiveResource
    +      case other =>
    +        throw new AnalysisException(s"Resource Type '$resourceType' is not supported.")
    +    }
    +  }
    +}
    +
    +case class FunctionResource(resourceType: FunctionResourceType, uri: String)
    +
    +trait FunctionResourceLoader {
    --- End diff --
    
    can you add a comment to explain why we need this, i.e. to limit the number of dependencies in the things we pass around.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-205560436
  
    Thanks @yhuai. This looks pretty good. I left a few suggestions on how the documentation, code readability and placement of functionality can be improved, but these are all relatively minor. Once you address the comments I will go ahead and merge this.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [WIP] [SQL] Handle CreateFunctio...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-204827237
  
    **[Test build #54785 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/54785/consoleFull)** for PR 12117 at commit [`3718e61`](https://github.com/apache/spark/commit/3718e613498cbf9a996f52bc3f215f1051f2ae51).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [WIP] [SQL] Handle CreateFunctio...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58315281
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveSqlParser.scala ---
    @@ -272,10 +272,7 @@ class HiveSqlAstBuilder extends SparkSqlAstBuilder {
           name: String,
           expressions: Seq[Expression],
           ctx: LateralViewContext): Generator = {
    -    val info = Option(FunctionRegistry.getFunctionInfo(name.toLowerCase)).getOrElse {
    -      throw new ParseException(s"Couldn't find Generator function '$name'", ctx)
    -    }
    -    HiveGenericUDTF(name, new HiveFunctionWrapper(info.getFunctionClass.getName), expressions)
    +    UnresolvedGenerator(name, expressions)
    --- End diff --
    
    Since we have a `UnresolvedGenerator` generator now, we can actually remove this entire method and just add it here: https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala#L530-L537


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [WIP] [SQL] Handle CreateFunctio...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58301788
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -491,40 +528,31 @@ class SessionCatalog(
       }
     
       /**
    -   * Rename a function.
    -   *
    -   * If a database is specified in `oldName`, this will rename the function in that database.
    -   * If no database is specified, this will first attempt to rename a temporary function with
    -   * the same name, then, if that does not exist, rename the function in the current database.
    -   *
    -   * This assumes the database specified in `oldName` matches the one specified in `newName`.
    -   */
    -  def renameFunction(oldName: FunctionIdentifier, newName: FunctionIdentifier): Unit = {
    --- End diff --
    
    Users cannot rename a function (no API exposed). So, I just delete it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-205878605
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/54987/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58473612
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala ---
    @@ -0,0 +1,109 @@
    +/*
    + * 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.spark.sql.execution.command
    +
    +import org.apache.spark.sql.{AnalysisException, Row, SQLContext}
    +import org.apache.spark.sql.catalyst.FunctionIdentifier
    +import org.apache.spark.sql.catalyst.catalog.CatalogFunction
    +import org.apache.spark.sql.catalyst.expressions.ExpressionInfo
    +
    +
    +/**
    + * The DDL command that creates a function.
    + * alias: the class name that implements the created function.
    + * resources: Jars, files, or archives which need to be added to the environment when the function
    + *            is referenced for the first time by a session.
    + * isTemp: indicates if it is a temporary function.
    --- End diff --
    
    Done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SQL] Handle CreateFunction/Drop...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58456181
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -524,7 +537,39 @@ class SessionCatalog(
        * Note: This is currently only used for temporary functions.
        */
       def lookupFunction(name: String, children: Seq[Expression]): Expression = {
    -    functionRegistry.lookupFunction(name, children)
    +    // TODO: Right now, the name can be qualified or not qualified.
    +    // It will be better to get a FunctionIdentifier.
    +    // TODO: Right now, we assume that name is not qualified!
    +    val qualifiedName = FunctionIdentifier(name, Some(currentDb)).unquotedString
    +    if (functionRegistry.functionExists(name)) {
    +      // This function has been already loaded into the function registry.
    +      functionRegistry.lookupFunction(name, children)
    +    } else if (functionRegistry.functionExists(qualifiedName)) {
    +      // This function has been already loaded into the function registry.
    +      // Unlike the above block, we find this function by using the qualified name.
    +      functionRegistry.lookupFunction(qualifiedName, children)
    +    } else {
    +      // The function has not been loaded to the function registry, which means
    +      // that the function is a permanent function (if it actually has been registered
    +      // in the metastore). We need to first put the function in FunctionRegistry.
    +      val catalogFunction = try {
    +        externalCatalog.getFunction(currentDb, name)
    +      } catch {
    +        case e: AnalysisException => failFunctionLookup(name)
    +        case e: NoSuchFunctionException => failFunctionLookup(name)
    +      }
    +      loadFunctionResources(catalogFunction.resources)
    +      // Please note that qualifiedName is provided by the user. However,
    +      // catalogFunction.identifier.unquotedString is returned by the underlying
    +      // catalog. So, it is possible that qualifiedName is not exactly the same as
    +      // catalogFunction.identifier.unquotedString (difference is on case-sensitivity).
    +      // At here, we preserve the input from the user.
    +      val info = new ExpressionInfo(catalogFunction.className, qualifiedName)
    +      val builder = makeFunctionBuilder(qualifiedName, catalogFunction.className)
    +      createTempFunction(qualifiedName, info, builder, ignoreIfExists = false)
    --- End diff --
    
    the javadoc says nothing about this. We should add a sentence there to say we cache it in the registry.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58473230
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala ---
    @@ -396,18 +400,16 @@ case class ShowFunctions(db: Option[String], pattern: Option[String]) extends Ru
         schema.toAttributes
       }
     
    -  override def run(sqlContext: SQLContext): Seq[Row] = pattern match {
    -    case Some(p) =>
    -      try {
    -        val regex = java.util.regex.Pattern.compile(p)
    -        sqlContext.sessionState.functionRegistry.listFunction()
    -          .filter(regex.matcher(_).matches()).map(Row(_))
    -      } catch {
    -        // probably will failed in the regex that user provided, then returns empty row.
    -        case _: Throwable => Seq.empty[Row]
    -      }
    -    case None =>
    -      sqlContext.sessionState.functionRegistry.listFunction().map(Row(_))
    +  override def run(sqlContext: SQLContext): Seq[Row] = {
    +    val dbName = db.getOrElse(sqlContext.sessionState.catalog.getCurrentDatabase)
    +    val functionNames =
    +      sqlContext.sessionState.catalog
    +        .listFunctions(dbName, pattern.getOrElse(".*"))
    --- End diff --
    
    oh, I was confused last time when I was trying to figure out the syntax.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-205625867
  
    **[Test build #54945 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/54945/consoleFull)** for PR 12117 at commit [`8a41f6d`](https://github.com/apache/spark/commit/8a41f6df56cede48107e81f50df2111ddaf5e18d).
     * This patch **fails MiMa tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [WIP] [SQL] Handle CreateFunctio...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-205057833
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/54810/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SQL] Handle CreateFunction/Drop...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58448559
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala ---
    @@ -341,9 +343,14 @@ object FunctionRegistry {
       /** See usage above. */
       def expression[T <: Expression](name: String)
           (implicit tag: ClassTag[T]): (String, (ExpressionInfo, FunctionBuilder)) = {
    +    expression(name, tag.runtimeClass.asInstanceOf[Class[T]])
    +  }
     
    +  def expression[T <: Expression](
    +      name: String,
    +      clazz: Class[T]): (String, (ExpressionInfo, FunctionBuilder)) = {
    --- End diff --
    
    Oh, I thought this one is also used...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [WIP] [SQL] Handle CreateFunctio...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-204654042
  
    **[Test build #54764 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/54764/consoleFull)** for PR 12117 at commit [`aa41b20`](https://github.com/apache/spark/commit/aa41b20eece2f77de9815c66d35b0f2ef0464b88).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [WIP] [SQL] Handle CreateFunctio...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58317331
  
    --- Diff: sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala ---
    @@ -363,7 +363,11 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
         // Index commands are not supported
         "drop_index",
         "drop_index_removes_partition_dirs",
    -    "alter_index"
    +    "alter_index",
    +
    +    // We do not support macro for now.
    +    // TODO: throw an exception if a user is trying to create a macro.
    --- End diff --
    
    This has been done in https://github.com/apache/spark/commit/c238cd07448f94bbceb661daad90b6a6d597a846


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-205878276
  
    **[Test build #54987 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/54987/consoleFull)** for PR 12117 at commit [`3938766`](https://github.com/apache/spark/commit/39387666fe24a2e65a267eabc15c7816c8738449).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58456861
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala ---
    @@ -208,6 +208,22 @@ class SQLContext private[sql](
         sparkContext.addJar(path)
       }
     
    +  /** Returns a [[FunctionResourceLoader]] that can be used in SessionCatalog. */
    +  protected[sql] def functionResourceLoader: FunctionResourceLoader = {
    --- End diff --
    
    also does it belong to `SessionState`, in which case it should be a lazy val?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SQL] Handle CreateFunction/Drop...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58418254
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala ---
    @@ -112,4 +124,122 @@ class HiveSessionCatalog(
         metastoreCatalog.cachedDataSourceTables.getIfPresent(key)
       }
     
    +  override def makeFunctionBuilder(funcName: String, className: String): FunctionBuilder = {
    +    makeFunctionBuilder(funcName, Utils.classForName(className))
    +  }
    +
    +  /**
    +   * Construct a [[FunctionBuilder]] based on the provided class that represents a function.
    +   */
    +  private def makeFunctionBuilder(name: String, clazz: Class[_]): FunctionBuilder = {
    +    // When we instantiate hive UDF wrapper class, we may throw exception if the input
    +    // expressions don't satisfy the hive UDF, such as type mismatch, input number
    +    // mismatch, etc. Here we catch the exception and throw AnalysisException instead.
    +    (children: Seq[Expression]) => {
    +      try {
    +        if (classOf[UDF].isAssignableFrom(clazz)) {
    +          val udf = HiveSimpleUDF(name, new HiveFunctionWrapper(clazz.getName), children)
    +          udf.dataType // Force it to check input data types.
    +          udf
    +        } else if (classOf[GenericUDF].isAssignableFrom(clazz)) {
    +          val udf = HiveGenericUDF(name, new HiveFunctionWrapper(clazz.getName), children)
    +          udf.dataType // Force it to check input data types.
    +          udf
    +        } else if (classOf[AbstractGenericUDAFResolver].isAssignableFrom(clazz)) {
    +          val udaf = HiveUDAFFunction(name, new HiveFunctionWrapper(clazz.getName), children)
    +          udaf.dataType // Force it to check input data types.
    +          udaf
    +        } else if (classOf[UDAF].isAssignableFrom(clazz)) {
    +          val udaf = HiveUDAFFunction(
    +            name,
    +            new HiveFunctionWrapper(clazz.getName),
    +            children,
    +            isUDAFBridgeRequired = true)
    +          udaf.dataType  // Force it to check input data types.
    +          udaf
    +        } else if (classOf[GenericUDTF].isAssignableFrom(clazz)) {
    +          val udtf = HiveGenericUDTF(name, new HiveFunctionWrapper(clazz.getName), children)
    +          udtf.elementTypes // Force it to check input data types.
    +          udtf
    +        } else {
    +          throw new AnalysisException(s"No handler for Hive UDF '${clazz.getCanonicalName}'")
    +        }
    +      } catch {
    +        case ae: AnalysisException =>
    +          throw ae
    +        case NonFatal(e) =>
    +          val analysisException =
    +            new AnalysisException(s"No handler for Hive UDF '${clazz.getCanonicalName}': $e")
    +          analysisException.setStackTrace(e.getStackTrace)
    +          throw analysisException
    +      }
    +    }
    +  }
    +
    +  // We have a list of Hive built-in functions that we do not support. So, we will check
    +  // Hive's function registry and lazily load needed functions into our own function registry.
    +  // Those Hive built-in functions are
    +  // assert_true, collect_list, collect_set, compute_stats, context_ngrams, create_union,
    +  // current_user ,elt, ewah_bitmap, ewah_bitmap_and, ewah_bitmap_empty, ewah_bitmap_or, field,
    +  // histogram_numeric, in_file, index, inline, java_method, map_keys, map_values,
    +  // matchpath, ngrams, noop, noopstreaming, noopwithmap, noopwithmapstreaming,
    +  // parse_url, parse_url_tuple, percentile, percentile_approx, posexplode, reflect, reflect2,
    +  // regexp, sentences, stack, std, str_to_map, windowingtablefunction, xpath, xpath_boolean,
    +  // xpath_double, xpath_float, xpath_int, xpath_long, xpath_number,
    +  // xpath_short, and xpath_string.
    +  override def lookupFunction(name: String, children: Seq[Expression]): Expression = {
    +    Try(super.lookupFunction(name, children)) match {
    --- End diff --
    
    I'd like to change the name provided to lookupFunction to FunctionIdentifier. Then, it will be easier to do
    ```
    if (super.functionExists(name)) {
      super.lookupFunction(name, children)
    } else {
      // This function is a Hive builtin function. 
      ...
    }
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58472516
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -524,7 +537,39 @@ class SessionCatalog(
        * Note: This is currently only used for temporary functions.
        */
       def lookupFunction(name: String, children: Seq[Expression]): Expression = {
    -    functionRegistry.lookupFunction(name, children)
    +    // TODO: Right now, the name can be qualified or not qualified.
    +    // It will be better to get a FunctionIdentifier.
    --- End diff --
    
    Thanks!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58475308
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala ---
    @@ -0,0 +1,109 @@
    +/*
    + * 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.spark.sql.execution.command
    +
    +import org.apache.spark.sql.{AnalysisException, Row, SQLContext}
    +import org.apache.spark.sql.catalyst.FunctionIdentifier
    +import org.apache.spark.sql.catalyst.catalog.CatalogFunction
    +import org.apache.spark.sql.catalyst.expressions.ExpressionInfo
    +
    +
    +/**
    + * The DDL command that creates a function.
    + * alias: the class name that implements the created function.
    + * resources: Jars, files, or archives which need to be added to the environment when the function
    + *            is referenced for the first time by a session.
    + * isTemp: indicates if it is a temporary function.
    + */
    +// TODO: Use Seq[FunctionResource] instead of Seq[(String, String)] for resources.
    +case class CreateFunction(
    +    databaseName: Option[String],
    +    functionName: String,
    +    className: String,
    +    resources: Seq[(String, String)],
    +    isTemp: Boolean)
    +  extends RunnableCommand {
    +
    +  override def run(sqlContext: SQLContext): Seq[Row] = {
    +    if (isTemp) {
    +      if (databaseName.isDefined) {
    +        throw new AnalysisException(
    +          s"It is not allowed to provide database name when defining a temporary function. " +
    +            s"However, database name ${databaseName.get} is provided.")
    +      }
    +      // We first load resources and then put the builder in the function registry.
    +      // Please note that it is allowed to overwrite an existing temp function.
    +      sqlContext.sessionState.catalog.loadFunctionResources(resources)
    +      val info = new ExpressionInfo(className, functionName)
    +      val builder =
    +        sqlContext.sessionState.catalog.makeFunctionBuilder(functionName, className)
    +      sqlContext.sessionState.catalog.createTempFunction(
    +        functionName, info, builder, ignoreIfExists = false)
    +    } else {
    +      val dbName = databaseName.getOrElse(sqlContext.sessionState.catalog.getCurrentDatabase)
    +      val func = FunctionIdentifier(functionName, Some(dbName))
    +      val catalogFunc = CatalogFunction(func, className, resources)
    +      // We are creating a permanent function. First, we want to check if this function
    +      // has already been created.
    +      // Check if the function to create is already existing. If so, throw exception.
    +      if (sqlContext.sessionState.catalog.functionExists(func)) {
    +        throw new AnalysisException(
    +          s"Function '$functionName' already exists in database '$dbName'.")
    +      }
    +      // This function will be loaded into the FunctionRegistry when a query uses it.
    +      sqlContext.sessionState.catalog.createFunction(catalogFunc)
    +    }
    +    Seq.empty[Row]
    +  }
    +}
    +
    +/**
    + * The DDL command that drops a function.
    + * ifExists: returns an error if the function doesn't exist, unless this is true.
    + * isTemp: indicates if it is a temporary function.
    + */
    +case class DropFunction(
    +    databaseName: Option[String],
    +    functionName: String,
    +    ifExists: Boolean,
    +    isTemp: Boolean)
    +  extends RunnableCommand {
    +
    +  override def run(sqlContext: SQLContext): Seq[Row] = {
    +    if (isTemp) {
    +      if (databaseName.isDefined) {
    +        throw new AnalysisException(
    +          s"It is not allowed to provide database name when dropping a temporary function. " +
    +            s"However, database name ${databaseName.get} is provided.")
    +      }
    +      sqlContext.sessionState.catalog.dropTempFunction(functionName, ifExists)
    +    } else {
    +      // We are dropping a permanent.
    +      val dbName = databaseName.getOrElse(sqlContext.sessionState.catalog.getCurrentDatabase)
    +      val func = FunctionIdentifier(functionName, Some(dbName))
    +      if (!ifExists) {
    +        if (!sqlContext.sessionState.catalog.functionExists(func)) {
    --- End diff --
    
    Done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SQL] Handle CreateFunction/Drop...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-205493063
  
    **[Test build #54877 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/54877/consoleFull)** for PR 12117 at commit [`1f77973`](https://github.com/apache/spark/commit/1f779736c3c7f0b778fab4739fb7a1b4fd909d6a).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58456761
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala ---
    @@ -208,6 +208,22 @@ class SQLContext private[sql](
         sparkContext.addJar(path)
       }
     
    +  /** Returns a [[FunctionResourceLoader]] that can be used in SessionCatalog. */
    +  protected[sql] def functionResourceLoader: FunctionResourceLoader = {
    --- End diff --
    
    does this belong to `SessionState`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SQL] Handle CreateFunction/DropFunction

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58282309
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala ---
    @@ -112,4 +124,121 @@ class HiveSessionCatalog(
         metastoreCatalog.cachedDataSourceTables.getIfPresent(key)
       }
     
    +  override def makeFunctionBuilder(funcName: String, className: String): FunctionBuilder = {
    +    makeFunctionBuilder(funcName, Utils.classForName(className))
    +  }
    +
    +  /**
    +   * Construct a [[FunctionBuilder]] based on the provided class that represents a function.
    +   */
    +  private def makeFunctionBuilder(name: String, clazz: Class[_]): FunctionBuilder = {
    +    // When we instantiate hive UDF wrapper class, we may throw exception if the input
    +    // expressions don't satisfy the hive UDF, such as type mismatch, input number
    +    // mismatch, etc. Here we catch the exception and throw AnalysisException instead.
    +    (children: Seq[Expression]) => {
    +      try {
    +        if (classOf[UDF].isAssignableFrom(clazz)) {
    +          val udf = HiveSimpleUDF(name, new HiveFunctionWrapper(clazz.getName), children)
    +          udf.dataType // Force it to check input data types.
    +          udf
    +        } else if (classOf[GenericUDF].isAssignableFrom(clazz)) {
    +          val udf = HiveGenericUDF(name, new HiveFunctionWrapper(clazz.getName), children)
    +          udf.dataType // Force it to check input data types.
    +          udf
    +        } else if (classOf[AbstractGenericUDAFResolver].isAssignableFrom(clazz)) {
    +          val udaf = HiveUDAFFunction(name, new HiveFunctionWrapper(clazz.getName), children)
    +          udaf.dataType // Force it to check input data types.
    +          udaf
    +        } else if (classOf[UDAF].isAssignableFrom(clazz)) {
    +          val udaf = HiveUDAFFunction(
    +            name,
    +            new HiveFunctionWrapper(clazz.getName),
    +            children,
    +            isUDAFBridgeRequired = true)
    +          udaf.dataType  // Force it to check input data types.
    +          udaf
    +        } else if (classOf[GenericUDTF].isAssignableFrom(clazz)) {
    +          val udtf = HiveGenericUDTF(name, new HiveFunctionWrapper(clazz.getName), children)
    +          udtf.elementTypes // Force it to check input data types.
    +          udtf
    +        } else {
    +          throw new AnalysisException(s"No handler for Hive UDF '${clazz.getCanonicalName}'")
    +        }
    +      } catch {
    +        case ae: AnalysisException =>
    +          throw ae
    +        case NonFatal(e) =>
    +          val analysisException =
    +            new AnalysisException(s"No handler for Hive UDF '${clazz.getCanonicalName}': $e")
    +          analysisException.setStackTrace(e.getStackTrace)
    +          throw analysisException
    +      }
    +    }
    +  }
    +
    +  // We have a list of Hive built-in functions that we do not support. So, we will check
    +  // Hive's function registry and lazily load needed functions into our own function registry.
    +  // Those Hive built-in functions are
    +  // assert_true, collect_list, collect_set, compute_stats, context_ngrams, create_union,
    +  // current_user ,elt, ewah_bitmap, ewah_bitmap_and, ewah_bitmap_empty, ewah_bitmap_or, field,
    +  // histogram_numeric, in_file, index, inline, java_method, map_keys, map_values,
    +  // matchpath, ngrams, noop, noopstreaming, noopwithmap, noopwithmapstreaming,
    +  // parse_url, parse_url_tuple, percentile, percentile_approx, posexplode, reflect, reflect2,
    +  // regexp, sentences, stack, std, str_to_map, windowingtablefunction, xpath, xpath_boolean,
    +  // xpath_double, xpath_float, xpath_int, xpath_long, xpath_number,
    +  // xpath_short, and xpath_string.
    +  override def lookupFunction(name: String, children: Seq[Expression]): Expression = {
    +    Try(super.lookupFunction(name, children)) match {
    +      case Success(expr) => expr
    +      case Failure(error) =>
    +        if (functionRegistry.functionExists(name)) {
    +          // If the function actually exists in functionRegistry, it means that there is an
    +          // error when we create the Expression using the given children.
    +          // We need to throw the original exception.
    +          throw error
    --- End diff --
    
    This pattern is kind of weird. We don't need to do a `Try` here and then turn around and check the `functionRegistry`. Just check the registry directly, then, if it does not contain `name`, then ask Hive assuming it's a built-in function.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SQL] Handle CreateFunction/Drop...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58447785
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala ---
    @@ -49,7 +49,6 @@ object SimpleAnalyzer
     class SimpleAnalyzer(functionRegistry: FunctionRegistry, conf: CatalystConf)
       extends Analyzer(
         new SessionCatalog(new InMemoryCatalog, functionRegistry, conf),
    -    functionRegistry,
         conf)
    --- End diff --
    
    this prob fits in 1 line now


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-205654370
  
    LGTM now.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [WIP] [SQL] Handle CreateFunctio...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-204846056
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [WIP] [SQL] Handle CreateFunctio...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58295218
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -69,6 +73,7 @@ class SessionCatalog(
       /**
        * Format table name, taking into account case sensitivity.
        */
    +  // TODO: Should we use it for temp function name?
    --- End diff --
    
    Think about it again. I feel it is not really useful to set case sensitive by default for function names. I will remove this TODO. Function names are case sensitive (i.e. the current behavior. ). 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SQL] Handle CreateFunction/Drop...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58455116
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -428,53 +432,85 @@ class SessionCatalog(
        */
       def dropFunction(name: FunctionIdentifier): Unit = {
         val db = name.database.getOrElse(currentDb)
    +    val qualified = name.copy(database = Some(db)).unquotedString
    +    if (functionRegistry.functionExists(qualified)) {
    +      // If we have loaded this function into FunctionRegistry,
    +      // also drop it from there.
    +      functionRegistry.dropFunction(qualified)
    +    }
         externalCatalog.dropFunction(db, name.funcName)
       }
     
       /**
    -   * Alter a metastore function whose name that matches the one specified in `funcDefinition`.
    -   *
    -   * If no database is specified in `funcDefinition`, assume the function is in the
    -   * current database.
    -   *
    -   * Note: If the underlying implementation does not support altering a certain field,
    -   * this becomes a no-op.
    -   */
    -  def alterFunction(funcDefinition: CatalogFunction): Unit = {
    -    val db = funcDefinition.identifier.database.getOrElse(currentDb)
    -    val newFuncDefinition = funcDefinition.copy(
    -      identifier = FunctionIdentifier(funcDefinition.identifier.funcName, Some(db)))
    -    externalCatalog.alterFunction(db, newFuncDefinition)
    -  }
    -
    -  /**
        * Retrieve the metadata of a metastore function.
        *
        * If a database is specified in `name`, this will return the function in that database.
        * If no database is specified, this will return the function in the current database.
        */
    +  // TODO: have a better name. This method is actually for fetching the metadata of a function.
       def getFunction(name: FunctionIdentifier): CatalogFunction = {
         val db = name.database.getOrElse(currentDb)
         externalCatalog.getFunction(db, name.funcName)
       }
     
    +  /**
    +   * Check if a function is already existing.
    +   *
    +   */
    +  def functionExists(name: FunctionIdentifier): Boolean = {
    +    if (functionRegistry.functionExists(name.unquotedString)) {
    +      // This function exists in the FunctionRegistry.
    +      true
    +    } else {
    +      // Need to check if this function exists in the metastore.
    +      try {
    +        getFunction(name) != null
    +      } catch {
    +        case _: NoSuchFunctionException => false
    +        case _: AnalysisException => false // HiveExternalCatalog wraps all exceptions with it.
    +      }
    +    }
    +  }
     
       // ----------------------------------------------------------------
       // | Methods that interact with temporary and metastore functions |
       // ----------------------------------------------------------------
     
       /**
    +   * Construct a [[FunctionBuilder]] based on the provided class that represents a function.
    +   *
    +   * This performs reflection to decide what type of [[Expression]] to return in the builder.
    +   */
    +  private[sql] def makeFunctionBuilder(name: String, functionClassName: String): FunctionBuilder = {
    +    // TODO: at least support UDAFs here
    +    throw new UnsupportedOperationException("Use sqlContext.udf.register(...) instead.")
    +  }
    +
    +  /**
    +   * Loads resources such as JARs and Files for a function.
    +   */
    +  def loadFunctionResources(resources: Seq[(String, String)]): Unit = {
    --- End diff --
    
    need to document in the javadoc what the keys and values are.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58472415
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -428,53 +432,85 @@ class SessionCatalog(
        */
       def dropFunction(name: FunctionIdentifier): Unit = {
         val db = name.database.getOrElse(currentDb)
    +    val qualified = name.copy(database = Some(db)).unquotedString
    +    if (functionRegistry.functionExists(qualified)) {
    +      // If we have loaded this function into FunctionRegistry,
    +      // also drop it from there.
    +      functionRegistry.dropFunction(qualified)
    +    }
         externalCatalog.dropFunction(db, name.funcName)
       }
     
       /**
    -   * Alter a metastore function whose name that matches the one specified in `funcDefinition`.
    -   *
    -   * If no database is specified in `funcDefinition`, assume the function is in the
    -   * current database.
    -   *
    -   * Note: If the underlying implementation does not support altering a certain field,
    -   * this becomes a no-op.
    -   */
    -  def alterFunction(funcDefinition: CatalogFunction): Unit = {
    -    val db = funcDefinition.identifier.database.getOrElse(currentDb)
    -    val newFuncDefinition = funcDefinition.copy(
    -      identifier = FunctionIdentifier(funcDefinition.identifier.funcName, Some(db)))
    -    externalCatalog.alterFunction(db, newFuncDefinition)
    -  }
    -
    -  /**
        * Retrieve the metadata of a metastore function.
        *
        * If a database is specified in `name`, this will return the function in that database.
        * If no database is specified, this will return the function in the current database.
        */
    +  // TODO: have a better name. This method is actually for fetching the metadata of a function.
       def getFunction(name: FunctionIdentifier): CatalogFunction = {
         val db = name.database.getOrElse(currentDb)
         externalCatalog.getFunction(db, name.funcName)
       }
     
    +  /**
    +   * Check if a function is already existing.
    +   *
    --- End diff --
    
    Done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [WIP] [SQL] Handle CreateFunctio...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58301126
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala ---
    @@ -133,6 +133,30 @@ object UnresolvedAttribute {
       }
     }
     
    +case class UnresolvedGenerator(
    --- End diff --
    
    Done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SQL] Handle CreateFunction/Drop...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58329961
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala ---
    @@ -175,13 +175,54 @@ case class DescribeDatabase(
       }
     }
     
    +/**
    + * The DDL command that creates a function.
    + * alias: the class name that implements the created function.
    + * resources: Jars, files, or archives which need to be added to the environment when the function
    + *            is referenced for the first time by a session.
    + * isTemp: indicates if it is a temporary function.
    + */
    +// TODO: Use Seq[FunctionResource] instead of Seq[(String, String)] for resources.
     case class CreateFunction(
    --- End diff --
    
    can we move the function related commands out, i.e.
    
    functions.scala
    
    this file is going to become thousands of lines long.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SQL] Handle CreateFunction/DropFunction

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-204608546
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/54726/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [WIP] [SQL] Handle CreateFunctio...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58287031
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala ---
    @@ -112,4 +124,121 @@ class HiveSessionCatalog(
         metastoreCatalog.cachedDataSourceTables.getIfPresent(key)
       }
     
    +  override def makeFunctionBuilder(funcName: String, className: String): FunctionBuilder = {
    +    makeFunctionBuilder(funcName, Utils.classForName(className))
    +  }
    +
    +  /**
    +   * Construct a [[FunctionBuilder]] based on the provided class that represents a function.
    +   */
    +  private def makeFunctionBuilder(name: String, clazz: Class[_]): FunctionBuilder = {
    +    // When we instantiate hive UDF wrapper class, we may throw exception if the input
    +    // expressions don't satisfy the hive UDF, such as type mismatch, input number
    +    // mismatch, etc. Here we catch the exception and throw AnalysisException instead.
    +    (children: Seq[Expression]) => {
    +      try {
    +        if (classOf[UDF].isAssignableFrom(clazz)) {
    +          val udf = HiveSimpleUDF(name, new HiveFunctionWrapper(clazz.getName), children)
    +          udf.dataType // Force it to check input data types.
    +          udf
    +        } else if (classOf[GenericUDF].isAssignableFrom(clazz)) {
    +          val udf = HiveGenericUDF(name, new HiveFunctionWrapper(clazz.getName), children)
    +          udf.dataType // Force it to check input data types.
    +          udf
    +        } else if (classOf[AbstractGenericUDAFResolver].isAssignableFrom(clazz)) {
    +          val udaf = HiveUDAFFunction(name, new HiveFunctionWrapper(clazz.getName), children)
    +          udaf.dataType // Force it to check input data types.
    +          udaf
    +        } else if (classOf[UDAF].isAssignableFrom(clazz)) {
    +          val udaf = HiveUDAFFunction(
    +            name,
    +            new HiveFunctionWrapper(clazz.getName),
    +            children,
    +            isUDAFBridgeRequired = true)
    +          udaf.dataType  // Force it to check input data types.
    +          udaf
    +        } else if (classOf[GenericUDTF].isAssignableFrom(clazz)) {
    +          val udtf = HiveGenericUDTF(name, new HiveFunctionWrapper(clazz.getName), children)
    +          udtf.elementTypes // Force it to check input data types.
    +          udtf
    +        } else {
    +          throw new AnalysisException(s"No handler for Hive UDF '${clazz.getCanonicalName}'")
    +        }
    +      } catch {
    +        case ae: AnalysisException =>
    +          throw ae
    +        case NonFatal(e) =>
    +          val analysisException =
    +            new AnalysisException(s"No handler for Hive UDF '${clazz.getCanonicalName}': $e")
    +          analysisException.setStackTrace(e.getStackTrace)
    +          throw analysisException
    +      }
    +    }
    +  }
    +
    +  // We have a list of Hive built-in functions that we do not support. So, we will check
    +  // Hive's function registry and lazily load needed functions into our own function registry.
    +  // Those Hive built-in functions are
    +  // assert_true, collect_list, collect_set, compute_stats, context_ngrams, create_union,
    +  // current_user ,elt, ewah_bitmap, ewah_bitmap_and, ewah_bitmap_empty, ewah_bitmap_or, field,
    +  // histogram_numeric, in_file, index, inline, java_method, map_keys, map_values,
    +  // matchpath, ngrams, noop, noopstreaming, noopwithmap, noopwithmapstreaming,
    +  // parse_url, parse_url_tuple, percentile, percentile_approx, posexplode, reflect, reflect2,
    +  // regexp, sentences, stack, std, str_to_map, windowingtablefunction, xpath, xpath_boolean,
    +  // xpath_double, xpath_float, xpath_int, xpath_long, xpath_number,
    +  // xpath_short, and xpath_string.
    +  override def lookupFunction(name: String, children: Seq[Expression]): Expression = {
    +    Try(super.lookupFunction(name, children)) match {
    +      case Success(expr) => expr
    +      case Failure(error) =>
    +        if (functionRegistry.functionExists(name)) {
    +          // If the function actually exists in functionRegistry, it means that there is an
    +          // error when we create the Expression using the given children.
    +          // We need to throw the original exception.
    +          throw error
    +        } else {
    +          // This function is not in functionRegistry, let's try to load it as a Hive's
    +          // built-in function.
    +          val functionName = name.toLowerCase
    +          // TODO: This may not really work for current_user because current_user is not evaluated
    +          // with session info.
    +          // We do not need to use executionHive at here because we only load
    +          // Hive's builtin functions, which do not need current db.
    +          val functionInfo = {
    +            try {
    +              Option(HiveFunctionRegistry.getFunctionInfo(functionName)).getOrElse(
    +                throw new AnalysisException(s"Undefined Hive UDF: $name"))
    +            } catch {
    +              // If HiveFunctionRegistry.getFunctionInfo throws an exception,
    +              // we are failing to load a Hive builtin function, which means that
    +              // the given function is not a Hive builtin function.
    --- End diff --
    
    nvm, I saw you move it to `SessionCatalog`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SQL] Handle CreateFunction/Drop...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58455184
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -428,53 +432,85 @@ class SessionCatalog(
        */
       def dropFunction(name: FunctionIdentifier): Unit = {
         val db = name.database.getOrElse(currentDb)
    +    val qualified = name.copy(database = Some(db)).unquotedString
    +    if (functionRegistry.functionExists(qualified)) {
    +      // If we have loaded this function into FunctionRegistry,
    +      // also drop it from there.
    +      functionRegistry.dropFunction(qualified)
    --- End diff --
    
    With this PR, Function registry is not just for temp functions. Builders of permanent functions will be lazily loaded.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SQL] Handle CreateFunction/DropFunction

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-204614062
  
    **[Test build #54731 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/54731/consoleFull)** for PR 12117 at commit [`979b03e`](https://github.com/apache/spark/commit/979b03e16c4853d255401dfb2b104bfe10cdcb64).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58478146
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/UDFSuite.scala ---
    @@ -32,4 +68,122 @@ class UDFSuite extends QueryTest with TestHiveSingleton {
         assert(hiveContext.sql("SELECT RANDOm1() FROM src LIMIT 1").head().getDouble(0) >= 0.0)
         assert(hiveContext.sql("SELECT strlenscala('test', 1) FROM src LIMIT 1").head().getInt(0) === 5)
       }
    +
    +  test("temporary function: create and drop") {
    +    withUserDefinedFunction(functionName -> true) {
    +      intercept[AnalysisException] {
    +        sql(s"CREATE TEMPORARY FUNCTION default.$functionName AS '$functionClass'")
    +      }
    +      sql(s"CREATE TEMPORARY FUNCTION $functionName AS '$functionClass'")
    +      checkAnswer(
    +        sql(s"SELECT myupper(value) from $testTableName"),
    --- End diff --
    
    I saw myUpper defined at the top of test. But we still directly write myupper in many places.
    
    Is it better to also create a lowercase one ?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SQL] Handle CreateFunction/DropFunction

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58282739
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -453,28 +464,80 @@ class SessionCatalog(
        * If a database is specified in `name`, this will return the function in that database.
        * If no database is specified, this will return the function in the current database.
        */
    +  // TODO: have a better name. This method is actually for fetching the metadata of a function.
    --- End diff --
    
    I'd just call it `getFunctionMetadata`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SQL] Handle CreateFunction/DropFunction

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-204588294
  
    **[Test build #54726 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/54726/consoleFull)** for PR 12117 at commit [`04a5926`](https://github.com/apache/spark/commit/04a59263664df573565e287381c8fd1dcad59197).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58483084
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/UDFSuite.scala ---
    @@ -32,4 +68,122 @@ class UDFSuite extends QueryTest with TestHiveSingleton {
         assert(hiveContext.sql("SELECT RANDOm1() FROM src LIMIT 1").head().getDouble(0) >= 0.0)
         assert(hiveContext.sql("SELECT strlenscala('test', 1) FROM src LIMIT 1").head().getInt(0) === 5)
       }
    +
    +  test("temporary function: create and drop") {
    +    withUserDefinedFunction(functionName -> true) {
    +      intercept[AnalysisException] {
    +        sql(s"CREATE TEMPORARY FUNCTION default.$functionName AS '$functionClass'")
    +      }
    +      sql(s"CREATE TEMPORARY FUNCTION $functionName AS '$functionClass'")
    +      checkAnswer(
    +        sql(s"SELECT myupper(value) from $testTableName"),
    --- End diff --
    
    Updated


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SQL] Handle CreateFunction/DropFunction

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-204583813
  
    **[Test build #54723 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/54723/consoleFull)** for PR 12117 at commit [`e0570cd`](https://github.com/apache/spark/commit/e0570cdacda5b3cc81efa3b317e75882186f2d97).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SQL] Handle CreateFunction/DropFunction

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58282358
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala ---
    @@ -112,4 +124,121 @@ class HiveSessionCatalog(
         metastoreCatalog.cachedDataSourceTables.getIfPresent(key)
       }
     
    +  override def makeFunctionBuilder(funcName: String, className: String): FunctionBuilder = {
    +    makeFunctionBuilder(funcName, Utils.classForName(className))
    +  }
    +
    +  /**
    +   * Construct a [[FunctionBuilder]] based on the provided class that represents a function.
    +   */
    +  private def makeFunctionBuilder(name: String, clazz: Class[_]): FunctionBuilder = {
    +    // When we instantiate hive UDF wrapper class, we may throw exception if the input
    +    // expressions don't satisfy the hive UDF, such as type mismatch, input number
    +    // mismatch, etc. Here we catch the exception and throw AnalysisException instead.
    +    (children: Seq[Expression]) => {
    +      try {
    +        if (classOf[UDF].isAssignableFrom(clazz)) {
    +          val udf = HiveSimpleUDF(name, new HiveFunctionWrapper(clazz.getName), children)
    +          udf.dataType // Force it to check input data types.
    +          udf
    +        } else if (classOf[GenericUDF].isAssignableFrom(clazz)) {
    +          val udf = HiveGenericUDF(name, new HiveFunctionWrapper(clazz.getName), children)
    +          udf.dataType // Force it to check input data types.
    +          udf
    +        } else if (classOf[AbstractGenericUDAFResolver].isAssignableFrom(clazz)) {
    +          val udaf = HiveUDAFFunction(name, new HiveFunctionWrapper(clazz.getName), children)
    +          udaf.dataType // Force it to check input data types.
    +          udaf
    +        } else if (classOf[UDAF].isAssignableFrom(clazz)) {
    +          val udaf = HiveUDAFFunction(
    +            name,
    +            new HiveFunctionWrapper(clazz.getName),
    +            children,
    +            isUDAFBridgeRequired = true)
    +          udaf.dataType  // Force it to check input data types.
    +          udaf
    +        } else if (classOf[GenericUDTF].isAssignableFrom(clazz)) {
    +          val udtf = HiveGenericUDTF(name, new HiveFunctionWrapper(clazz.getName), children)
    +          udtf.elementTypes // Force it to check input data types.
    +          udtf
    +        } else {
    +          throw new AnalysisException(s"No handler for Hive UDF '${clazz.getCanonicalName}'")
    +        }
    +      } catch {
    +        case ae: AnalysisException =>
    +          throw ae
    +        case NonFatal(e) =>
    +          val analysisException =
    +            new AnalysisException(s"No handler for Hive UDF '${clazz.getCanonicalName}': $e")
    +          analysisException.setStackTrace(e.getStackTrace)
    +          throw analysisException
    +      }
    +    }
    +  }
    +
    +  // We have a list of Hive built-in functions that we do not support. So, we will check
    +  // Hive's function registry and lazily load needed functions into our own function registry.
    +  // Those Hive built-in functions are
    +  // assert_true, collect_list, collect_set, compute_stats, context_ngrams, create_union,
    +  // current_user ,elt, ewah_bitmap, ewah_bitmap_and, ewah_bitmap_empty, ewah_bitmap_or, field,
    +  // histogram_numeric, in_file, index, inline, java_method, map_keys, map_values,
    +  // matchpath, ngrams, noop, noopstreaming, noopwithmap, noopwithmapstreaming,
    +  // parse_url, parse_url_tuple, percentile, percentile_approx, posexplode, reflect, reflect2,
    +  // regexp, sentences, stack, std, str_to_map, windowingtablefunction, xpath, xpath_boolean,
    +  // xpath_double, xpath_float, xpath_int, xpath_long, xpath_number,
    +  // xpath_short, and xpath_string.
    +  override def lookupFunction(name: String, children: Seq[Expression]): Expression = {
    +    Try(super.lookupFunction(name, children)) match {
    +      case Success(expr) => expr
    +      case Failure(error) =>
    +        if (functionRegistry.functionExists(name)) {
    +          // If the function actually exists in functionRegistry, it means that there is an
    +          // error when we create the Expression using the given children.
    +          // We need to throw the original exception.
    +          throw error
    +        } else {
    +          // This function is not in functionRegistry, let's try to load it as a Hive's
    +          // built-in function.
    +          val functionName = name.toLowerCase
    +          // TODO: This may not really work for current_user because current_user is not evaluated
    +          // with session info.
    +          // We do not need to use executionHive at here because we only load
    +          // Hive's builtin functions, which do not need current db.
    +          val functionInfo = {
    +            try {
    +              Option(HiveFunctionRegistry.getFunctionInfo(functionName)).getOrElse(
    +                throw new AnalysisException(s"Undefined Hive UDF: $name"))
    +            } catch {
    +              // If HiveFunctionRegistry.getFunctionInfo throws an exception,
    +              // we are failing to load a Hive builtin function, which means that
    +              // the given function is not a Hive builtin function.
    +              case NonFatal(e) => throw new AnalysisException(s"Undefined Hive UDF: $name")
    +            }
    +          }
    +          val className = functionInfo.getFunctionClass.getName
    +          val builder = makeFunctionBuilder(functionName, className)
    +          // Put this Hive built-in function to our function registry.
    +          val info = new ExpressionInfo(className, functionName)
    +          createTempFunction(functionName, info, builder, ignoreIfExists = false)
    +          // Now, we need to create the Expression.
    +          functionRegistry.lookupFunction(functionName, children)
    +        }
    +    }
    +  }
    +
    +  // Pre-load a few commonly used Hive built-in functions.
    +  HiveSessionCatalog.preloadedHiveBuiltinFunctions.foreach {
    --- End diff --
    
    This is run in the constructor. Can you put it at the top?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [WIP] [SQL] Handle CreateFunctio...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-204652079
  
    @viirya I have made a few changes based on your PR. I am removing HiveFunctionRegistry and make HiveSessionCatalog to load Hive's builtin functions if necessary. I still need to do the cleanup work and add more tests. Once I finish these, I will ping you. (btw, please feel free to review the current version)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58472493
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -428,53 +432,85 @@ class SessionCatalog(
        */
       def dropFunction(name: FunctionIdentifier): Unit = {
         val db = name.database.getOrElse(currentDb)
    +    val qualified = name.copy(database = Some(db)).unquotedString
    +    if (functionRegistry.functionExists(qualified)) {
    +      // If we have loaded this function into FunctionRegistry,
    +      // also drop it from there.
    +      functionRegistry.dropFunction(qualified)
    +    }
         externalCatalog.dropFunction(db, name.funcName)
       }
     
       /**
    -   * Alter a metastore function whose name that matches the one specified in `funcDefinition`.
    -   *
    -   * If no database is specified in `funcDefinition`, assume the function is in the
    -   * current database.
    -   *
    -   * Note: If the underlying implementation does not support altering a certain field,
    -   * this becomes a no-op.
    -   */
    -  def alterFunction(funcDefinition: CatalogFunction): Unit = {
    -    val db = funcDefinition.identifier.database.getOrElse(currentDb)
    -    val newFuncDefinition = funcDefinition.copy(
    -      identifier = FunctionIdentifier(funcDefinition.identifier.funcName, Some(db)))
    -    externalCatalog.alterFunction(db, newFuncDefinition)
    -  }
    -
    -  /**
        * Retrieve the metadata of a metastore function.
        *
        * If a database is specified in `name`, this will return the function in that database.
        * If no database is specified, this will return the function in the current database.
        */
    +  // TODO: have a better name. This method is actually for fetching the metadata of a function.
       def getFunction(name: FunctionIdentifier): CatalogFunction = {
         val db = name.database.getOrElse(currentDb)
         externalCatalog.getFunction(db, name.funcName)
       }
     
    +  /**
    +   * Check if a function is already existing.
    +   *
    +   */
    +  def functionExists(name: FunctionIdentifier): Boolean = {
    +    if (functionRegistry.functionExists(name.unquotedString)) {
    +      // This function exists in the FunctionRegistry.
    +      true
    +    } else {
    +      // Need to check if this function exists in the metastore.
    +      try {
    +        getFunction(name) != null
    +      } catch {
    +        case _: NoSuchFunctionException => false
    +        case _: AnalysisException => false // HiveExternalCatalog wraps all exceptions with it.
    +      }
    +    }
    +  }
     
       // ----------------------------------------------------------------
       // | Methods that interact with temporary and metastore functions |
       // ----------------------------------------------------------------
     
       /**
    +   * Construct a [[FunctionBuilder]] based on the provided class that represents a function.
    +   *
    +   * This performs reflection to decide what type of [[Expression]] to return in the builder.
    +   */
    +  private[sql] def makeFunctionBuilder(name: String, functionClassName: String): FunctionBuilder = {
    +    // TODO: at least support UDAFs here
    +    throw new UnsupportedOperationException("Use sqlContext.udf.register(...) instead.")
    +  }
    +
    +  /**
    +   * Loads resources such as JARs and Files for a function.
    +   */
    +  def loadFunctionResources(resources: Seq[(String, String)]): Unit = {
    --- End diff --
    
    Done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SQL] Handle CreateFunction/Drop...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-205493524
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/54877/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SQL] Handle CreateFunction/Drop...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-205349596
  
    **[Test build #54857 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/54857/consoleFull)** for PR 12117 at commit [`a16395a`](https://github.com/apache/spark/commit/a16395a343e024cee8c641ee1ccd23161be1d447).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [WIP] [SQL] Handle CreateFunctio...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-204655379
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SQL] Handle CreateFunction/Drop...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-205164097
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SQL] Handle CreateFunction/Drop...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58447748
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala ---
    @@ -52,6 +52,8 @@ trait FunctionRegistry {
       /** Drop a function and return whether the function existed. */
       def dropFunction(name: String): Boolean
     
    +  /** Checks if a function with a given name exits. */
    --- End diff --
    
    exists


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [WIP] [SQL] Handle CreateFunctio...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58286788
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveSqlParser.scala ---
    @@ -272,10 +272,7 @@ class HiveSqlAstBuilder extends SparkSqlAstBuilder {
           name: String,
           expressions: Seq[Expression],
           ctx: LateralViewContext): Generator = {
    -    val info = Option(FunctionRegistry.getFunctionInfo(name.toLowerCase)).getOrElse {
    -      throw new ParseException(s"Couldn't find Generator function '$name'", ctx)
    -    }
    -    HiveGenericUDTF(name, new HiveFunctionWrapper(info.getFunctionClass.getName), expressions)
    +    UnresolvedGenerator(name, expressions)
    --- End diff --
    
    yeah, I have considered to do this to avoid passing `FunctionRegistry` into `HiveSqlParser`, but not do it yet.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [WIP] [SQL] Handle CreateFunctio...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58315454
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala ---
    @@ -316,8 +316,7 @@ class SparkSqlAstBuilder extends AstBuilder {
           function,
           string(ctx.className), // TODO this is not an alias.
    --- End diff --
    
    TODO can be removed now.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SQL] Handle CreateFunction/DropFunction

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-204608545
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SQL] Handle CreateFunction/Drop...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58447880
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala ---
    @@ -341,9 +343,14 @@ object FunctionRegistry {
       /** See usage above. */
       def expression[T <: Expression](name: String)
           (implicit tag: ClassTag[T]): (String, (ExpressionInfo, FunctionBuilder)) = {
    +    expression(name, tag.runtimeClass.asInstanceOf[Class[T]])
    +  }
     
    +  def expression[T <: Expression](
    +      name: String,
    +      clazz: Class[T]): (String, (ExpressionInfo, FunctionBuilder)) = {
    --- End diff --
    
    you don't need to change these right? IIUC the only change that's necessary in this file is adding `functionExists`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SQL] Handle CreateFunction/Drop...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58454099
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala ---
    @@ -159,6 +160,7 @@ abstract class ExternalCatalog {
        */
       def alterFunction(db: String, funcDefinition: CatalogFunction): Unit
    --- End diff --
    
    you can also remove this and all associated things in `CatalogTestCases`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58478223
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala ---
    @@ -201,8 +201,13 @@ class TestHiveContext private[hive](
         }
     
         override lazy val functionRegistry = {
    -      new TestHiveFunctionRegistry(
    -        org.apache.spark.sql.catalyst.analysis.FunctionRegistry.builtin.copy(), self.executionHive)
    +      // TestHiveFunctionRegistry tracks removed functions. So, we cannot simply use
    --- End diff --
    
    Actually I do not fully understand this comment.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58472439
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -453,28 +464,80 @@ class SessionCatalog(
        * If a database is specified in `name`, this will return the function in that database.
        * If no database is specified, this will return the function in the current database.
        */
    +  // TODO: have a better name. This method is actually for fetching the metadata of a function.
    --- End diff --
    
    OK. Will do it in another PR.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SQL] Handle CreateFunction/Drop...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-205183929
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/54830/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SQL] Handle CreateFunction/DropFunction

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58282723
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -69,6 +73,7 @@ class SessionCatalog(
       /**
        * Format table name, taking into account case sensitivity.
        */
    +  // TODO: Should we use it for temp function name?
    --- End diff --
    
    Yes, but let's do that separately.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58472917
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala ---
    @@ -146,6 +146,7 @@ abstract class ExternalCatalog {
     
       def createFunction(db: String, funcDefinition: CatalogFunction): Unit
     
    +  // TODO: dropFunction should take a FunctionIdentifier.
    --- End diff --
    
    Done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [WIP] [SQL] Handle CreateFunctio...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-204655350
  
    **[Test build #54764 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/54764/consoleFull)** for PR 12117 at commit [`aa41b20`](https://github.com/apache/spark/commit/aa41b20eece2f77de9815c66d35b0f2ef0464b88).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SQL] Handle CreateFunction/Drop...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58454975
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -428,53 +432,85 @@ class SessionCatalog(
        */
       def dropFunction(name: FunctionIdentifier): Unit = {
         val db = name.database.getOrElse(currentDb)
    +    val qualified = name.copy(database = Some(db)).unquotedString
    +    if (functionRegistry.functionExists(qualified)) {
    +      // If we have loaded this function into FunctionRegistry,
    +      // also drop it from there.
    +      functionRegistry.dropFunction(qualified)
    +    }
         externalCatalog.dropFunction(db, name.funcName)
       }
     
       /**
    -   * Alter a metastore function whose name that matches the one specified in `funcDefinition`.
    -   *
    -   * If no database is specified in `funcDefinition`, assume the function is in the
    -   * current database.
    -   *
    -   * Note: If the underlying implementation does not support altering a certain field,
    -   * this becomes a no-op.
    -   */
    -  def alterFunction(funcDefinition: CatalogFunction): Unit = {
    -    val db = funcDefinition.identifier.database.getOrElse(currentDb)
    -    val newFuncDefinition = funcDefinition.copy(
    -      identifier = FunctionIdentifier(funcDefinition.identifier.funcName, Some(db)))
    -    externalCatalog.alterFunction(db, newFuncDefinition)
    -  }
    -
    -  /**
        * Retrieve the metadata of a metastore function.
        *
        * If a database is specified in `name`, this will return the function in that database.
        * If no database is specified, this will return the function in the current database.
        */
    +  // TODO: have a better name. This method is actually for fetching the metadata of a function.
       def getFunction(name: FunctionIdentifier): CatalogFunction = {
         val db = name.database.getOrElse(currentDb)
         externalCatalog.getFunction(db, name.funcName)
       }
     
    +  /**
    +   * Check if a function is already existing.
    --- End diff --
    
    `Check if the specified function exists.`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SQL] Handle CreateFunction/DropFunction

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-204592574
  
    **[Test build #54731 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/54731/consoleFull)** for PR 12117 at commit [`979b03e`](https://github.com/apache/spark/commit/979b03e16c4853d255401dfb2b104bfe10cdcb64).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [WIP] [SQL] Handle CreateFunctio...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-204663926
  
    @yhuai OK. Thanks for letting me know that.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-205616585
  
    **[Test build #54931 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/54931/consoleFull)** for PR 12117 at commit [`88fd93c`](https://github.com/apache/spark/commit/88fd93c54cf8b6e70edab25526be93cd91e2563a).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58474058
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala ---
    @@ -0,0 +1,109 @@
    +/*
    + * 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.spark.sql.execution.command
    +
    +import org.apache.spark.sql.{AnalysisException, Row, SQLContext}
    +import org.apache.spark.sql.catalyst.FunctionIdentifier
    +import org.apache.spark.sql.catalyst.catalog.CatalogFunction
    +import org.apache.spark.sql.catalyst.expressions.ExpressionInfo
    +
    +
    +/**
    + * The DDL command that creates a function.
    + * alias: the class name that implements the created function.
    + * resources: Jars, files, or archives which need to be added to the environment when the function
    + *            is referenced for the first time by a session.
    + * isTemp: indicates if it is a temporary function.
    + */
    +// TODO: Use Seq[FunctionResource] instead of Seq[(String, String)] for resources.
    +case class CreateFunction(
    +    databaseName: Option[String],
    +    functionName: String,
    +    className: String,
    +    resources: Seq[(String, String)],
    +    isTemp: Boolean)
    +  extends RunnableCommand {
    +
    +  override def run(sqlContext: SQLContext): Seq[Row] = {
    +    if (isTemp) {
    +      if (databaseName.isDefined) {
    +        throw new AnalysisException(
    +          s"It is not allowed to provide database name when defining a temporary function. " +
    +            s"However, database name ${databaseName.get} is provided.")
    +      }
    +      // We first load resources and then put the builder in the function registry.
    +      // Please note that it is allowed to overwrite an existing temp function.
    +      sqlContext.sessionState.catalog.loadFunctionResources(resources)
    --- End diff --
    
    Done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58473146
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala ---
    @@ -208,6 +208,22 @@ class SQLContext private[sql](
         sparkContext.addJar(path)
       }
     
    +  /** Returns a [[FunctionResourceLoader]] that can be used in SessionCatalog. */
    +  protected[sql] def functionResourceLoader: FunctionResourceLoader = {
    --- End diff --
    
    Changed it to a lazy val.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58471897
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala ---
    @@ -52,6 +52,8 @@ trait FunctionRegistry {
       /** Drop a function and return whether the function existed. */
       def dropFunction(name: String): Boolean
     
    +  /** Checks if a function with a given name exits. */
    --- End diff --
    
    Done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [WIP] [SQL] Handle CreateFunctio...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-205059190
  
    **[Test build #54813 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/54813/consoleFull)** for PR 12117 at commit [`776c09a`](https://github.com/apache/spark/commit/776c09afd8f448c78b063de510b14434764d72cd).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-205570551
  
    Thanks @yhuai for improving the original changes. This looks great now.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58465870
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala ---
    @@ -0,0 +1,109 @@
    +/*
    + * 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.spark.sql.execution.command
    +
    +import org.apache.spark.sql.{AnalysisException, Row, SQLContext}
    +import org.apache.spark.sql.catalyst.FunctionIdentifier
    +import org.apache.spark.sql.catalyst.catalog.CatalogFunction
    +import org.apache.spark.sql.catalyst.expressions.ExpressionInfo
    +
    +
    +/**
    + * The DDL command that creates a function.
    + * alias: the class name that implements the created function.
    + * resources: Jars, files, or archives which need to be added to the environment when the function
    + *            is referenced for the first time by a session.
    + * isTemp: indicates if it is a temporary function.
    + */
    +// TODO: Use Seq[FunctionResource] instead of Seq[(String, String)] for resources.
    +case class CreateFunction(
    +    databaseName: Option[String],
    +    functionName: String,
    +    className: String,
    +    resources: Seq[(String, String)],
    +    isTemp: Boolean)
    +  extends RunnableCommand {
    +
    +  override def run(sqlContext: SQLContext): Seq[Row] = {
    +    if (isTemp) {
    +      if (databaseName.isDefined) {
    +        throw new AnalysisException(
    +          s"It is not allowed to provide database name when defining a temporary function. " +
    +            s"However, database name ${databaseName.get} is provided.")
    +      }
    +      // We first load resources and then put the builder in the function registry.
    +      // Please note that it is allowed to overwrite an existing temp function.
    +      sqlContext.sessionState.catalog.loadFunctionResources(resources)
    +      val info = new ExpressionInfo(className, functionName)
    +      val builder =
    +        sqlContext.sessionState.catalog.makeFunctionBuilder(functionName, className)
    +      sqlContext.sessionState.catalog.createTempFunction(
    +        functionName, info, builder, ignoreIfExists = false)
    +    } else {
    +      val dbName = databaseName.getOrElse(sqlContext.sessionState.catalog.getCurrentDatabase)
    +      val func = FunctionIdentifier(functionName, Some(dbName))
    +      val catalogFunc = CatalogFunction(func, className, resources)
    +      // We are creating a permanent function. First, we want to check if this function
    +      // has already been created.
    +      // Check if the function to create is already existing. If so, throw exception.
    --- End diff --
    
    this comment is highly unnecessary


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SQL] Handle CreateFunction/DropFunction

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-204584187
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/54723/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58456603
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala ---
    @@ -146,6 +146,7 @@ abstract class ExternalCatalog {
     
       def createFunction(db: String, funcDefinition: CatalogFunction): Unit
     
    +  // TODO: dropFunction should take a FunctionIdentifier.
    --- End diff --
    
    (same in L163)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58469841
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala ---
    @@ -341,9 +343,14 @@ object FunctionRegistry {
       /** See usage above. */
       def expression[T <: Expression](name: String)
           (implicit tag: ClassTag[T]): (String, (ExpressionInfo, FunctionBuilder)) = {
    +    expression(name, tag.runtimeClass.asInstanceOf[Class[T]])
    +  }
     
    +  def expression[T <: Expression](
    +      name: String,
    +      clazz: Class[T]): (String, (ExpressionInfo, FunctionBuilder)) = {
    --- End diff --
    
    yeah, it can be removed now.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-205957877
  
    Thanks @yhuai @viirya I'm merging this into master.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SQL] Handle CreateFunction/Drop...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-205163815
  
    **[Test build #54824 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/54824/consoleFull)** for PR 12117 at commit [`9d39a83`](https://github.com/apache/spark/commit/9d39a835a25ea2a509fa490099a7923b70e8bcbd).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `  implicit class DebugQuery(query: Dataset[_]) extends Logging `


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SQL] Handle CreateFunction/Drop...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58329279
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -491,40 +528,31 @@ class SessionCatalog(
       }
     
       /**
    -   * Rename a function.
    -   *
    -   * If a database is specified in `oldName`, this will rename the function in that database.
    -   * If no database is specified, this will first attempt to rename a temporary function with
    -   * the same name, then, if that does not exist, rename the function in the current database.
    -   *
    -   * This assumes the database specified in `oldName` matches the one specified in `newName`.
    -   */
    -  def renameFunction(oldName: FunctionIdentifier, newName: FunctionIdentifier): Unit = {
    -    if (oldName.database != newName.database) {
    -      throw new AnalysisException("rename does not support moving functions across databases")
    -    }
    -    val db = oldName.database.getOrElse(currentDb)
    -    val oldBuilder = functionRegistry.lookupFunctionBuilder(oldName.funcName)
    -    if (oldName.database.isDefined || oldBuilder.isEmpty) {
    -      externalCatalog.renameFunction(db, oldName.funcName, newName.funcName)
    -    } else {
    -      val oldExpressionInfo = functionRegistry.lookupFunction(oldName.funcName).get
    -      val newExpressionInfo = new ExpressionInfo(
    -        oldExpressionInfo.getClassName,
    -        newName.funcName,
    -        oldExpressionInfo.getUsage,
    -        oldExpressionInfo.getExtended)
    -      functionRegistry.dropFunction(oldName.funcName)
    -      functionRegistry.registerFunction(newName.funcName, newExpressionInfo, oldBuilder.get)
    -    }
    -  }
    -
    -  /**
        * Return an [[Expression]] that represents the specified function, assuming it exists.
        * Note: This is currently only used for temporary functions.
        */
       def lookupFunction(name: String, children: Seq[Expression]): Expression = {
    -    functionRegistry.lookupFunction(name, children)
    +    // TODO: Right now, the name can be qualified or not qualified.
    +    // It will be better to get a FunctionIdentifier.
    +    // TODO: Right now, we assume that name is not qualified!
    --- End diff --
    
    Added in `org.apache.spark.sql.hive.UDFSuite`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SQL] Handle CreateFunction/DropFunction

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-204584184
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SQL] Handle CreateFunction/Drop...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58413111
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala ---
    @@ -112,4 +124,121 @@ class HiveSessionCatalog(
         metastoreCatalog.cachedDataSourceTables.getIfPresent(key)
       }
     
    +  override def makeFunctionBuilder(funcName: String, className: String): FunctionBuilder = {
    +    makeFunctionBuilder(funcName, Utils.classForName(className))
    +  }
    +
    +  /**
    +   * Construct a [[FunctionBuilder]] based on the provided class that represents a function.
    +   */
    +  private def makeFunctionBuilder(name: String, clazz: Class[_]): FunctionBuilder = {
    +    // When we instantiate hive UDF wrapper class, we may throw exception if the input
    +    // expressions don't satisfy the hive UDF, such as type mismatch, input number
    +    // mismatch, etc. Here we catch the exception and throw AnalysisException instead.
    +    (children: Seq[Expression]) => {
    +      try {
    +        if (classOf[UDF].isAssignableFrom(clazz)) {
    +          val udf = HiveSimpleUDF(name, new HiveFunctionWrapper(clazz.getName), children)
    +          udf.dataType // Force it to check input data types.
    +          udf
    +        } else if (classOf[GenericUDF].isAssignableFrom(clazz)) {
    +          val udf = HiveGenericUDF(name, new HiveFunctionWrapper(clazz.getName), children)
    +          udf.dataType // Force it to check input data types.
    +          udf
    +        } else if (classOf[AbstractGenericUDAFResolver].isAssignableFrom(clazz)) {
    +          val udaf = HiveUDAFFunction(name, new HiveFunctionWrapper(clazz.getName), children)
    +          udaf.dataType // Force it to check input data types.
    +          udaf
    +        } else if (classOf[UDAF].isAssignableFrom(clazz)) {
    +          val udaf = HiveUDAFFunction(
    +            name,
    +            new HiveFunctionWrapper(clazz.getName),
    +            children,
    +            isUDAFBridgeRequired = true)
    +          udaf.dataType  // Force it to check input data types.
    +          udaf
    +        } else if (classOf[GenericUDTF].isAssignableFrom(clazz)) {
    +          val udtf = HiveGenericUDTF(name, new HiveFunctionWrapper(clazz.getName), children)
    +          udtf.elementTypes // Force it to check input data types.
    +          udtf
    +        } else {
    +          throw new AnalysisException(s"No handler for Hive UDF '${clazz.getCanonicalName}'")
    +        }
    +      } catch {
    +        case ae: AnalysisException =>
    +          throw ae
    +        case NonFatal(e) =>
    +          val analysisException =
    +            new AnalysisException(s"No handler for Hive UDF '${clazz.getCanonicalName}': $e")
    +          analysisException.setStackTrace(e.getStackTrace)
    +          throw analysisException
    +      }
    +    }
    +  }
    +
    +  // We have a list of Hive built-in functions that we do not support. So, we will check
    +  // Hive's function registry and lazily load needed functions into our own function registry.
    +  // Those Hive built-in functions are
    +  // assert_true, collect_list, collect_set, compute_stats, context_ngrams, create_union,
    +  // current_user ,elt, ewah_bitmap, ewah_bitmap_and, ewah_bitmap_empty, ewah_bitmap_or, field,
    +  // histogram_numeric, in_file, index, inline, java_method, map_keys, map_values,
    +  // matchpath, ngrams, noop, noopstreaming, noopwithmap, noopwithmapstreaming,
    +  // parse_url, parse_url_tuple, percentile, percentile_approx, posexplode, reflect, reflect2,
    +  // regexp, sentences, stack, std, str_to_map, windowingtablefunction, xpath, xpath_boolean,
    +  // xpath_double, xpath_float, xpath_int, xpath_long, xpath_number,
    +  // xpath_short, and xpath_string.
    +  override def lookupFunction(name: String, children: Seq[Expression]): Expression = {
    +    Try(super.lookupFunction(name, children)) match {
    +      case Success(expr) => expr
    +      case Failure(error) =>
    +        if (functionRegistry.functionExists(name)) {
    +          // If the function actually exists in functionRegistry, it means that there is an
    +          // error when we create the Expression using the given children.
    +          // We need to throw the original exception.
    +          throw error
    --- End diff --
    
    no I mean `lookupFunction(name)` that returns an option


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58468638
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala ---
    @@ -0,0 +1,109 @@
    +/*
    + * 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.spark.sql.execution.command
    +
    +import org.apache.spark.sql.{AnalysisException, Row, SQLContext}
    +import org.apache.spark.sql.catalyst.FunctionIdentifier
    +import org.apache.spark.sql.catalyst.catalog.CatalogFunction
    +import org.apache.spark.sql.catalyst.expressions.ExpressionInfo
    +
    +
    +/**
    + * The DDL command that creates a function.
    + * alias: the class name that implements the created function.
    + * resources: Jars, files, or archives which need to be added to the environment when the function
    + *            is referenced for the first time by a session.
    + * isTemp: indicates if it is a temporary function.
    + */
    +// TODO: Use Seq[FunctionResource] instead of Seq[(String, String)] for resources.
    +case class CreateFunction(
    +    databaseName: Option[String],
    +    functionName: String,
    +    className: String,
    +    resources: Seq[(String, String)],
    +    isTemp: Boolean)
    +  extends RunnableCommand {
    +
    +  override def run(sqlContext: SQLContext): Seq[Row] = {
    +    if (isTemp) {
    +      if (databaseName.isDefined) {
    +        throw new AnalysisException(
    +          s"It is not allowed to provide database name when defining a temporary function. " +
    +            s"However, database name ${databaseName.get} is provided.")
    +      }
    +      // We first load resources and then put the builder in the function registry.
    +      // Please note that it is allowed to overwrite an existing temp function.
    +      sqlContext.sessionState.catalog.loadFunctionResources(resources)
    --- End diff --
    
    Permanent function in Hive will be loaded when it is first time referred in a session.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-205616658
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/54931/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58465597
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala ---
    @@ -0,0 +1,109 @@
    +/*
    + * 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.spark.sql.execution.command
    +
    +import org.apache.spark.sql.{AnalysisException, Row, SQLContext}
    +import org.apache.spark.sql.catalyst.FunctionIdentifier
    +import org.apache.spark.sql.catalyst.catalog.CatalogFunction
    +import org.apache.spark.sql.catalyst.expressions.ExpressionInfo
    +
    +
    +/**
    + * The DDL command that creates a function.
    + * alias: the class name that implements the created function.
    + * resources: Jars, files, or archives which need to be added to the environment when the function
    + *            is referenced for the first time by a session.
    + * isTemp: indicates if it is a temporary function.
    + */
    +// TODO: Use Seq[FunctionResource] instead of Seq[(String, String)] for resources.
    +case class CreateFunction(
    +    databaseName: Option[String],
    +    functionName: String,
    +    className: String,
    +    resources: Seq[(String, String)],
    +    isTemp: Boolean)
    +  extends RunnableCommand {
    +
    +  override def run(sqlContext: SQLContext): Seq[Row] = {
    +    if (isTemp) {
    +      if (databaseName.isDefined) {
    +        throw new AnalysisException(
    +          s"It is not allowed to provide database name when defining a temporary function. " +
    +            s"However, database name ${databaseName.get} is provided.")
    +      }
    +      // We first load resources and then put the builder in the function registry.
    +      // Please note that it is allowed to overwrite an existing temp function.
    +      sqlContext.sessionState.catalog.loadFunctionResources(resources)
    --- End diff --
    
    Why do we have the following behavior: for temporary functions, load the resources when we create the function, but for persistent functions, load the resources lazily when we call `lookupFunction` later?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SQL] Handle CreateFunction/DropFunction

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58278714
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -453,28 +464,80 @@ class SessionCatalog(
        * If a database is specified in `name`, this will return the function in that database.
        * If no database is specified, this will return the function in the current database.
        */
    +  // TODO: have a better name. This method is actually for fetching the metadata of a function.
    --- End diff --
    
    fetchFunctionMetadata?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-205652502
  
    **[Test build #54948 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/54948/consoleFull)** for PR 12117 at commit [`361421c`](https://github.com/apache/spark/commit/361421c02390dd2c0dc00bb4e09a150c37bee54a).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SQL] Handle CreateFunction/Drop...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-205183785
  
    **[Test build #54830 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/54830/consoleFull)** for PR 12117 at commit [`0572acc`](https://github.com/apache/spark/commit/0572acce35f93961ac5f56bd215c736da00d4aea).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `case class CreateFunction(`
      * `case class DropFunction(`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-205831352
  
    **[Test build #54987 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/54987/consoleFull)** for PR 12117 at commit [`3938766`](https://github.com/apache/spark/commit/39387666fe24a2e65a267eabc15c7816c8738449).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SQL] Handle CreateFunction/DropFunction

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-204584953
  
    **[Test build #54724 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/54724/consoleFull)** for PR 12117 at commit [`0a1866c`](https://github.com/apache/spark/commit/0a1866c95bbe4ef260206360f18d521a99f5f482).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [WIP] [SQL] Handle CreateFunctio...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58286763
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala ---
    @@ -67,9 +70,14 @@ class SimpleFunctionRegistry extends FunctionRegistry {
       }
     
       override def lookupFunction(name: String, children: Seq[Expression]): Expression = {
    +    val builder = functionBuilders.get(name)
    +    if (builder.isEmpty) {
    +      throw new AnalysisException(s"undefined function $name")
    +    }
         val func = synchronized {
    -      functionBuilders.get(name).map(_._2).getOrElse {
    -        throw new AnalysisException(s"undefined function $name")
    +      Try(builder.map(_._2)) match {
    --- End diff --
    
    Me too.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58456466
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -524,7 +537,39 @@ class SessionCatalog(
        * Note: This is currently only used for temporary functions.
        */
       def lookupFunction(name: String, children: Seq[Expression]): Expression = {
    -    functionRegistry.lookupFunction(name, children)
    +    // TODO: Right now, the name can be qualified or not qualified.
    +    // It will be better to get a FunctionIdentifier.
    --- End diff --
    
    I filed https://issues.apache.org/jira/browse/SPARK-14385 for this.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-205629207
  
    **[Test build #54948 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/54948/consoleFull)** for PR 12117 at commit [`361421c`](https://github.com/apache/spark/commit/361421c02390dd2c0dc00bb4e09a150c37bee54a).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SQL] Handle CreateFunction/Drop...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-205164101
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/54824/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58466440
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala ---
    @@ -0,0 +1,109 @@
    +/*
    + * 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.spark.sql.execution.command
    +
    +import org.apache.spark.sql.{AnalysisException, Row, SQLContext}
    +import org.apache.spark.sql.catalyst.FunctionIdentifier
    +import org.apache.spark.sql.catalyst.catalog.CatalogFunction
    +import org.apache.spark.sql.catalyst.expressions.ExpressionInfo
    +
    +
    +/**
    + * The DDL command that creates a function.
    + * alias: the class name that implements the created function.
    + * resources: Jars, files, or archives which need to be added to the environment when the function
    + *            is referenced for the first time by a session.
    + * isTemp: indicates if it is a temporary function.
    + */
    +// TODO: Use Seq[FunctionResource] instead of Seq[(String, String)] for resources.
    +case class CreateFunction(
    +    databaseName: Option[String],
    +    functionName: String,
    +    className: String,
    +    resources: Seq[(String, String)],
    +    isTemp: Boolean)
    +  extends RunnableCommand {
    +
    +  override def run(sqlContext: SQLContext): Seq[Row] = {
    +    if (isTemp) {
    +      if (databaseName.isDefined) {
    +        throw new AnalysisException(
    +          s"It is not allowed to provide database name when defining a temporary function. " +
    +            s"However, database name ${databaseName.get} is provided.")
    +      }
    +      // We first load resources and then put the builder in the function registry.
    +      // Please note that it is allowed to overwrite an existing temp function.
    +      sqlContext.sessionState.catalog.loadFunctionResources(resources)
    +      val info = new ExpressionInfo(className, functionName)
    +      val builder =
    +        sqlContext.sessionState.catalog.makeFunctionBuilder(functionName, className)
    +      sqlContext.sessionState.catalog.createTempFunction(
    +        functionName, info, builder, ignoreIfExists = false)
    +    } else {
    +      val dbName = databaseName.getOrElse(sqlContext.sessionState.catalog.getCurrentDatabase)
    +      val func = FunctionIdentifier(functionName, Some(dbName))
    +      val catalogFunc = CatalogFunction(func, className, resources)
    +      // We are creating a permanent function. First, we want to check if this function
    +      // has already been created.
    +      // Check if the function to create is already existing. If so, throw exception.
    +      if (sqlContext.sessionState.catalog.functionExists(func)) {
    +        throw new AnalysisException(
    +          s"Function '$functionName' already exists in database '$dbName'.")
    +      }
    +      // This function will be loaded into the FunctionRegistry when a query uses it.
    +      sqlContext.sessionState.catalog.createFunction(catalogFunc)
    +    }
    +    Seq.empty[Row]
    +  }
    +}
    +
    +/**
    + * The DDL command that drops a function.
    + * ifExists: returns an error if the function doesn't exist, unless this is true.
    + * isTemp: indicates if it is a temporary function.
    + */
    +case class DropFunction(
    +    databaseName: Option[String],
    +    functionName: String,
    +    ifExists: Boolean,
    +    isTemp: Boolean)
    +  extends RunnableCommand {
    +
    +  override def run(sqlContext: SQLContext): Seq[Row] = {
    +    if (isTemp) {
    +      if (databaseName.isDefined) {
    +        throw new AnalysisException(
    +          s"It is not allowed to provide database name when dropping a temporary function. " +
    +            s"However, database name ${databaseName.get} is provided.")
    +      }
    +      sqlContext.sessionState.catalog.dropTempFunction(functionName, ifExists)
    +    } else {
    +      // We are dropping a permanent.
    +      val dbName = databaseName.getOrElse(sqlContext.sessionState.catalog.getCurrentDatabase)
    +      val func = FunctionIdentifier(functionName, Some(dbName))
    --- End diff --
    
    Resolving the database to use should be done in `SessionCatalog`, not here. It's OK to pass in `None` as a database here and let the catalog resolve that for us based in the current database.
    ```
    val func = FunctionIdentifier(functionName, databaseName)
    ```



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SQL] Handle CreateFunction/DropFunction

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-204614917
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/54731/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [WIP] [SQL] Handle CreateFunctio...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58316149
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -491,40 +528,31 @@ class SessionCatalog(
       }
     
       /**
    -   * Rename a function.
    -   *
    -   * If a database is specified in `oldName`, this will rename the function in that database.
    -   * If no database is specified, this will first attempt to rename a temporary function with
    -   * the same name, then, if that does not exist, rename the function in the current database.
    -   *
    -   * This assumes the database specified in `oldName` matches the one specified in `newName`.
    -   */
    -  def renameFunction(oldName: FunctionIdentifier, newName: FunctionIdentifier): Unit = {
    -    if (oldName.database != newName.database) {
    -      throw new AnalysisException("rename does not support moving functions across databases")
    -    }
    -    val db = oldName.database.getOrElse(currentDb)
    -    val oldBuilder = functionRegistry.lookupFunctionBuilder(oldName.funcName)
    -    if (oldName.database.isDefined || oldBuilder.isEmpty) {
    -      externalCatalog.renameFunction(db, oldName.funcName, newName.funcName)
    -    } else {
    -      val oldExpressionInfo = functionRegistry.lookupFunction(oldName.funcName).get
    -      val newExpressionInfo = new ExpressionInfo(
    -        oldExpressionInfo.getClassName,
    -        newName.funcName,
    -        oldExpressionInfo.getUsage,
    -        oldExpressionInfo.getExtended)
    -      functionRegistry.dropFunction(oldName.funcName)
    -      functionRegistry.registerFunction(newName.funcName, newExpressionInfo, oldBuilder.get)
    -    }
    -  }
    -
    -  /**
        * Return an [[Expression]] that represents the specified function, assuming it exists.
        * Note: This is currently only used for temporary functions.
        */
       def lookupFunction(name: String, children: Seq[Expression]): Expression = {
    -    functionRegistry.lookupFunction(name, children)
    +    // TODO: Right now, the name can be qualified or not qualified.
    +    // It will be better to get a FunctionIdentifier.
    +    // TODO: Right now, we assume that name is not qualified!
    --- End diff --
    
    I will add a test for this (the test will be ignored right now).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SPARK-14384] [SQL] Handle Creat...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/12117#issuecomment-205625930
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-14123] [SQL] Handle CreateFunction/Drop...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12117#discussion_r58442629
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -534,17 +579,11 @@ class SessionCatalog(
         val dbFunctions =
           externalCatalog.listFunctions(db, pattern).map { f => FunctionIdentifier(f, Some(db)) }
         val regex = pattern.replaceAll("\\*", ".*").r
    --- End diff --
    
    let's do that separately so we can get this patch in first.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org