Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
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
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import scala.util.parsing.combinator.syntactical.StandardTokenParsers
import scala.util.parsing.combinator.PackratParsers
import scala.util.parsing.input.CharArrayReader.EofCh

import org.apache.spark.sql.catalyst.parser.ParserInterface
import org.apache.spark.sql.catalyst.plans.logical._

private[sql] abstract class AbstractSparkSQLParser
Copy link
Contributor

Choose a reason for hiding this comment

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

This class seems to belong to your new parser package. I would move it there instead of changing the import here. Also, CatalystQlSuite needs to be moved since CatalystQl is now part of the parser package.

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ import java.util.concurrent.ConcurrentHashMap

import scala.collection.JavaConverters._

import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.{AnalysisException, Row}
import org.apache.spark.sql.catalyst.{CatalystConf, EmptyConf, TableIdentifier}
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}

Expand All @@ -41,6 +41,10 @@ trait Catalog {
throw new UnsupportedOperationException
}

def runNativeCommand(sql: String): 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.

Running commands shouldn't be part of the catalog interface. This is only used in 1 place anyway so I wouldn't bother making an abstraction here.

throw new UnsupportedOperationException
}

/**
* Returns tuples of (tableName, isTemporary) for all tables in the given database.
* isTemporary is a Boolean value indicates if a table is a temporary or not.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,11 +14,12 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.sql.catalyst
package org.apache.spark.sql.catalyst.parser

import java.sql.Date

import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.analysis._
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.aggregate.Count
Expand All @@ -33,14 +34,8 @@ import org.apache.spark.util.random.RandomSampler
/**
* This class translates SQL to Catalyst [[LogicalPlan]]s or [[Expression]]s.
*/
private[sql] class CatalystQl(val conf: ParserConf = SimpleParserConf()) extends ParserInterface {
object Token {
def unapply(node: ASTNode): Some[(String, List[ASTNode])] = {
CurrentOrigin.setPosition(node.line, node.positionInLine)
node.pattern
}
}

private[sql] class CatalystQl(conf: ParserConf = SimpleParserConf()) extends ParserInterface {
import ParserSupport._
/**
* The safeParse method allows a user to focus on the parsing/AST transformation logic. This
* method will take care of possible errors during the parsing process.
Expand Down Expand Up @@ -90,92 +85,13 @@ private[sql] class CatalystQl(val conf: ParserConf = SimpleParserConf()) extends
}
}

protected def getClauses(
clauseNames: Seq[String],
nodeList: Seq[ASTNode]): Seq[Option[ASTNode]] = {
var remainingNodes = nodeList
val clauses = clauseNames.map { clauseName =>
val (matches, nonMatches) = remainingNodes.partition(_.text.toUpperCase == clauseName)
remainingNodes = nonMatches ++ (if (matches.nonEmpty) matches.tail else Nil)
matches.headOption
}

if (remainingNodes.nonEmpty) {
sys.error(
s"""Unhandled clauses: ${remainingNodes.map(_.treeString).mkString("\n")}.
|You are likely trying to use an unsupported Hive feature."""".stripMargin)
}
clauses
}

protected def getClause(clauseName: String, nodeList: Seq[ASTNode]): ASTNode =
getClauseOption(clauseName, nodeList).getOrElse(sys.error(
s"Expected clause $clauseName missing from ${nodeList.map(_.treeString).mkString("\n")}"))

protected def getClauseOption(clauseName: String, nodeList: Seq[ASTNode]): Option[ASTNode] = {
nodeList.filter { case ast: ASTNode => ast.text == clauseName } match {
case Seq(oneMatch) => Some(oneMatch)
case Seq() => None
case _ => sys.error(s"Found multiple instances of clause $clauseName")
}
}

protected def nodeToAttribute(node: ASTNode): Attribute = node match {
def nodeToAttribute(node: ASTNode): Attribute = node match {
Copy link
Contributor

Choose a reason for hiding this comment

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

only used in 1 place. Can you make it private?

Copy link
Contributor

Choose a reason for hiding this comment

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

actually, the only usage is parseDdl, which is not used anywhere. Would you mind deleting this method and parseDdl to simplify the code?

case Token("TOK_TABCOL", Token(colName, Nil) :: dataType :: Nil) =>
AttributeReference(colName, nodeToDataType(dataType), nullable = true)()
case _ =>
noParseRule("Attribute", node)
}

protected def nodeToDataType(node: ASTNode): DataType = node match {
case Token("TOK_DECIMAL", precision :: scale :: Nil) =>
DecimalType(precision.text.toInt, scale.text.toInt)
case Token("TOK_DECIMAL", precision :: Nil) =>
DecimalType(precision.text.toInt, 0)
case Token("TOK_DECIMAL", Nil) => DecimalType.USER_DEFAULT
case Token("TOK_BIGINT", Nil) => LongType
case Token("TOK_INT", Nil) => IntegerType
case Token("TOK_TINYINT", Nil) => ByteType
case Token("TOK_SMALLINT", Nil) => ShortType
case Token("TOK_BOOLEAN", Nil) => BooleanType
case Token("TOK_STRING", Nil) => StringType
case Token("TOK_VARCHAR", Token(_, Nil) :: Nil) => StringType
case Token("TOK_CHAR", Token(_, Nil) :: Nil) => StringType
case Token("TOK_FLOAT", Nil) => FloatType
case Token("TOK_DOUBLE", Nil) => DoubleType
case Token("TOK_DATE", Nil) => DateType
case Token("TOK_TIMESTAMP", Nil) => TimestampType
case Token("TOK_BINARY", Nil) => BinaryType
case Token("TOK_LIST", elementType :: Nil) => ArrayType(nodeToDataType(elementType))
case Token("TOK_STRUCT", Token("TOK_TABCOLLIST", fields) :: Nil) =>
StructType(fields.map(nodeToStructField))
case Token("TOK_MAP", keyType :: valueType :: Nil) =>
MapType(nodeToDataType(keyType), nodeToDataType(valueType))
case _ =>
noParseRule("DataType", node)
}

protected def nodeToStructField(node: ASTNode): StructField = node match {
case Token("TOK_TABCOL", Token(fieldName, Nil) :: dataType :: Nil) =>
StructField(cleanIdentifier(fieldName), nodeToDataType(dataType), nullable = true)
case Token("TOK_TABCOL", Token(fieldName, Nil) :: dataType :: comment :: Nil) =>
val meta = new MetadataBuilder().putString("comment", unquoteString(comment.text)).build()
StructField(cleanIdentifier(fieldName), nodeToDataType(dataType), nullable = true, meta)
case _ =>
noParseRule("StructField", node)
}

protected def extractTableIdent(tableNameParts: ASTNode): TableIdentifier = {
tableNameParts.children.map {
case Token(part, Nil) => cleanIdentifier(part)
} match {
case Seq(tableOnly) => TableIdentifier(tableOnly)
case Seq(databaseName, table) => TableIdentifier(table, Some(databaseName))
case other => sys.error("Hive only supports tables names like 'tableName' " +
s"or 'databaseName.tableName', found '$other'")
}
}

/**
* SELECT MAX(value) FROM src GROUP BY k1, k2, k3 GROUPING SETS((k1, k2), (k2))
* is equivalent to
Expand Down Expand Up @@ -623,22 +539,6 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C
noParseRule("Select", node)
}

protected val escapedIdentifier = "`(.+)`".r
protected val doubleQuotedString = "\"([^\"]+)\"".r
protected val singleQuotedString = "'([^']+)'".r

protected def unquoteString(str: String) = str match {
case singleQuotedString(s) => s
case doubleQuotedString(s) => s
case other => other
}

/** Strips backticks from ident if present */
protected def cleanIdentifier(ident: String): String = ident match {
case escapedIdentifier(i) => i
case plainIdent => plainIdent
}

/* Case insensitive matches */
val COUNT = "(?i)COUNT".r
val SUM = "(?i)SUM".r
Expand Down Expand Up @@ -1004,7 +904,4 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C
}

protected def nodeToGenerator(node: ASTNode): Generator = noParseRule("Generator", node)

protected def noParseRule(msg: String, node: ASTNode): Nothing = throw new NotImplementedError(
s"[$msg]: No parse rules for ASTNode type: ${node.tokenType}, tree:\n${node.treeString}")
}
Original file line number Diff line number Diff line change
Expand Up @@ -15,8 +15,9 @@
* limitations under the License.
*/

