Skip to content

Commit

Permalink
add test cases and update code style
Browse files Browse the repository at this point in the history
  • Loading branch information
bomeng committed Mar 27, 2016
1 parent 033b1ac commit bdf44b4
Show file tree
Hide file tree
Showing 2 changed files with 32 additions and 5 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -446,10 +446,8 @@ case class SetDatabaseCommand(databaseName: String) extends RunnableCommand {
* }}}
*/
case class CreateFunction(
functionName: String,
alias: String,
resources: Seq[(String, String)],
isTemp: Boolean)(sql: String) extends RunnableCommand {
functionName: String, alias: String, resources: Seq[(String, String)],

This comment has been minimized.

Copy link
@gatorsmile

gatorsmile Mar 27, 2016

Member

Indents are still not right. You have to follow the indentation requirement, as shown in the following link:
https://cwiki.apache.org/confluence/display/SPARK/Spark+Code+Style+Guide

isTemp: Boolean)(sql: String) extends RunnableCommand {
override def run(sqlContext: SQLContext): Seq[Row] = {
val catalog = sqlContext.sessionState.catalog
val functionIdentifier = FunctionIdentifier(functionName, Some(catalog.getCurrentDatabase))
Expand All @@ -458,4 +456,4 @@ case class CreateFunction(
}

override val output: Seq[Attribute] = Seq.empty
}
}
29 changes: 29 additions & 0 deletions sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -21,10 +21,13 @@ import java.math.MathContext
import java.sql.Timestamp

import org.apache.spark.AccumulatorSuite
import org.apache.spark.sql.catalyst.FunctionIdentifier
import org.apache.spark.sql.catalyst.analysis.UnresolvedException
import org.apache.spark.sql.catalyst.catalog.CatalogFunction
import org.apache.spark.sql.catalyst.expressions.SortOrder
import org.apache.spark.sql.catalyst.plans.logical.Aggregate
import org.apache.spark.sql.execution.aggregate
import org.apache.spark.sql.execution.command.CreateFunction
import org.apache.spark.sql.execution.joins.{BroadcastHashJoin, CartesianProduct, SortMergeJoin}
import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SQLConf
Expand Down Expand Up @@ -2376,4 +2379,30 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext {
Row("r3c1x", "r3c2", "t1r3c3", "r3c2", "t1r3c3") :: Nil)
}
}

test("SPARK-14123") {

This comment has been minimized.

Copy link
@gatorsmile

gatorsmile Mar 27, 2016

Member

Update the name of this test case to make the future code readers easy to understand?

This comment has been minimized.

Copy link
@gatorsmile

gatorsmile Mar 27, 2016

Member

I am not 100% sure if this is the right suite to add. Maybe we can create a separate suite for DDL in SQLContext?

val sql1 =
"""
|CREATE TEMPORARY FUNCTION helloworld1 AS
|'spark.example.SimpleUDFExample1' USING JAR '/path/to/jar1',
|JAR '/path/to/jar2'
""".stripMargin
val sql2 =
"""
|CREATE FUNCTION helloworld2 AS
|'spark.example.SimpleUDFExample2' USING ARCHIVE '/path/to/archive',
|FILE '/path/to/file'
""".stripMargin
sql(sql1)
sql(sql2)

val catalog = sqlContext.sessionState.catalog
val id1 = FunctionIdentifier("helloworld1", Some(catalog.getCurrentDatabase))
val id2 = FunctionIdentifier("helloworld2", Some(catalog.getCurrentDatabase))

val f1 = catalog.getFunction(id1)
val f2 = catalog.getFunction(id2)
assert(f1 == CatalogFunction(id1, "spark.example.SimpleUDFExample1"))
assert(f2 == CatalogFunction(id2, "spark.example.SimpleUDFExample2"))
}
}

0 comments on commit bdf44b4

Please sign in to comment.