Skip to content

Commit

Permalink
[SPARK-14591][SQL] Remove DataTypeParser and add more keywords to the…
Browse files Browse the repository at this point in the history
… nonReserved list.

## What changes were proposed in this pull request?
CatalystSqlParser can parse data types. So, we do not need to have an individual DataTypeParser.

## How was this patch tested?
Existing tests

Author: Yin Huai <yhuai@databricks.com>

Closes #12796 from yhuai/removeDataTypeParser.
  • Loading branch information
yhuai authored and rxin committed Apr 30, 2016
1 parent 7945f9f commit ac41fc6
Show file tree
Hide file tree
Showing 9 changed files with 26 additions and 232 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ import javax.annotation.Nullable
import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier}
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
import org.apache.spark.sql.catalyst.parser.DataTypeParser
import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan}


Expand Down Expand Up @@ -189,7 +189,7 @@ case class SimpleCatalogRelation(
(cols ++ catalogTable.partitionColumns).map { f =>
AttributeReference(
f.name,
DataTypeParser.parse(f.dataType),
CatalystSqlParser.parseDataType(f.dataType),
// Since data can be dumped in randomly with no validation, everything is nullable.
nullable = true
)(qualifier = Some(alias.getOrElse(metadata.identifier.table)))
Expand Down

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ import org.json4s.JsonDSL._
import org.apache.spark.SparkException
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, InterpretedOrdering}
import org.apache.spark.sql.catalyst.parser.{DataTypeParser, LegacyTypeStringParser}
import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, LegacyTypeStringParser}
import org.apache.spark.sql.catalyst.util.quoteIdentifier

/**
Expand Down Expand Up @@ -169,7 +169,7 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru
* }}}
*/
def add(name: String, dataType: String): StructType = {
add(name, DataTypeParser.parse(dataType), nullable = true, Metadata.empty)
add(name, CatalystSqlParser.parseDataType(dataType), nullable = true, Metadata.empty)
}

/**
Expand All @@ -184,7 +184,7 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru
* }}}
*/
def add(name: String, dataType: String, nullable: Boolean): StructType = {
add(name, DataTypeParser.parse(dataType), nullable, Metadata.empty)
add(name, CatalystSqlParser.parseDataType(dataType), nullable, Metadata.empty)
}

/**
Expand All @@ -202,7 +202,7 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru
dataType: String,
nullable: Boolean,
metadata: Metadata): StructType = {
add(name, DataTypeParser.parse(dataType), nullable, metadata)
add(name, CatalystSqlParser.parseDataType(dataType), nullable, metadata)
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,17 +20,18 @@ package org.apache.spark.sql.catalyst.parser
import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.types._

abstract class AbstractDataTypeParserSuite extends SparkFunSuite {
class CatalystQlDataTypeParserSuite extends SparkFunSuite {

def parse(sql: String): DataType
def parse(sql: String): DataType = CatalystSqlParser.parseDataType(sql)

def checkDataType(dataTypeString: String, expectedDataType: DataType): Unit = {
test(s"parse ${dataTypeString.replace("\n", "")}") {
assert(parse(dataTypeString) === expectedDataType)
}
}

def intercept(sql: String)
def intercept(sql: String): Unit =
intercept[ParseException](CatalystSqlParser.parseDataType(sql))

def unsupported(dataTypeString: String): Unit = {
test(s"$dataTypeString is not supported") {
Expand Down Expand Up @@ -115,38 +116,16 @@ abstract class AbstractDataTypeParserSuite extends SparkFunSuite {
unsupported("it is not a data type")
unsupported("struct<x+y: int, 1.1:timestamp>")
unsupported("struct<x: int")
}

class DataTypeParserSuite extends AbstractDataTypeParserSuite {
override def intercept(sql: String): Unit =
intercept[DataTypeException](DataTypeParser.parse(sql))

override def parse(sql: String): DataType =
DataTypeParser.parse(sql)

// A column name can be a reserved word in our DDL parser and SqlParser.
// DataType parser accepts certain reserved keywords.
checkDataType(
"Struct<TABLE: string, CASE:boolean>",
"Struct<TABLE: string, DATE:boolean>",
StructType(
StructField("TABLE", StringType, true) ::
StructField("CASE", BooleanType, true) :: Nil)
StructField("DATE", BooleanType, true) :: Nil)
)

unsupported("struct<x int, y string>")

unsupported("struct<`x``y` int>")
}

class CatalystQlDataTypeParserSuite extends AbstractDataTypeParserSuite {
override def intercept(sql: String): Unit =
intercept[ParseException](CatalystSqlParser.parseDataType(sql))

override def parse(sql: String): DataType =
CatalystSqlParser.parseDataType(sql)

// A column name can be a reserved word in our DDL parser and SqlParser.
unsupported("Struct<TABLE: string, CASE:boolean>")

// Define struct columns without ':'
checkDataType(
"struct<x int, y string>",
(new StructType).add("x", IntegerType).add("y", StringType))
Expand Down
4 changes: 2 additions & 2 deletions sql/core/src/main/scala/org/apache/spark/sql/Column.scala
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ import org.apache.spark.internal.Logging
import org.apache.spark.sql.catalyst.analysis._
import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder}
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.parser.DataTypeParser
import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
import org.apache.spark.sql.catalyst.util.usePrettyExpression
import org.apache.spark.sql.execution.aggregate.TypedAggregateExpression
import org.apache.spark.sql.functions.lit
Expand Down Expand Up @@ -979,7 +979,7 @@ class Column(protected[sql] val expr: Expression) extends Logging {
* @group expr_ops
* @since 1.3.0
*/
def cast(to: String): Column = cast(DataTypeParser.parse(to))
def cast(to: String): Column = cast(CatalystSqlParser.parseDataType(to))