package org.apache.spark.sql.catalyst
package org.apache.spark.sql.catalyst.parser

import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.expressions.Expression
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,134 @@
/*
* 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.parser

import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.catalyst.trees.CurrentOrigin
import org.apache.spark.sql.types._

object ParserSupport {
Copy link
Contributor

Choose a reason for hiding this comment

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

this is more like ParserUtils

protected val escapedIdentifier = "`(.+)`".r
protected val doubleQuotedString = "\"([^\"]+)\"".r
protected val singleQuotedString = "'([^']+)'".r
Copy link
Contributor

Choose a reason for hiding this comment

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

these can all be private


private[sql] def unquoteString(str: String): String = str match {
case singleQuotedString(s) => s
case doubleQuotedString(s) => s
case other => other
}

/** Strips backticks from ident if present */
private[sql] def cleanIdentifier(ident: String): String = ident match {
case escapedIdentifier(i) => i
case plainIdent => plainIdent
}

private[sql] def cleanAndUnquoteString(str: String): String = {
cleanIdentifier(unquoteString(str))
}

object Token {
Copy link
Contributor

Choose a reason for hiding this comment

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

can you move this to the top so it's not hidden among other methods?

def unapply(node: ASTNode): Some[(String, List[ASTNode])] = {
CurrentOrigin.setPosition(node.line, node.positionInLine)
node.pattern
}
}

