Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
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
Next Next commit
parser
  • Loading branch information
allisonwang-db committed Jun 19, 2024
commit 452387a2b48466f757f049a25342eca1b88a55c8
Original file line number Diff line number Diff line change
Expand Up @@ -146,6 +146,7 @@ BUCKETS: 'BUCKETS';
BY: 'BY';
BYTE: 'BYTE';
CACHE: 'CACHE';
CALLED: 'CALLED';
CASCADE: 'CASCADE';
CASE: 'CASE';
CAST: 'CAST';
Expand All @@ -172,6 +173,7 @@ COMPENSATION: 'COMPENSATION';
COMPUTE: 'COMPUTE';
CONCATENATE: 'CONCATENATE';
CONSTRAINT: 'CONSTRAINT';
CONTAINS: 'CONTAINS';
COST: 'COST';
CREATE: 'CREATE';
CROSS: 'CROSS';
Expand All @@ -198,10 +200,12 @@ DECIMAL: 'DECIMAL';
DECLARE: 'DECLARE';
DEFAULT: 'DEFAULT';
DEFINED: 'DEFINED';
DEFINER: 'DEFINER';
DELETE: 'DELETE';
DELIMITED: 'DELIMITED';
DESC: 'DESC';
DESCRIBE: 'DESCRIBE';
DETERMINISTIC: 'DETERMINISTIC';
DFS: 'DFS';
DIRECTORIES: 'DIRECTORIES';
DIRECTORY: 'DIRECTORY';
Expand Down Expand Up @@ -260,17 +264,20 @@ INDEX: 'INDEX';
INDEXES: 'INDEXES';
INNER: 'INNER';
INPATH: 'INPATH';
INPUT: 'INPUT';
INPUTFORMAT: 'INPUTFORMAT';
INSERT: 'INSERT';
INTERSECT: 'INTERSECT';
INTERVAL: 'INTERVAL';
INT: 'INT';
INTEGER: 'INTEGER';
INTO: 'INTO';
INVOKER: 'INVOKER';
IS: 'IS';
ITEMS: 'ITEMS';
JOIN: 'JOIN';
KEYS: 'KEYS';
LANGUAGE: 'LANGUAGE';
LAST: 'LAST';
LATERAL: 'LATERAL';
LAZY: 'LAZY';
Expand Down Expand Up @@ -298,6 +305,7 @@ MILLISECOND: 'MILLISECOND';
MILLISECONDS: 'MILLISECONDS';
MINUTE: 'MINUTE';
MINUTES: 'MINUTES';
MODIFIES: 'MODIFIES';
MONTH: 'MONTH';
MONTHS: 'MONTHS';
MSCK: 'MSCK';
Expand Down Expand Up @@ -342,6 +350,7 @@ PURGE: 'PURGE';
QUARTER: 'QUARTER';
QUERY: 'QUERY';
RANGE: 'RANGE';
READS: 'READS';
REAL: 'REAL';
RECORDREADER: 'RECORDREADER';
RECORDWRITER: 'RECORDWRITER';
Expand All @@ -356,6 +365,8 @@ REPLACE: 'REPLACE';
RESET: 'RESET';
RESPECT: 'RESPECT';
RESTRICT: 'RESTRICT';
RETURN: 'RETURN';
RETURNS: 'RETURNS';
REVOKE: 'REVOKE';
RIGHT: 'RIGHT';
RLIKE: 'RLIKE' | 'REGEXP';
Expand All @@ -369,6 +380,7 @@ SECOND: 'SECOND';
SECONDS: 'SECONDS';
SCHEMA: 'SCHEMA';
SCHEMAS: 'SCHEMAS';
SECURITY: 'SECURITY';
SELECT: 'SELECT';
SEMI: 'SEMI';
SEPARATED: 'SEPARATED';
Expand All @@ -387,6 +399,8 @@ SOME: 'SOME';
SORT: 'SORT';
SORTED: 'SORTED';
SOURCE: 'SOURCE';
SPECIFIC: 'SPECIFIC';
SQL: 'SQL';
START: 'START';
STATISTICS: 'STATISTICS';
STORED: 'STORED';
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -190,6 +190,11 @@ statement
| CREATE (OR REPLACE)? TEMPORARY? FUNCTION (IF errorCapturingNot EXISTS)?
identifierReference AS className=stringLit
(USING resource (COMMA resource)*)? #createFunction
| CREATE (OR REPLACE)? TEMPORARY? FUNCTION (IF errorCapturingNot EXISTS)?
identifierReference LEFT_PAREN parameters=colDefinitionList? RIGHT_PAREN
(RETURNS (dataType | TABLE LEFT_PAREN returnParams=colTypeList RIGHT_PAREN))?
routineCharacteristics
RETURN (query | expression) #createUserDefinedFunction
| DROP TEMPORARY? FUNCTION (IF EXISTS)? identifierReference #dropFunction
| DECLARE (OR REPLACE)? VARIABLE?
identifierReference dataType? variableDefaultExpression? #createVariable
Expand Down Expand Up @@ -1216,6 +1221,14 @@ createOrReplaceTableColType
: colName=errorCapturingIdentifier dataType colDefinitionOption*
;

colDefinitionList
: colDefinition (COMMA colDefinition)*
;

colDefinition
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: looks like we can remove createOrReplaceTableColType as it's completely the same as this new colDefinition

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Good point. Will create a follow up PR for this.

: colName=errorCapturingIdentifier dataType colDefinitionOption*
;

colDefinitionOption
: errorCapturingNot NULL
| defaultExpression
Expand All @@ -1235,6 +1248,46 @@ complexColType
: errorCapturingIdentifier COLON? dataType (errorCapturingNot NULL)? commentSpec?
;