/**
* Returns an ordering used in sorting.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@ import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
import org.apache.spark.sql.catalyst.catalog._
import org.apache.spark.sql.catalyst.expressions.{AttributeMap, AttributeReference, Expression}
import org.apache.spark.sql.catalyst.parser.DataTypeParser
import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
import org.apache.spark.sql.execution.FileRelation
import org.apache.spark.sql.hive.client.HiveClient
Expand Down Expand Up @@ -188,7 +188,7 @@ private[hive] case class MetastoreRelation(
implicit class SchemaAttribute(f: CatalogColumn) {
def toAttribute: AttributeReference = AttributeReference(
f.name,
DataTypeParser.parse(f.dataType),
CatalystSqlParser.parseDataType(f.dataType),
// Since data can be dumped in randomly with no validation, everything is nullable.
nullable = true
)(qualifier = Some(alias.getOrElse(tableName)))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector

import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.internal.Logging
import org.apache.spark.sql.catalyst.parser.DataTypeParser
import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
import org.apache.spark.sql.types.StructType

private[orc] object OrcFileOperator extends Logging {
Expand Down Expand Up @@ -78,7 +78,7 @@ private[orc] object OrcFileOperator extends Logging {
val readerInspector = reader.getObjectInspector.asInstanceOf[StructObjectInspector]
val schema = readerInspector.getTypeName
logDebug(s"Reading schema from file $paths, got Hive schema string: $schema")
DataTypeParser.parse(schema).asInstanceOf[StructType]
CatalystSqlParser.parseDataType(schema).asInstanceOf[StructType]
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ import java.io.File
import org.apache.spark.sql.{QueryTest, Row, SaveMode}
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.catalog.CatalogTableType
import org.apache.spark.sql.catalyst.parser.DataTypeParser
import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
import org.apache.spark.sql.hive.test.TestHiveSingleton
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.test.{ExamplePointUDT, SQLTestUtils}
Expand All @@ -33,7 +33,7 @@ class HiveMetastoreCatalogSuite extends TestHiveSingleton {

test("struct field should accept underscore in sub-column name") {
val hiveTypeStr = "struct<a: int, b_1: string, c: string>"
val dateType = DataTypeParser.parse(hiveTypeStr)
val dateType = CatalystSqlParser.parseDataType(hiveTypeStr)
assert(dateType.isInstanceOf[StructType])
}

Expand Down

0 comments on commit ac41fc6

Please sign in to comment.