def getClauses(
clauseNames: Seq[String],
nodeList: Seq[ASTNode]): Seq[Option[ASTNode]] = {
var remainingNodes = nodeList
val clauses = clauseNames.map { clauseName =>
val (matches, nonMatches) = remainingNodes.partition(_.text.toUpperCase == clauseName)
remainingNodes = nonMatches ++ (if (matches.nonEmpty) matches.tail else Nil)
matches.headOption
}

if (remainingNodes.nonEmpty) {
sys.error(
s"""Unhandled clauses: ${remainingNodes.map(_.treeString).mkString("\n")}.
|You are likely trying to use an unsupported Hive feature."""".stripMargin)
}
clauses
}

def getClause(clauseName: String, nodeList: Seq[ASTNode]): ASTNode =
getClauseOption(clauseName, nodeList).getOrElse(sys.error(
s"Expected clause $clauseName missing from ${nodeList.map(_.treeString).mkString("\n")}"))

def getClauseOption(clauseName: String, nodeList: Seq[ASTNode]): Option[ASTNode] = {
nodeList.filter { case ast: ASTNode => ast.text == clauseName } match {
case Seq(oneMatch) => Some(oneMatch)
case Seq() => None
case _ => sys.error(s"Found multiple instances of clause $clauseName")
}
}

def extractTableIdent(tableNameParts: ASTNode): TableIdentifier = {
tableNameParts.children.map {
case Token(part, Nil) => cleanIdentifier(part)
} match {
case Seq(tableOnly) => TableIdentifier(tableOnly)
case Seq(databaseName, table) => TableIdentifier(table, Some(databaseName))
case other => sys.error("Hive only supports tables names like 'tableName' " +
s"or 'databaseName.tableName', found '$other'")
}
}

def nodeToDataType(node: ASTNode): DataType = node match {
case Token("TOK_DECIMAL", precision :: scale :: Nil) =>
DecimalType(precision.text.toInt, scale.text.toInt)
case Token("TOK_DECIMAL", precision :: Nil) =>
DecimalType(precision.text.toInt, 0)
case Token("TOK_DECIMAL", Nil) => DecimalType.USER_DEFAULT
case Token("TOK_BIGINT", Nil) => LongType
case Token("TOK_INT", Nil) => IntegerType
case Token("TOK_TINYINT", Nil) => ByteType
case Token("TOK_SMALLINT", Nil) => ShortType
case Token("TOK_BOOLEAN", Nil) => BooleanType
case Token("TOK_STRING", Nil) => StringType
case Token("TOK_VARCHAR", Token(_, Nil) :: Nil) => StringType
case Token("TOK_CHAR", Token(_, Nil) :: Nil) => StringType
case Token("TOK_FLOAT", Nil) => FloatType
case Token("TOK_DOUBLE", Nil) => DoubleType
case Token("TOK_DATE", Nil) => DateType
case Token("TOK_TIMESTAMP", Nil) => TimestampType
case Token("TOK_BINARY", Nil) => BinaryType
case Token("TOK_LIST", elementType :: Nil) => ArrayType(nodeToDataType(elementType))
case Token("TOK_STRUCT", Token("TOK_TABCOLLIST", fields) :: Nil) =>
StructType(fields.map(nodeToStructField))
case Token("TOK_MAP", keyType :: valueType :: Nil) =>
MapType(nodeToDataType(keyType), nodeToDataType(valueType))
case _ =>
noParseRule("DataType", node)
}

def nodeToStructField(node: ASTNode): StructField = node match {
case Token("TOK_TABCOL", Token(fieldName, Nil) :: dataType :: Nil) =>
StructField(cleanIdentifier(fieldName), nodeToDataType(dataType), nullable = true)
case Token("TOK_TABCOL", Token(fieldName, Nil) :: dataType :: comment :: Nil) =>
val meta = new MetadataBuilder().putString("comment", unquoteString(comment.text)).build()
StructField(cleanIdentifier(fieldName), nodeToDataType(dataType), nullable = true, meta)
case _ =>
noParseRule("StructField", node)
}

def noParseRule(msg: String, node: ASTNode): Nothing = throw new NotImplementedError(
s"[$msg]: No parse rules for ASTNode type: ${node.tokenType}, tree:\n${node.treeString}")
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst
import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.analysis._
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.parser.CatalystQl
import org.apache.spark.sql.catalyst.plans.PlanTest
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.unsafe.types.CalendarInterval
Expand Down
3 changes: 2 additions & 1 deletion sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,8 @@ import org.apache.spark.sql.catalyst.optimizer.CombineUnions
import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.util.usePrettyExpression
import org.apache.spark.sql.execution.{ExplainCommand, FileRelation, LogicalRDD, Queryable, QueryExecution, SQLExecution}
import org.apache.spark.sql.execution.{FileRelation, LogicalRDD, Queryable, QueryExecution, SQLExecution}
import org.apache.spark.sql.execution.commands.ExplainCommand
import org.apache.spark.sql.execution.datasources.{CreateTableUsingAsSelect, LogicalRelation}
import org.apache.spark.sql.execution.datasources.json.JacksonGenerator
import org.apache.spark.sql.execution.python.EvaluatePython
Expand Down
2 changes: 2 additions & 0 deletions sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
Original file line number Diff line number Diff line change
Expand Up @@ -35,9 +35,11 @@ import org.apache.spark.sql.catalyst.analysis._
import org.apache.spark.sql.catalyst.encoders.encoderFor
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.optimizer.Optimizer
import org.apache.spark.sql.catalyst.parser.ParserInterface
import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, Range}
import org.apache.spark.sql.catalyst.rules.RuleExecutor
import org.apache.spark.sql.execution._
import org.apache.spark.sql.execution.commands._
import org.apache.spark.sql.execution.datasources._
import org.apache.spark.sql.execution.ui.{SQLListener, SQLTab}
import org.apache.spark.sql.internal.{SessionState, SQLConf}
Expand Down
Loading