Skip to content
Closed
Original file line number Diff line number Diff line change
Expand Up @@ -167,8 +167,8 @@ intervalLiteral
((intervalConstant KW_HOUR)=> hour=intervalConstant KW_HOUR)?
((intervalConstant KW_MINUTE)=> minute=intervalConstant KW_MINUTE)?
((intervalConstant KW_SECOND)=> second=intervalConstant KW_SECOND)?
(millisecond=intervalConstant KW_MILLISECOND)?
(microsecond=intervalConstant KW_MICROSECOND)?
((intervalConstant KW_MILLISECOND)=> millisecond=intervalConstant KW_MILLISECOND)?
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 is a leftover chore from a previous PR.

((intervalConstant KW_MICROSECOND)=> microsecond=intervalConstant KW_MICROSECOND)?
-> ^(TOK_INTERVAL
^(TOK_INTERVAL_YEAR_LITERAL $year?)
^(TOK_INTERVAL_MONTH_LITERAL $month?)
Expand Down Expand Up @@ -505,10 +505,8 @@ identifier
functionIdentifier
@init { gParent.pushMsg("function identifier", state); }
@after { gParent.popMsg(state); }
: db=identifier DOT fn=identifier
-> Identifier[$db.text + "." + $fn.text]
|
identifier
:
identifier (DOT identifier)? -> identifier+
;

principalIdentifier
Expand Down Expand Up @@ -553,6 +551,8 @@ nonReserved
| KW_SNAPSHOT
| KW_AUTOCOMMIT
| KW_ANTI
| KW_WEEK | KW_MILLISECOND | KW_MICROSECOND
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 is a leftover chore from a previous PR.

| KW_CLEAR | KW_LAZY | KW_CACHE | KW_UNCACHE | KW_DFS
;

//The following SQL2011 reserved keywords are used as cast function name only, but not as identifiers.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -327,6 +327,11 @@ KW_AUTOCOMMIT: 'AUTOCOMMIT';
KW_WEEK: 'WEEK'|'WEEKS';
KW_MILLISECOND: 'MILLISECOND'|'MILLISECONDS';
KW_MICROSECOND: 'MICROSECOND'|'MICROSECONDS';
KW_CLEAR: 'CLEAR';
KW_LAZY: 'LAZY';
KW_CACHE: 'CACHE';
KW_UNCACHE: 'UNCACHE';
KW_DFS: 'DFS';

// Operators
// NOTE: if you add a new function/operator, add it to sysFuncNames so that describe function _FUNC_ will work.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -371,6 +371,13 @@ TOK_TXN_READ_WRITE;
TOK_COMMIT;
TOK_ROLLBACK;
TOK_SET_AUTOCOMMIT;
TOK_CACHETABLE;
TOK_UNCACHETABLE;
TOK_CLEARCACHE;
TOK_SETCONFIG;
TOK_DFS;
TOK_ADDFILE;
TOK_ADDJAR;
}


Expand Down Expand Up @@ -515,6 +522,11 @@ import java.util.HashMap;
xlateMap.put("KW_WEEK", "WEEK");
xlateMap.put("KW_MILLISECOND", "MILLISECOND");
xlateMap.put("KW_MICROSECOND", "MICROSECOND");
xlateMap.put("KW_CLEAR", "CLEAR");
xlateMap.put("KW_LAZY", "LAZY");
xlateMap.put("KW_CACHE", "CACHE");
xlateMap.put("KW_UNCACHE", "UNCACHE");
xlateMap.put("KW_DFS", "DFS");

