Skip to content
Closed
Show file tree
Hide file tree
Changes from 4 commits
Commits
Show all changes
29 commits
Select commit Hold shift + click to select a range
f2433c2
init commit
lianhuiwang Jun 16, 2016
0b93636
fix unit test
lianhuiwang Jun 16, 2016
301e950
update
lianhuiwang Jun 18, 2016
808a5fa
update createTempMacro
lianhuiwang Jun 18, 2016
f4ed3bc
address comments
lianhuiwang Jun 20, 2016
af0136d
update
lianhuiwang Jun 20, 2016
5550496
based master
lianhuiwang Nov 10, 2016
9fe1881
update code
lianhuiwang Nov 10, 2016
b8ffdc9
fix function
lianhuiwang Nov 10, 2016
fb8b57a
update comment
lianhuiwang Nov 11, 2016
e895a9c
update comments
lianhuiwang Nov 11, 2016
8d520eb
Merge branch 'master' of https://github.com/apache/spark into macro
lianhuiwang May 27, 2017
651b485
Merge branch 'master' of https://github.com/apache/spark into macro
lianhuiwang May 27, 2017
277ba9f
Merge branch 'macro' of https://github.com/lianhuiwang/spark into macro
lianhuiwang May 27, 2017
314913d
Merge branch 'macro' of https://github.com/lianhuiwang/spark into macro
lianhuiwang May 27, 2017
3d05e4f
reformat code.
lianhuiwang May 27, 2017
22d8b1a
reformat code.
lianhuiwang May 27, 2017
d91f633
reformat code.
lianhuiwang May 27, 2017
1eb23c7
reformat code.
lianhuiwang May 27, 2017
ad85109
remove type check for macro as same with hive.
lianhuiwang May 27, 2017
b52698f
add import
lianhuiwang May 27, 2017
3eacebc
treat macro as temp function like hive
lianhuiwang May 27, 2017
fce1121
add Modifier for FunctionRegistry.
lianhuiwang May 27, 2017
eaff4e9
update comments.
lianhuiwang May 27, 2017
97632a9
add dropMacro().
lianhuiwang May 27, 2017
4ee32e9
reformat code style
lianhuiwang May 27, 2017
b539e94
address some comments.
lianhuiwang May 30, 2017
1563f12
address some comments.
lianhuiwang May 30, 2017
4d8e843
update
lianhuiwang May 30, 2017
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -97,6 +97,9 @@ statement
| CREATE TEMPORARY? FUNCTION qualifiedName AS className=STRING
(USING resource (',' resource)*)? #createFunction
| DROP TEMPORARY? FUNCTION (IF EXISTS)? qualifiedName #dropFunction
| CREATE TEMPORARY MACRO macroName=identifier
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Does Hive also support non-temporary macro's.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

No, Now Hive only support temporary macro's.

'(' colTypeList? ')' expression #createMacro
| DROP TEMPORARY MACRO (IF EXISTS)? macroName=identifier #dropMacro
| EXPLAIN (LOGICAL | FORMATTED | EXTENDED | CODEGEN)? statement #explain
| SHOW TABLES ((FROM | IN) db=identifier)?
(LIKE? pattern=STRING)? #showTables
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

package org.apache.spark.sql.catalyst.analysis