routineCharacteristics
: (routineLanguage
| specificName
| deterministic
| sqlDataAccess
| nullCall
| commentSpec
| rightsClause)*
;

routineLanguage
: LANGUAGE (SQL | IDENTIFIER)
;

specificName
: SPECIFIC specific=errorCapturingIdentifier
;

deterministic
: DETERMINISTIC
| errorCapturingNot DETERMINISTIC
;

sqlDataAccess
: access=NO SQL
| access=CONTAINS SQL
| access=READS SQL DATA
| access=MODIFIES SQL DATA
;

nullCall
: RETURNS NULL ON NULL INPUT
| CALLED ON NULL INPUT
;

rightsClause
: SQL SECURITY INVOKER
| SQL SECURITY DEFINER
;

whenClause
: WHEN condition=expression THEN result=expression
;
Expand Down Expand Up @@ -1698,6 +1751,7 @@ nonReserved
| BY
| BYTE
| CACHE
| CALLED
| CASCADE
| CASE
| CAST
Expand All @@ -1724,6 +1778,7 @@ nonReserved
| COMPUTE
| CONCATENATE
| CONSTRAINT
| CONTAINS
| COST
| CREATE
| CUBE
Expand All @@ -1749,10 +1804,12 @@ nonReserved
| DECLARE
| DEFAULT
| DEFINED
| DEFINER
| DELETE
| DELIMITED
| DESC
| DESCRIBE
| DETERMINISTIC
| DFS
| DIRECTORIES
| DIRECTORY
Expand Down Expand Up @@ -1808,15 +1865,18 @@ nonReserved
| INDEX
| INDEXES
| INPATH
| INPUT
| INPUTFORMAT
| INSERT
| INT
| INTEGER
| INTERVAL
| INTO
| INVOKER
| IS
| ITEMS
| KEYS
| LANGUAGE
| LAST
| LAZY
| LEADING
Expand All @@ -1843,6 +1903,7 @@ nonReserved
| MILLISECONDS
| MINUTE
| MINUTES
| MODIFIES
| MONTH
| MONTHS
| MSCK
Expand Down Expand Up @@ -1885,6 +1946,7 @@ nonReserved
| QUARTER
| QUERY
| RANGE
| READS
| REAL
| RECORDREADER
| RECORDWRITER
Expand All @@ -1899,6 +1961,8 @@ nonReserved
| RESET
| RESPECT
| RESTRICT
| RETURN
| RETURNS
| REVOKE
| RLIKE
| ROLE
Expand All @@ -1911,6 +1975,7 @@ nonReserved
| SCHEMAS
| SECOND
| SECONDS
| SECURITY
| SELECT
| SEPARATED
| SERDE
Expand All @@ -1927,6 +1992,8 @@ nonReserved
| SORT
| SORTED
| SOURCE
| SPECIFIC
| SQL
| START
| STATISTICS
| STORED
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -574,19 +574,19 @@ private[sql] object QueryParsingErrors extends DataTypeErrorsBase {
ctx)
}

def createFuncWithBothIfNotExistsAndReplaceError(ctx: CreateFunctionContext): Throwable = {
def createFuncWithBothIfNotExistsAndReplaceError(ctx: ParserRuleContext): Throwable = {
new ParseException(
errorClass = "INVALID_SQL_SYNTAX.CREATE_ROUTINE_WITH_IF_NOT_EXISTS_AND_REPLACE",
ctx)
}

def defineTempFuncWithIfNotExistsError(ctx: CreateFunctionContext): Throwable = {
def defineTempFuncWithIfNotExistsError(ctx: ParserRuleContext): Throwable = {
new ParseException(
errorClass = "INVALID_SQL_SYNTAX.CREATE_TEMP_FUNC_WITH_IF_NOT_EXISTS",
ctx)
}

def unsupportedFunctionNameError(funcName: Seq[String], ctx: CreateFunctionContext): Throwable = {
def unsupportedFunctionNameError(funcName: Seq[String], ctx: ParserRuleContext): Throwable = {
new ParseException(
errorClass = "INVALID_SQL_SYNTAX.MULTI_PART_NAME",
messageParameters = Map(
Expand All @@ -597,7 +597,7 @@ private[sql] object QueryParsingErrors extends DataTypeErrorsBase {

def specifyingDBInCreateTempFuncError(
databaseName: String,
ctx: CreateFunctionContext): Throwable = {
ctx: ParserRuleContext): Throwable = {
new ParseException(
errorClass = "INVALID_SQL_SYNTAX.CREATE_TEMP_FUNC_WITH_DATABASE",
messageParameters = Map("database" -> toSQLId(databaseName)),
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,29 @@
/*
* 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

/**
* Supported routine languages for UDFs created via SQL.
*/
sealed trait RoutineLanguage {
def name: String
}

case object LanguageSQL extends RoutineLanguage {
override def name: String = "SQL"
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,33 @@
/*
* 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.errors.QueryErrorsBase

/**
* Errors during registering and executing [[UserDefinedFunction]]s.
*/
object UserDefinedFunctionErrors extends QueryErrorsBase {
def unsupportedUserDefinedFunction(language: RoutineLanguage): Throwable = {
unsupportedUserDefinedFunction(language.name)
}

def unsupportedUserDefinedFunction(language: String): Throwable = {
new IllegalArgumentException(s"Unsupported user defined function type: $language")
Copy link
Contributor

Choose a reason for hiding this comment

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

shall we add an error class?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This should be an internal error since we blocked languages other than SQL in the parser.

}
}
Loading