// Operators
xlateMap.put("DOT", ".");
Expand Down Expand Up @@ -687,8 +699,12 @@ catch (RecognitionException e) {

// starting rule
statement
: explainStatement EOF
| execStatement EOF
: explainStatement EOF
| execStatement EOF
| KW_ADD KW_JAR -> ^(TOK_ADDJAR)
Copy link
Contributor Author

Choose a reason for hiding this comment

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

These rules do not match until EOF. We will use the raw input after we encouter such a token.

| KW_ADD KW_FILE -> ^(TOK_ADDFILE)
| KW_DFS -> ^(TOK_DFS)
| (KW_SET)=> KW_SET -> ^(TOK_SETCONFIG)
;

explainStatement
Expand Down Expand Up @@ -717,6 +733,7 @@ execStatement
| deleteStatement
| updateStatement
| sqlTransactionStatement
| cacheStatement
;

loadStatement
Expand Down Expand Up @@ -1390,7 +1407,7 @@ showStatement
@init { pushMsg("show statement", state); }
@after { popMsg(state); }
: KW_SHOW (KW_DATABASES|KW_SCHEMAS) (KW_LIKE showStmtIdentifier)? -> ^(TOK_SHOWDATABASES showStmtIdentifier?)
| KW_SHOW KW_TABLES ((KW_FROM|KW_IN) db_name=identifier)? (KW_LIKE showStmtIdentifier|showStmtIdentifier)? -> ^(TOK_SHOWTABLES (TOK_FROM $db_name)? showStmtIdentifier?)
| KW_SHOW KW_TABLES ((KW_FROM|KW_IN) db_name=identifier)? (KW_LIKE showStmtIdentifier|showStmtIdentifier)? -> ^(TOK_SHOWTABLES ^(TOK_FROM $db_name)? showStmtIdentifier?)
| KW_SHOW KW_COLUMNS (KW_FROM|KW_IN) tableName ((KW_FROM|KW_IN) db_name=identifier)?
-> ^(TOK_SHOWCOLUMNS tableName $db_name?)
| KW_SHOW KW_FUNCTIONS (KW_LIKE showFunctionIdentifier|showFunctionIdentifier)? -> ^(TOK_SHOWFUNCTIONS KW_LIKE? showFunctionIdentifier?)
Expand Down Expand Up @@ -2438,12 +2455,11 @@ BEGIN user defined transaction boundaries; follows SQL 2003 standard exactly exc
sqlTransactionStatement
@init { pushMsg("transaction statement", state); }
@after { popMsg(state); }
:
startTransactionStatement
| commitStatement
| rollbackStatement
| setAutoCommitStatement
;
: startTransactionStatement
| commitStatement
| rollbackStatement
| setAutoCommitStatement
;

startTransactionStatement
:
Expand Down Expand Up @@ -2489,3 +2505,31 @@ setAutoCommitStatement
/*
END user defined transaction boundaries
*/

/*
Table Caching statements.
*/
cacheStatement
@init { pushMsg("cache statement", state); }
@after { popMsg(state); }
:
cacheTableStatement
| uncacheTableStatement
| clearCacheStatement
;

cacheTableStatement
:
KW_CACHE (lazy=KW_LAZY)? KW_TABLE identifier (KW_AS selectStatementWithCTE)? -> ^(TOK_CACHETABLE identifier $lazy? selectStatementWithCTE?)
;

uncacheTableStatement
:
KW_UNCACHE KW_TABLE identifier -> ^(TOK_UNCACHETABLE identifier)
;

clearCacheStatement
:
KW_CLEAR KW_CACHE -> ^(TOK_CLEARCACHE)
;

Original file line number Diff line number Diff line change
Expand Up @@ -210,6 +210,28 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C
}