import scala.collection.mutable.HashSet
import scala.language.existentials
import scala.reflect.ClassTag
import scala.util.{Failure, Success, Try}
Expand Down Expand Up @@ -59,6 +60,10 @@ trait FunctionRegistry {
/** Checks if a function with a given name exists. */
def functionExists(name: String): Boolean = lookupFunction(name).isDefined

def registerMacro(name: String, info: ExpressionInfo, builder: FunctionBuilder): Unit

def dropMacro(name: String): Boolean

/** Clear all registered functions. */
def clear(): Unit

Expand All @@ -69,6 +74,8 @@ class SimpleFunctionRegistry extends FunctionRegistry {
protected val functionBuilders =
StringKeyHashMap[(ExpressionInfo, FunctionBuilder)](caseSensitive = false)

val macros = new HashSet[String]

override def registerFunction(
name: String,
info: ExpressionInfo,
Expand Down Expand Up @@ -101,8 +108,26 @@ class SimpleFunctionRegistry extends FunctionRegistry {
functionBuilders.remove(name).isDefined
}

override def registerMacro(
name: String,
info: ExpressionInfo,
builder: FunctionBuilder): Unit = synchronized {
functionBuilders.put(name, (info, builder))
macros += name.toLowerCase()
}

override def dropMacro(name: String): Boolean = synchronized {
if (macros.contains(name.toLowerCase)) {
macros -= name.toLowerCase
functionBuilders.remove(name).isDefined
} else {
false
}
}

override def clear(): Unit = synchronized {
functionBuilders.clear()
macros.clear()
}

def copy(): SimpleFunctionRegistry = synchronized {
Expand Down Expand Up @@ -144,6 +169,14 @@ object EmptyFunctionRegistry extends FunctionRegistry {
throw new UnsupportedOperationException
}

override def registerMacro(name: String, info: ExpressionInfo, builder: FunctionBuilder): Unit = {
throw new UnsupportedOperationException
}

override def dropMacro(name: String): Boolean = {
throw new UnsupportedOperationException
}

override def clear(): Unit = {
throw new UnsupportedOperationException
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -52,3 +52,6 @@ class NoSuchPartitionsException(db: String, table: String, specs: Seq[TableParti

class NoSuchTempFunctionException(func: String)
extends AnalysisException(s"Temporary function '$func' not found")

class NoSuchTempMacroException(func: String)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please remove it. For reasons, please see the PR #17716.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, Thanks.

extends AnalysisException(s"Temporary macro '$func' not found")
Original file line number Diff line number Diff line change
Expand Up @@ -758,6 +758,24 @@ class SessionCatalog(
}
}

/** Create a temporary macro. */
def createTempMacro(
name: String,
info: ExpressionInfo,
funcDefinition: FunctionBuilder): Unit = {
if (functionRegistry.functionExists(name)) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I am not entirely sure if we should throw an exception here. It unfortunately depends on the semantics you follow, SQL will throw an exception, whereas the Dataframe API will just overwrite the function. Let's follow Hive for now.

Copy link
Member

@gatorsmile gatorsmile May 29, 2017

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

hive> create temporary macro max(x int)
    > x*x;
OK
Time taken: 0.014 seconds

hive> select max(3) from t1;
OK
9
Time taken: 0.468 seconds, Fetched: 1 row(s)

hive> select max(3,4) from t1;
FAILED: SemanticException [Error 10015]: Line 1:7 Arguments length mismatch '4': The macro max accepts exactly 1 arguments.

Hive overwrites the temporary function without issuing an exception.

throw new TempFunctionAlreadyExistsException(name)
}
functionRegistry.registerMacro(name, info, funcDefinition)
}

/** Drop a temporary macro. */
def dropTempMacro(name: String, ignoreIfNotExists: Boolean): Unit = {
if (!functionRegistry.dropMacro(name) && !ignoreIfNotExists) {
throw new NoSuchTempMacroException(name)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

hive>  DROP TEMPORARY MACRO max;
OK
Time taken: 0.01 seconds
hive> select max(3) from t1;
OK
3

After we drop the macro, the existing function works well. That means, we did not delete the original built-in functions. The built-in function will not be dropped by DROP TEMPORARY MACRO. After we drop the macro with the same name, the original function max is using the original built-in function.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, I have update it with this case.

}
}

protected def failFunctionLookup(name: String): Nothing = {
throw new NoSuchFunctionException(db = currentDb, func = name)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,14 +25,16 @@ import org.antlr.v4.runtime.tree.TerminalNode

import org.apache.spark.sql.SaveMode
import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier}
import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
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, OneRowRelation, ScriptInputOutputSchema}
import org.apache.spark.sql.execution.command._
import org.apache.spark.sql.execution.datasources.{CreateTempViewUsing, _}
import org.apache.spark.sql.internal.{HiveSerDe, SQLConf, VariableSubstitution}
import org.apache.spark.sql.types.DataType
import org.apache.spark.sql.types.{DataType, StructField}

/**
* Concrete parser for Spark SQL statements.
Expand Down Expand Up @@ -589,6 +591,53 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder {
ctx.TEMPORARY != null)
}

/**
* Create a [[CreateMacroCommand]] command.
*
* For example:
* {{{
* CREATE TEMPORARY MACRO macro_name([col_name col_type, ...]) expression;
* }}}
*/
override def visitCreateMacro(ctx: CreateMacroContext): LogicalPlan = withOrigin(ctx) {
val arguments = Option(ctx.colTypeList).map(visitColTypeList(_))
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nit: you can avoid (_)...

.getOrElse(Seq.empty[StructField]).map { col =>
AttributeReference(col.name, col.dataType, col.nullable, col.metadata)() }
val colToIndex: Map[String, Int] = arguments.map(_.name).zipWithIndex.toMap
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Move this into the CreateMacroCommand command. This would also be relevant if we were to offer a different API for creating macro's.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why i do not to move this into the CreateMacroCommand? Because analyzer.checkAnalysis() will check if macroFunction of CreateMacroCommand is invalid. macroFunction has UnresolvedAttributes, So analyzer.checkAnalysis() will throw a unresolved exception. If it resolved UnresolvedAttributes before, analyzer.checkAnalysis() does not throw a exception.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@hvanhovell So i think i will create a new Wrapper class to avoid unresolved exception in order to DataFrame can reuse this feature later.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ah, I see. You could also move this code into the companion object of the CreateMacroCommand. That woud also work. It is just that this code isn't parser specific.

if (colToIndex.size != arguments.size) {
throw operationNotAllowed(
s"Cannot support duplicate colNames for CREATE TEMPORARY MACRO ", ctx)
}
val macroFunction = expression(ctx.expression).transformUp {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ditto

case u: UnresolvedAttribute =>
val index = colToIndex.get(u.name).getOrElse(
throw new ParseException(
s"Cannot find colName: [${u}] for CREATE TEMPORARY MACRO", ctx))
BoundReference(index, arguments(index).dataType, arguments(index).nullable)
case _: SubqueryExpression =>
throw operationNotAllowed(s"Cannot support Subquery for CREATE TEMPORARY MACRO", ctx)
}

CreateMacroCommand(
ctx.macroName.getText,
arguments,
macroFunction)
}

/**
* Create a [[DropMacroCommand]] command.
*
* For example:
* {{{
* DROP TEMPORARY MACRO [IF EXISTS] macro_name;
* }}}
*/
override def visitDropMacro(ctx: DropMacroContext): LogicalPlan = withOrigin(ctx) {
DropMacroCommand(
ctx.macroName.getText,
ctx.EXISTS != null)
}

/**
* Create a [[DropTableCommand]] command.
*/
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,69 @@
/*
* 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, SparkSession}
import org.apache.spark.sql.catalyst.expressions._

/**
* The DDL command that creates a macro.
* To create a temporary macro, the syntax of using this command in SQL is:
* {{{
* CREATE TEMPORARY MACRO macro_name([col_name col_type, ...]) expression;
* }}}
*/
case class CreateMacroCommand(
macroName: String,
columns: Seq[AttributeReference],
macroFunction: Expression)
extends RunnableCommand {

override def run(sparkSession: SparkSession): Seq[Row] = {
val catalog = sparkSession.sessionState.catalog
val macroInfo = columns.mkString(",") + " -> " + macroFunction.toString
val info = new ExpressionInfo(macroInfo, macroName)
val builder = (children: Seq[Expression]) => {
if (children.size != columns.size) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It is slightly better to columns.size in a separate variable, so we do not include columns in the closure.

throw new AnalysisException(s"Actual number of columns: ${children.size} != " +
s"expected number of columns: ${columns.size} for Macro $macroName")
}
macroFunction.transformUp {
case b: BoundReference => children(b.ordinal)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We do not validate the input type here. This would be entirely fine if macro arguments were defined without a DataType. I am not sure what we need to do here though. We have two options:

  • Ignore the DataType and rely on the expressions inputTypes to get casting done. This must be documented though.
  • Introduce casts to make sure the input conforms to the required input.

What do you think?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@hvanhovell good points. Because Analyzer will check expression's checkInputDataTypes after ResolveFunctions, I think we do not validate input type here. Now i do not think it has benefits if we did casts, but it maybe cause unnecessary casts. I will add some comments for it. Thanks.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ok that is perfect.

}
}
catalog.createTempMacro(macroName, info, builder)
Seq.empty[Row]
}
}

/**
* The DDL command that drops a macro.
* ifExists: returns an error if the macro doesn't exist, unless this is true.
* {{{
* DROP TEMPORARY MACRO [IF EXISTS] macro_name;
* }}}
*/
case class DropMacroCommand(macroName: String, ifExists: Boolean)
extends RunnableCommand {

override def run(sparkSession: SparkSession): Seq[Row] = {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This will drop any function... Can we make it Macro specific?

val catalog = sparkSession.sessionState.catalog
catalog.dropTempMacro(macroName, ifExists)
Seq.empty[Row]
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -1309,4 +1309,32 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach {
assertUnsupported("TRUNCATE TABLE my_tab PARTITION (age=10)")
}

test("create/drop temporary macro") {
intercept[AnalysisException] {
sql(s"CREATE TEMPORARY MACRO simple_add_error(x int) x + y")
}
intercept[AnalysisException] {
sql(s"CREATE TEMPORARY MACRO simple_add_error(x int, x int) x + y")
}
intercept[AnalysisException] {
sql(s"CREATE TEMPORARY MACRO simple_add_error(x int) x NOT IN (select c2 from t2) ")
}
sql("CREATE TEMPORARY MACRO fixed_number() 42")
checkAnswer(sql("SELECT fixed_number()"), Row(42))
sql("CREATE TEMPORARY MACRO string_len_plus_two(x string) length(x) + 2")
checkAnswer(sql("SELECT string_len_plus_two('abc')"), Row(5))
sql("CREATE TEMPORARY MACRO simple_add(x int, y int) x + y")
checkAnswer(sql("SELECT simple_add(1, 2)"), Row(3))
intercept[AnalysisException] {
sql(s"SELECT simple_add(1)")
}
sql("DROP TEMPORARY MACRO fixed_number")
intercept[AnalysisException] {
sql(s"DROP TEMPORARY MACRO abs")
}
intercept[AnalysisException] {
sql("DROP TEMPORARY MACRO SOME_MACRO")
}
sql("DROP TEMPORARY MACRO IF EXISTS SOME_MACRO")
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -1205,13 +1205,6 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter {
assertUnsupportedFeature {
sql("ALTER INDEX my_index ON my_table set IDXPROPERTIES (\"prop1\"=\"val1_new\")")}
}

test("create/drop macro commands are not supported") {
assertUnsupportedFeature {
sql("CREATE TEMPORARY MACRO SIGMOID (x DOUBLE) 1.0 / (1.0 + EXP(-x))")
}
assertUnsupportedFeature { sql("DROP TEMPORARY MACRO SIGMOID") }
}
}

// for SPARK-2180 test
Expand Down