From b21157db8596342a406dd14832ae18e9632f21fb Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 24 Nov 2020 01:42:53 +0800 Subject: [PATCH] update --- docs/sql-ref-datatypes.md | 2 ++ .../spark/sql/catalyst/analysis/CheckAnalysis.scala | 2 +- .../spark/sql/catalyst/util/CharVarcharUtils.scala | 10 ---------- .../scala/org/apache/spark/sql/types/DataType.scala | 4 +++- .../org/apache/spark/sql/CharVarcharTestSuite.scala | 11 ++++------- 5 files changed, 10 insertions(+), 19 deletions(-) diff --git a/docs/sql-ref-datatypes.md b/docs/sql-ref-datatypes.md index f27f1a0ca967f..322b223d1de4f 100644 --- a/docs/sql-ref-datatypes.md +++ b/docs/sql-ref-datatypes.md @@ -37,6 +37,8 @@ Spark SQL and DataFrames support the following data types: - `DecimalType`: Represents arbitrary-precision signed decimal numbers. Backed internally by `java.math.BigDecimal`. A `BigDecimal` consists of an arbitrary precision integer unscaled value and a 32-bit integer scale. * String type - `StringType`: Represents character string values. + - `VarcharType(length)`: A variant of `StringType` which has a length limitation. Data writing will fail if the input string exceeds the length limitation. Note: this type can only be used in table schema, not functions/operators. + - `CharType(length)`: A variant of `VarcharType(length)` which is fixed length. Data writing will pad the input string if its length is smaller than the char type length. Char type comparison will pad the short one to the longer length. * Binary type - `BinaryType`: Represents byte sequence values. * Boolean type diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index b9696b2d8944b..bf9f720bc693f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -94,7 +94,7 @@ trait CheckAnalysis extends PredicateHelper { case p if p.analyzed => // Skip already analyzed sub-plans - case p if p.output.map(_.dataType).exists(CharVarcharUtils.hasCharVarchar) => + case p if p.resolved && p.output.map(_.dataType).exists(CharVarcharUtils.hasCharVarchar) => throw new IllegalStateException( "[BUG] logical plan should not have output of char/varchar type: " + p) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/CharVarcharUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/CharVarcharUtils.scala index eca6339073992..00a0689243617 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/CharVarcharUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/CharVarcharUtils.scala @@ -92,16 +92,6 @@ object CharVarcharUtils { } } - /** - * Re-construct the original StructType from the type strings in the metadata of StructFields. - * This is needed when dealing with char/varchar columns/fields. - */ - def getRawSchema(schema: StructType): StructType = { - StructType(schema.map { field => - getRawType(field.metadata).map(rawType => field.copy(dataType = rawType)).getOrElse(field) - }) - } - /** * Returns expressions to apply read-side char type padding for the given attributes. String * values should be right-padded to N characters if it's from a CHAR(N) column/field. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala index 6b871c9783471..5f6ebb2f20814 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala @@ -32,6 +32,7 @@ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.Resolver import org.apache.spark.sql.catalyst.expressions.{Cast, Expression} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser +import org.apache.spark.sql.catalyst.util.CharVarcharUtils import org.apache.spark.sql.catalyst.util.DataTypeJsonUtils.{DataTypeJsonDeserializer, DataTypeJsonSerializer} import org.apache.spark.sql.catalyst.util.StringUtils.StringConcat import org.apache.spark.sql.internal.SQLConf @@ -132,7 +133,8 @@ object DataType { ddl, CatalystSqlParser.parseDataType, "Cannot parse the data type: ", - fallbackParser = CatalystSqlParser.parseTableSchema) + fallbackParser = str => CharVarcharUtils.replaceCharVarcharWithString( + CatalystSqlParser.parseTableSchema(str))) } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CharVarcharTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CharVarcharTestSuite.scala index e192a63956232..97e66b2dfe2bc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CharVarcharTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CharVarcharTestSuite.scala @@ -35,13 +35,10 @@ trait CharVarcharTestSuite extends QueryTest with SQLTestUtils { } test("char type values should be padded: partitioned columns") { - // DS V2 doesn't support partitioned table. - if (!conf.contains(SQLConf.DEFAULT_CATALOG.key)) { - withTable("t") { - sql(s"CREATE TABLE t(i STRING, c CHAR(5)) USING $format PARTITIONED BY (c)") - sql("INSERT INTO t VALUES ('1', 'a')") - checkAnswer(spark.table("t"), Row("1", "a" + " " * 4)) - } + withTable("t") { + sql(s"CREATE TABLE t(i STRING, c CHAR(5)) USING $format PARTITIONED BY (c)") + sql("INSERT INTO t VALUES ('1', 'a')") + checkAnswer(spark.table("t"), Row("1", "a" + " " * 4)) } }