protected def nodeToPlan(node: ASTNode): LogicalPlan = node match {
case Token("TOK_SHOWFUNCTIONS", args) =>
// Skip LIKE.
val pattern = args match {
case like :: nodes if like.text.toUpperCase == "LIKE" => nodes
case nodes => nodes
}

// Extract Database and Function name
pattern match {
case Nil =>
ShowFunctions(None, None)
case Token(name, Nil) :: Nil =>
ShowFunctions(None, Some(unquoteString(name)))
case Token(db, Nil) :: Token(name, Nil) :: Nil =>
ShowFunctions(Some(unquoteString(db)), Some(unquoteString(name)))
case _ =>
noParseRule("SHOW FUNCTIONS", node)
}

case Token("TOK_DESCFUNCTION", Token(functionName, Nil) :: isExtended) =>
DescribeFunction(functionName, isExtended.nonEmpty)

case Token("TOK_QUERY", queryArgs @ Token("TOK_CTE" | "TOK_FROM" | "TOK_INSERT", _) :: _) =>
val (fromClause: Option[ASTNode], insertClauses, cteRelations) =
queryArgs match {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,10 +27,10 @@ case class ASTNode(
children: List[ASTNode],
stream: TokenRewriteStream) extends TreeNode[ASTNode] {
/** Cache the number of children. */
val numChildren = children.size
val numChildren: Int = children.size

/** tuple used in pattern matching. */
val pattern = Some((token.getText, children))
val pattern: Some[(String, List[ASTNode])] = Some((token.getText, children))

/** Line in which the ASTNode starts. */
lazy val line: Int = {
Expand All @@ -55,10 +55,16 @@ case class ASTNode(
}

/** Origin of the ASTNode. */
override val origin = Origin(Some(line), Some(positionInLine))
override val origin: Origin = Origin(Some(line), Some(positionInLine))

/** Source text. */
Copy link
Contributor

Choose a reason for hiding this comment

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

this comment isn't super useful - can you explain a bit more what "source text" is?

lazy val source = stream.toString(startIndex, stopIndex)
lazy val source: String = stream.toString(startIndex, stopIndex)

/** Get the source text that remains after this token. */
Copy link
Contributor

Choose a reason for hiding this comment

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

would be great to briefly mention what this is used for (set?)

lazy val remainder: String = {
stream.fill()
stream.toString(stopIndex + 1, stream.size() - 1).trim()
}

def text: String = token.getText

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -203,7 +203,7 @@ class SQLContext private[sql](
protected[sql] lazy val optimizer: Optimizer = new SparkOptimizer(this)

@transient
protected[sql] val sqlParser: ParserInterface = new SparkSQLParser(new SparkQl(conf))
protected[sql] val sqlParser: ParserInterface = new SparkQl(conf)

@transient
protected[sql] val ddlParser: DDLParser = new DDLParser(sqlParser)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,13 +20,26 @@ import org.apache.spark.sql.catalyst.{CatalystQl, TableIdentifier}
import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, SimpleParserConf}
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
import org.apache.spark.sql.catalyst.plans.logical

private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends CatalystQl(conf) {
/** Check if a command should not be explained. */
protected def isNoExplainCommand(command: String): Boolean = "TOK_DESCTABLE" == command

protected override def nodeToPlan(node: ASTNode): LogicalPlan = {
node match {
case Token("TOK_SETCONFIG", Nil) =>
val keyValueSeparatorIndex = node.remainder.indexOf('=')
if (keyValueSeparatorIndex >= 0) {
val key = node.remainder.substring(0, keyValueSeparatorIndex).trim
val value = node.remainder.substring(keyValueSeparatorIndex + 1).trim
SetCommand(Some(key -> Option(value)))
} else if (node.remainder.nonEmpty) {
SetCommand(Some(node.remainder -> None))
} else {
SetCommand(None)
}

// Just fake explain for any of the native commands.
case Token("TOK_EXPLAIN", explainArgs) if isNoExplainCommand(explainArgs.head.text) =>
ExplainCommand(OneRowRelation)
Expand Down Expand Up @@ -75,6 +88,24 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly
}
}

case Token("TOK_CACHETABLE", Token(tableName, Nil) :: args) =>
val Seq(lzy, selectAst) = getClauses(Seq("LAZY", "TOK_QUERY"), args)
CacheTableCommand(tableName, selectAst.map(nodeToPlan), lzy.isDefined)

case Token("TOK_UNCACHETABLE", Token(tableName, Nil) :: Nil) =>
UncacheTableCommand(tableName)

case Token("TOK_CLEARCACHE", Nil) =>
ClearCacheCommand

case Token("TOK_SHOWTABLES", args) =>
val databaseName = args match {
case Nil => None
case Token("TOK_FROM", Token(dbName, Nil) :: Nil) :: Nil => Option(dbName)
case _ => noParseRule("SHOW TABLES", node)
}
ShowTablesCommand(databaseName)

case _ =>
super.nodeToPlan(node)
}
Expand Down

This file was deleted.

Loading