aboutsummaryrefslogtreecommitdiff
path: root/sql/core/src/test
diff options
context:
space:
mode:
authorWenchen Fan <wenchen@databricks.com>2016-07-31 18:18:53 -0700
committerYin Huai <yhuai@databricks.com>2016-07-31 18:18:53 -0700
commit301fb0d7236eb55d53c9cd60804a2d755b4ad3b2 (patch)
tree51383907d4c442831fac687f61711ba198949561 /sql/core/src/test
parent064d91ff7342002414d3274694a8e2e37f154986 (diff)
downloadspark-301fb0d7236eb55d53c9cd60804a2d755b4ad3b2.tar.gz
spark-301fb0d7236eb55d53c9cd60804a2d755b4ad3b2.tar.bz2
spark-301fb0d7236eb55d53c9cd60804a2d755b4ad3b2.zip
[SPARK-16731][SQL] use StructType in CatalogTable and remove CatalogColumn
## What changes were proposed in this pull request? `StructField` has very similar semantic with `CatalogColumn`, except that `CatalogColumn` use string to express data type. I think it's reasonable to use `StructType` as the `CatalogTable.schema` and remove `CatalogColumn`. ## How was this patch tested? existing tests. Author: Wenchen Fan <wenchen@databricks.com> Closes #14363 from cloud-fan/column.
Diffstat (limited to 'sql/core/src/test')
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala25
1 files changed, 11 insertions, 14 deletions
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala
index 7bd1b0bcdb..564fc73ee7 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala
@@ -27,7 +27,7 @@ import org.apache.spark.sql.{AnalysisException, QueryTest, Row, SaveMode}
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.analysis.{DatabaseAlreadyExistsException, FunctionRegistry, NoSuchPartitionException, NoSuchTableException, TempTableAlreadyExistsException}
import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogDatabase, CatalogStorageFormat}
-import org.apache.spark.sql.catalyst.catalog.{CatalogColumn, CatalogTable, CatalogTableType}
+import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType}
import org.apache.spark.sql.catalyst.catalog.{CatalogTablePartition, SessionCatalog}
import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
import org.apache.spark.sql.catalyst.parser.ParseException
@@ -89,11 +89,11 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach {
identifier = name,
tableType = CatalogTableType.EXTERNAL,
storage = storage,
- schema = Seq(
- CatalogColumn("col1", "int"),
- CatalogColumn("col2", "string"),
- CatalogColumn("a", "int"),
- CatalogColumn("b", "int")),
+ schema = new StructType()
+ .add("col1", "int")
+ .add("col2", "string")
+ .add("a", "int")
+ .add("b", "int"),
partitionColumnNames = Seq("a", "b"),
createTime = 0L)
}
@@ -258,9 +258,6 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach {
userSpecifiedPartitionCols: Option[String],
expectedSchema: StructType,
expectedPartitionCols: Seq[String]): Unit = {
- var tableSchema = StructType(Nil)
- var partCols = Seq.empty[String]
-
val tabName = "tab1"
withTable(tabName) {
val partitionClause =
@@ -277,11 +274,11 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach {
""".stripMargin)
val tableMetadata = spark.sessionState.catalog.getTableMetadata(TableIdentifier(tabName))
- tableSchema = DDLUtils.getSchemaFromTableProperties(tableMetadata)
- partCols = DDLUtils.getPartitionColumnsFromTableProperties(tableMetadata)
+ assert(expectedSchema ==
+ DDLUtils.getSchemaFromTableProperties(tableMetadata))
+ assert(expectedPartitionCols ==
+ DDLUtils.getPartitionColumnsFromTableProperties(tableMetadata))
}
- assert(tableSchema == expectedSchema)
- assert(partCols == expectedPartitionCols)
}
test("Create partitioned data source table without user specified schema") {
@@ -601,7 +598,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach {
sql("CREATE TABLE tbl(a INT, b INT) USING parquet")
val table = catalog.getTableMetadata(TableIdentifier("tbl"))
assert(table.tableType == CatalogTableType.MANAGED)
- assert(table.schema == Seq(CatalogColumn("a", "int"), CatalogColumn("b", "int")))
+ assert(table.schema == new StructType().add("a", "int").add("b", "int"))
assert(table.properties(DATASOURCE_PROVIDER) == "parquet")
}
}