[GitHub] [spark] rdblue commented on a change in pull request #24937: [SPARK-28139][SQL] Add v2 ALTER TABLE implementation.

2019-07-11 Thread GitBox
rdblue commented on a change in pull request #24937: [SPARK-28139][SQL] Add v2 
ALTER TABLE implementation.
URL: https://github.com/apache/spark/pull/24937#discussion_r302787029
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala
 ##
 @@ -353,6 +354,59 @@ trait CheckAnalysis extends PredicateHelper {
   case _ =>
 }
 
+  case alter: AlterTable if alter.childrenResolved =>
 
 Review comment:
   Yes. The meaning of `resolved` is that references are satisfied and there 
are no unresolved placeholders -- that's why `resolved` checks that the 
referenced fields exist. This checks whether that resolved plan has valid and 
reasonable changes, like whether the change updates a column from a string to a 
boolean.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] rdblue commented on a change in pull request #24937: [SPARK-28139][SQL] Add v2 ALTER TABLE implementation.

2019-07-11 Thread GitBox
rdblue commented on a change in pull request #24937: [SPARK-28139][SQL] Add v2 
ALTER TABLE implementation.
URL: https://github.com/apache/spark/pull/24937#discussion_r302786349
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ##
 @@ -787,6 +789,86 @@ class Analyzer(
 }
   }
 
+  /**
+   * Resolve ALTER TABLE statements that use a DSv2 catalog.
+   *
+   * This rule converts unresolved ALTER TABLE statements to v2 when a v2 
catalog is responsible
+   * for the table identifier. A v2 catalog is responsible for an identifier 
when the identifier
+   * has a catalog specified, like prod_catalog.db.table, or when a default v2 
catalog is set and
+   * the table identifier does not include a catalog.
+   */
+  object ResolveAlterTable extends Rule[LogicalPlan] {
+import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._
+override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators 
{
+  case alter @ AlterTableAddColumnsStatement(
+  CatalogObjectIdentifier(Some(v2Catalog), ident), cols) =>
+val changes = cols.map { col =>
+  TableChange.addColumn(col.name.toArray, col.dataType, true, 
col.comment.orNull)
+}
+
+AlterTable(
+  v2Catalog.asTableCatalog, ident,
+  UnresolvedRelation(alter.tableName),
+  changes)
+
+  case alter @ AlterTableAlterColumnStatement(
+  CatalogObjectIdentifier(Some(v2Catalog), ident), colName, dataType, 
comment) =>
+val typeChange = dataType.map { newDataType =>
+  TableChange.updateColumnType(colName.toArray, newDataType, true)
+}
+
+val commentChange = comment.map { newComment =>
+  TableChange.updateColumnComment(colName.toArray, newComment)
+}
+
+AlterTable(
+  v2Catalog.asTableCatalog, ident,
+  UnresolvedRelation(alter.tableName),
+  typeChange.toSeq ++ commentChange.toSeq)
+
+  case alter @ AlterTableRenameColumnStatement(
+  CatalogObjectIdentifier(Some(v2Catalog), ident), col, newName) =>
+AlterTable(
+  v2Catalog.asTableCatalog, ident,
+  UnresolvedRelation(alter.tableName),
+  Seq(TableChange.renameColumn(col.toArray, newName)))
+
+  case alter @ AlterTableDropColumnsStatement(
+  CatalogObjectIdentifier(Some(v2Catalog), ident), cols) =>
+val changes = cols.map(col => TableChange.deleteColumn(col.toArray))
+AlterTable(
+  v2Catalog.asTableCatalog, ident,
+  UnresolvedRelation(alter.tableName),
+  changes)
+
+  case alter @ AlterTableSetPropertiesStatement(
+  CatalogObjectIdentifier(Some(v2Catalog), ident), props) =>
+val changes = props.map {
+  case (key, value) =>
+TableChange.setProperty(key, value)
+}
+
+AlterTable(
+  v2Catalog.asTableCatalog, ident,
+  UnresolvedRelation(alter.tableName),
+  changes.toSeq)
+
+  case alter @ AlterTableUnsetPropertiesStatement(
+  CatalogObjectIdentifier(Some(v2Catalog), ident), keys, _) =>
+AlterTable(
+  v2Catalog.asTableCatalog, ident,
+  UnresolvedRelation(alter.tableName),
+  keys.map(key => TableChange.removeProperty(key)))
+
+  case alter @ AlterTableSetLocationStatement(
+  CatalogObjectIdentifier(Some(v2Catalog), ident), newLoc) =>
+AlterTable(
+  v2Catalog.asTableCatalog, ident,
+  UnresolvedRelation(alter.tableName),
+  Seq(TableChange.setProperty("location", newLoc)))
 
 Review comment:
   Not all tables have locations and we're using the convention of passing 
optional metadata as table properties. I think that given that we don't want to 
build special support for everything, this is the right way to pass the change.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] rdblue commented on a change in pull request #24937: [SPARK-28139][SQL] Add v2 ALTER TABLE implementation.

2019-07-11 Thread GitBox
rdblue commented on a change in pull request #24937: [SPARK-28139][SQL] Add v2 
ALTER TABLE implementation.
URL: https://github.com/apache/spark/pull/24937#discussion_r302786449
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala
 ##
 @@ -353,6 +354,59 @@ trait CheckAnalysis extends PredicateHelper {
   case _ =>
 }
 
+  case alter: AlterTable if alter.childrenResolved =>
+val table = alter.table
+def findField(operation: String, fieldName: Array[String]): 
StructField = {
+  // include collections because structs nested in maps and arrays 
may be altered
+  val field = table.schema.findNestedField(fieldName, 
includeCollections = true)
 
 Review comment:
   I believe it uses the resolver, so it should be case sensitive if the 
analyzer is.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] rdblue commented on a change in pull request #24937: [SPARK-28139][SQL] Add v2 ALTER TABLE implementation.

2019-07-02 Thread GitBox
rdblue commented on a change in pull request #24937: [SPARK-28139][SQL] Add v2 
ALTER TABLE implementation.
URL: https://github.com/apache/spark/pull/24937#discussion_r299761247
 
 

 ##
 File path: 
sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala
 ##
 @@ -344,4 +345,834 @@ class DataSourceV2SQLSuite extends QueryTest with 
SharedSQLContext with BeforeAn
 df_joined)
 }
   }
+
+  test("AlterTable: table does not exist") {
+val exc = intercept[AnalysisException] {
+  sql(s"ALTER TABLE testcat.ns1.table_name DROP COLUMN id")
+}
+
+assert(exc.getMessage.contains("testcat.ns1.table_name"))
+assert(exc.getMessage.contains("Table or view not found"))
+  }
+
+  test("AlterTable: change rejected by implementation") {
+val t = "testcat.ns1.table_name"
+withTable(t) {
+  sql(s"CREATE TABLE $t (id int) USING foo")
+
+  val exc = intercept[SparkException] {
+sql(s"ALTER TABLE $t DROP COLUMN id")
+  }
+
+  assert(exc.getMessage.contains("Unsupported table change"))
+  assert(exc.getMessage.contains("Cannot drop all fields")) // from the 
implementation
+
+  val testCatalog = spark.catalog("testcat").asTableCatalog
+  val table = testCatalog.loadTable(Identifier.of(Array("ns1"), 
"table_name"))
+
+  assert(table.name == "testcat.ns1.table_name")
+  assert(table.schema == new StructType().add("id", IntegerType))
+}
+  }
+
+  test("AlterTable: add top-level column") {
+val t = "testcat.ns1.table_name"
+withTable(t) {
+  sql(s"CREATE TABLE $t (id int) USING foo")
+  sql(s"ALTER TABLE $t ADD COLUMN data string")
+
+  val testCatalog = spark.catalog("testcat").asTableCatalog
+  val table = testCatalog.loadTable(Identifier.of(Array("ns1"), 
"table_name"))
+
+  assert(table.name == "testcat.ns1.table_name")
+  assert(table.schema == new StructType().add("id", 
IntegerType).add("data", StringType))
+}
+  }
+
+  test("AlterTable: add column with comment") {
+val t = "testcat.ns1.table_name"
+withTable(t) {
+  sql(s"CREATE TABLE $t (id int) USING foo")
+  sql(s"ALTER TABLE $t ADD COLUMN data string COMMENT 'doc'")
+
+  val testCatalog = spark.catalog("testcat").asTableCatalog
+  val table = testCatalog.loadTable(Identifier.of(Array("ns1"), 
"table_name"))
+
+  assert(table.name == "testcat.ns1.table_name")
+  assert(table.schema == StructType(Seq(
+StructField("id", IntegerType),
+StructField("data", StringType).withComment("doc"
+}
+  }
+
+  test("AlterTable: add multiple columns") {
+val t = "testcat.ns1.table_name"
+withTable(t) {
+  sql(s"CREATE TABLE $t (id int) USING foo")
+  sql(s"ALTER TABLE $t ADD COLUMNS data string COMMENT 'doc', ts 
timestamp")
+
+  val testCatalog = spark.catalog("testcat").asTableCatalog
+  val table = testCatalog.loadTable(Identifier.of(Array("ns1"), 
"table_name"))
+
+  assert(table.name == "testcat.ns1.table_name")
+  assert(table.schema == StructType(Seq(
+StructField("id", IntegerType),
+StructField("data", StringType).withComment("doc"),
+StructField("ts", TimestampType
+}
+  }
+
+  test("AlterTable: add nested column") {
+val t = "testcat.ns1.table_name"
+withTable(t) {
+  sql(s"CREATE TABLE $t (id int, point struct) USING 
foo")
+  sql(s"ALTER TABLE $t ADD COLUMN point.z double")
+
+  val testCatalog = spark.catalog("testcat").asTableCatalog
+  val table = testCatalog.loadTable(Identifier.of(Array("ns1"), 
"table_name"))
+
+  assert(table.name == "testcat.ns1.table_name")
+  assert(table.schema == new StructType()
+  .add("id", IntegerType)
+  .add("point", StructType(Seq(
+StructField("x", DoubleType),
+StructField("y", DoubleType),
+StructField("z", DoubleType)
+}
+  }
+
+  test("AlterTable: add nested column to map key") {
+val t = "testcat.ns1.table_name"
+withTable(t) {
+  sql(s"CREATE TABLE $t (id int, points map, 
bigint>) USING foo")
+  sql(s"ALTER TABLE $t ADD COLUMN points.key.z double")
+
+  val testCatalog = spark.catalog("testcat").asTableCatalog
+  val table = testCatalog.loadTable(Identifier.of(Array("ns1"), 
"table_name"))
+
+  assert(table.name == "testcat.ns1.table_name")
+  assert(table.schema == new StructType()
+  .add("id", IntegerType)
+  .add("points", MapType(StructType(Seq(
+StructField("x", DoubleType),
+StructField("y", DoubleType),
+StructField("z", DoubleType))), LongType)))
+}
+  }
+
+  test("AlterTable: add nested column to map value") {
+val t = "testcat.ns1.table_name"
+withTable(t) {
+  sql(s"CREATE TABLE $t (id int, points map>) USING foo")
+  sql(s"ALTER TABLE $t ADD COLUMN points.value.z double")
+
+  val testCatalog = spark.catalog("testcat").asTableCatalog
+  val 

[GitHub] [spark] rdblue commented on a change in pull request #24937: [SPARK-28139][SQL] Add v2 ALTER TABLE implementation.

2019-07-02 Thread GitBox
rdblue commented on a change in pull request #24937: [SPARK-28139][SQL] Add v2 
ALTER TABLE implementation.
URL: https://github.com/apache/spark/pull/24937#discussion_r299592088
 
 

 ##
 File path: 
sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala
 ##
 @@ -344,4 +345,834 @@ class DataSourceV2SQLSuite extends QueryTest with 
SharedSQLContext with BeforeAn
 df_joined)
 }
   }
+
+  test("AlterTable: table does not exist") {
+val exc = intercept[AnalysisException] {
+  sql(s"ALTER TABLE testcat.ns1.table_name DROP COLUMN id")
+}
+
+assert(exc.getMessage.contains("testcat.ns1.table_name"))
+assert(exc.getMessage.contains("Table or view not found"))
+  }
+
+  test("AlterTable: change rejected by implementation") {
+val t = "testcat.ns1.table_name"
+withTable(t) {
+  sql(s"CREATE TABLE $t (id int) USING foo")
+
+  val exc = intercept[SparkException] {
+sql(s"ALTER TABLE $t DROP COLUMN id")
+  }
+
+  assert(exc.getMessage.contains("Unsupported table change"))
+  assert(exc.getMessage.contains("Cannot drop all fields")) // from the 
implementation
+
+  val testCatalog = spark.catalog("testcat").asTableCatalog
+  val table = testCatalog.loadTable(Identifier.of(Array("ns1"), 
"table_name"))
+
+  assert(table.name == "testcat.ns1.table_name")
+  assert(table.schema == new StructType().add("id", IntegerType))
+}
+  }
+
+  test("AlterTable: add top-level column") {
+val t = "testcat.ns1.table_name"
+withTable(t) {
+  sql(s"CREATE TABLE $t (id int) USING foo")
+  sql(s"ALTER TABLE $t ADD COLUMN data string")
+
+  val testCatalog = spark.catalog("testcat").asTableCatalog
+  val table = testCatalog.loadTable(Identifier.of(Array("ns1"), 
"table_name"))
+
+  assert(table.name == "testcat.ns1.table_name")
+  assert(table.schema == new StructType().add("id", 
IntegerType).add("data", StringType))
+}
+  }
+
+  test("AlterTable: add column with comment") {
+val t = "testcat.ns1.table_name"
+withTable(t) {
+  sql(s"CREATE TABLE $t (id int) USING foo")
+  sql(s"ALTER TABLE $t ADD COLUMN data string COMMENT 'doc'")
+
+  val testCatalog = spark.catalog("testcat").asTableCatalog
+  val table = testCatalog.loadTable(Identifier.of(Array("ns1"), 
"table_name"))
+
+  assert(table.name == "testcat.ns1.table_name")
+  assert(table.schema == StructType(Seq(
+StructField("id", IntegerType),
+StructField("data", StringType).withComment("doc"
+}
+  }
+
+  test("AlterTable: add multiple columns") {
+val t = "testcat.ns1.table_name"
+withTable(t) {
+  sql(s"CREATE TABLE $t (id int) USING foo")
+  sql(s"ALTER TABLE $t ADD COLUMNS data string COMMENT 'doc', ts 
timestamp")
+
+  val testCatalog = spark.catalog("testcat").asTableCatalog
+  val table = testCatalog.loadTable(Identifier.of(Array("ns1"), 
"table_name"))
+
+  assert(table.name == "testcat.ns1.table_name")
+  assert(table.schema == StructType(Seq(
+StructField("id", IntegerType),
+StructField("data", StringType).withComment("doc"),
+StructField("ts", TimestampType
+}
+  }
+
+  test("AlterTable: add nested column") {
+val t = "testcat.ns1.table_name"
+withTable(t) {
+  sql(s"CREATE TABLE $t (id int, point struct) USING 
foo")
+  sql(s"ALTER TABLE $t ADD COLUMN point.z double")
+
+  val testCatalog = spark.catalog("testcat").asTableCatalog
+  val table = testCatalog.loadTable(Identifier.of(Array("ns1"), 
"table_name"))
+
+  assert(table.name == "testcat.ns1.table_name")
+  assert(table.schema == new StructType()
+  .add("id", IntegerType)
+  .add("point", StructType(Seq(
+StructField("x", DoubleType),
+StructField("y", DoubleType),
+StructField("z", DoubleType)
+}
+  }
+
+  test("AlterTable: add nested column to map key") {
+val t = "testcat.ns1.table_name"
+withTable(t) {
+  sql(s"CREATE TABLE $t (id int, points map, 
bigint>) USING foo")
+  sql(s"ALTER TABLE $t ADD COLUMN points.key.z double")
+
+  val testCatalog = spark.catalog("testcat").asTableCatalog
+  val table = testCatalog.loadTable(Identifier.of(Array("ns1"), 
"table_name"))
+
+  assert(table.name == "testcat.ns1.table_name")
+  assert(table.schema == new StructType()
+  .add("id", IntegerType)
+  .add("points", MapType(StructType(Seq(
+StructField("x", DoubleType),
+StructField("y", DoubleType),
+StructField("z", DoubleType))), LongType)))
+}
+  }
+
+  test("AlterTable: add nested column to map value") {
+val t = "testcat.ns1.table_name"
+withTable(t) {
+  sql(s"CREATE TABLE $t (id int, points map>) USING foo")
+  sql(s"ALTER TABLE $t ADD COLUMN points.value.z double")
+
+  val testCatalog = spark.catalog("testcat").asTableCatalog
+  val 

[GitHub] [spark] rdblue commented on a change in pull request #24937: [SPARK-28139][SQL] Add v2 ALTER TABLE implementation.

2019-07-02 Thread GitBox
rdblue commented on a change in pull request #24937: [SPARK-28139][SQL] Add v2 
ALTER TABLE implementation.
URL: https://github.com/apache/spark/pull/24937#discussion_r299591982
 
 

 ##
 File path: 
sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala
 ##
 @@ -344,4 +345,834 @@ class DataSourceV2SQLSuite extends QueryTest with 
SharedSQLContext with BeforeAn
 df_joined)
 }
   }
+
+  test("AlterTable: table does not exist") {
+val exc = intercept[AnalysisException] {
+  sql(s"ALTER TABLE testcat.ns1.table_name DROP COLUMN id")
+}
+
+assert(exc.getMessage.contains("testcat.ns1.table_name"))
+assert(exc.getMessage.contains("Table or view not found"))
+  }
+
+  test("AlterTable: change rejected by implementation") {
+val t = "testcat.ns1.table_name"
+withTable(t) {
+  sql(s"CREATE TABLE $t (id int) USING foo")
+
+  val exc = intercept[SparkException] {
+sql(s"ALTER TABLE $t DROP COLUMN id")
+  }
+
+  assert(exc.getMessage.contains("Unsupported table change"))
+  assert(exc.getMessage.contains("Cannot drop all fields")) // from the 
implementation
+
+  val testCatalog = spark.catalog("testcat").asTableCatalog
+  val table = testCatalog.loadTable(Identifier.of(Array("ns1"), 
"table_name"))
+
+  assert(table.name == "testcat.ns1.table_name")
+  assert(table.schema == new StructType().add("id", IntegerType))
+}
+  }
+
+  test("AlterTable: add top-level column") {
+val t = "testcat.ns1.table_name"
+withTable(t) {
+  sql(s"CREATE TABLE $t (id int) USING foo")
+  sql(s"ALTER TABLE $t ADD COLUMN data string")
+
+  val testCatalog = spark.catalog("testcat").asTableCatalog
+  val table = testCatalog.loadTable(Identifier.of(Array("ns1"), 
"table_name"))
+
+  assert(table.name == "testcat.ns1.table_name")
+  assert(table.schema == new StructType().add("id", 
IntegerType).add("data", StringType))
+}
+  }
+
+  test("AlterTable: add column with comment") {
+val t = "testcat.ns1.table_name"
+withTable(t) {
+  sql(s"CREATE TABLE $t (id int) USING foo")
+  sql(s"ALTER TABLE $t ADD COLUMN data string COMMENT 'doc'")
+
+  val testCatalog = spark.catalog("testcat").asTableCatalog
+  val table = testCatalog.loadTable(Identifier.of(Array("ns1"), 
"table_name"))
+
+  assert(table.name == "testcat.ns1.table_name")
+  assert(table.schema == StructType(Seq(
+StructField("id", IntegerType),
+StructField("data", StringType).withComment("doc"
+}
+  }
+
+  test("AlterTable: add multiple columns") {
+val t = "testcat.ns1.table_name"
+withTable(t) {
+  sql(s"CREATE TABLE $t (id int) USING foo")
+  sql(s"ALTER TABLE $t ADD COLUMNS data string COMMENT 'doc', ts 
timestamp")
+
+  val testCatalog = spark.catalog("testcat").asTableCatalog
+  val table = testCatalog.loadTable(Identifier.of(Array("ns1"), 
"table_name"))
+
+  assert(table.name == "testcat.ns1.table_name")
+  assert(table.schema == StructType(Seq(
+StructField("id", IntegerType),
+StructField("data", StringType).withComment("doc"),
+StructField("ts", TimestampType
+}
+  }
+
+  test("AlterTable: add nested column") {
+val t = "testcat.ns1.table_name"
+withTable(t) {
+  sql(s"CREATE TABLE $t (id int, point struct) USING 
foo")
+  sql(s"ALTER TABLE $t ADD COLUMN point.z double")
+
+  val testCatalog = spark.catalog("testcat").asTableCatalog
+  val table = testCatalog.loadTable(Identifier.of(Array("ns1"), 
"table_name"))
+
+  assert(table.name == "testcat.ns1.table_name")
+  assert(table.schema == new StructType()
+  .add("id", IntegerType)
+  .add("point", StructType(Seq(
+StructField("x", DoubleType),
+StructField("y", DoubleType),
+StructField("z", DoubleType)
+}
+  }
+
+  test("AlterTable: add nested column to map key") {
+val t = "testcat.ns1.table_name"
+withTable(t) {
+  sql(s"CREATE TABLE $t (id int, points map, 
bigint>) USING foo")
+  sql(s"ALTER TABLE $t ADD COLUMN points.key.z double")
+
+  val testCatalog = spark.catalog("testcat").asTableCatalog
+  val table = testCatalog.loadTable(Identifier.of(Array("ns1"), 
"table_name"))
+
+  assert(table.name == "testcat.ns1.table_name")
+  assert(table.schema == new StructType()
+  .add("id", IntegerType)
+  .add("points", MapType(StructType(Seq(
+StructField("x", DoubleType),
+StructField("y", DoubleType),
+StructField("z", DoubleType))), LongType)))
+}
+  }
+
+  test("AlterTable: add nested column to map value") {
+val t = "testcat.ns1.table_name"
+withTable(t) {
+  sql(s"CREATE TABLE $t (id int, points map>) USING foo")
+  sql(s"ALTER TABLE $t ADD COLUMN points.value.z double")
+
+  val testCatalog = spark.catalog("testcat").asTableCatalog
+  val 

[GitHub] [spark] rdblue commented on a change in pull request #24937: [SPARK-28139][SQL] Add v2 ALTER TABLE implementation.

2019-06-27 Thread GitBox
rdblue commented on a change in pull request #24937: [SPARK-28139][SQL] Add v2 
ALTER TABLE implementation.
URL: https://github.com/apache/spark/pull/24937#discussion_r298357154
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala
 ##
 @@ -313,6 +314,42 @@ trait CheckAnalysis extends PredicateHelper {
   failAnalysis(s"Invalid partitioning: ${badReferences.mkString(", 
")}")
 }
 
+  case alter: AlterTable if alter.childrenResolved =>
+val table = alter.table
+def findField(operation: String, fieldName: Array[String]): 
StructField = {
+  // include collections because structs nested in maps and arrays 
may be altered
+  val field = table.schema.findNestedField(fieldName, 
includeCollections = true)
+  if (field.isEmpty) {
+throw new AnalysisException(
+  s"Cannot $operation missing field in ${table.name} schema: 
${fieldName.quoted}")
+  }
+  field.get
+}
+
+alter.changes.foreach {
+  case add: AddColumn =>
+val parent = add.fieldNames.init
+if (parent.nonEmpty) {
+  findField("add to", parent)
+}
+  case update: UpdateColumnType =>
+val field = findField("update", update.fieldNames)
+if (!Cast.canUpCast(field.dataType, update.newDataType)) {
 
 Review comment:
   @gengliangwang, I've updated this and added tests for these cases. Thanks 
for catching the complex type case.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] rdblue commented on a change in pull request #24937: [SPARK-28139][SQL] Add v2 ALTER TABLE implementation.

2019-06-26 Thread GitBox
rdblue commented on a change in pull request #24937: [SPARK-28139][SQL] Add v2 
ALTER TABLE implementation.
URL: https://github.com/apache/spark/pull/24937#discussion_r297917661
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala
 ##
 @@ -313,6 +314,42 @@ trait CheckAnalysis extends PredicateHelper {
   failAnalysis(s"Invalid partitioning: ${badReferences.mkString(", 
")}")
 }
 
+  case alter: AlterTable if alter.childrenResolved =>
+val table = alter.table
+def findField(operation: String, fieldName: Array[String]): 
StructField = {
+  // include collections because structs nested in maps and arrays 
may be altered
+  val field = table.schema.findNestedField(fieldName, 
includeCollections = true)
+  if (field.isEmpty) {
+throw new AnalysisException(
+  s"Cannot $operation missing field in ${table.name} schema: 
${fieldName.quoted}")
+  }
+  field.get
+}
+
+alter.changes.foreach {
+  case add: AddColumn =>
+val parent = add.fieldNames.init
+if (parent.nonEmpty) {
+  findField("add to", parent)
+}
+  case update: UpdateColumnType =>
+val field = findField("update", update.fieldNames)
+if (!Cast.canUpCast(field.dataType, update.newDataType)) {
 
 Review comment:
   Okay, I agree that you should not be able to alter complex types with 
another complex type


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] rdblue commented on a change in pull request #24937: [SPARK-28139][SQL] Add v2 ALTER TABLE implementation.

2019-06-26 Thread GitBox
rdblue commented on a change in pull request #24937: [SPARK-28139][SQL] Add v2 
ALTER TABLE implementation.
URL: https://github.com/apache/spark/pull/24937#discussion_r297908457
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ##
 @@ -34,6 +34,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate._
 import org.apache.spark.sql.catalyst.expressions.objects._
 import org.apache.spark.sql.catalyst.plans._
 import org.apache.spark.sql.catalyst.plans.logical._
+import 
org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, 
AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, 
AlterTableRenameColumnStatement, AlterTableSetLocationStatement, 
AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement}
 
 Review comment:
   No, I've noted in several places that this is a bad practice because it 
causes namespace problems (duplicate names accidentally imported) and commit 
conflicts.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] rdblue commented on a change in pull request #24937: [SPARK-28139][SQL] Add v2 ALTER TABLE implementation.

2019-06-26 Thread GitBox
rdblue commented on a change in pull request #24937: [SPARK-28139][SQL] Add v2 
ALTER TABLE implementation.
URL: https://github.com/apache/spark/pull/24937#discussion_r297908184
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala
 ##
 @@ -313,6 +314,42 @@ trait CheckAnalysis extends PredicateHelper {
   failAnalysis(s"Invalid partitioning: ${badReferences.mkString(", 
")}")
 }
 
+  case alter: AlterTable if alter.childrenResolved =>
+val table = alter.table
+def findField(operation: String, fieldName: Array[String]): 
StructField = {
+  // include collections because structs nested in maps and arrays 
may be altered
+  val field = table.schema.findNestedField(fieldName, 
includeCollections = true)
+  if (field.isEmpty) {
+throw new AnalysisException(
+  s"Cannot $operation missing field in ${table.name} schema: 
${fieldName.quoted}")
+  }
+  field.get
+}
+
+alter.changes.foreach {
+  case add: AddColumn =>
+val parent = add.fieldNames.init
+if (parent.nonEmpty) {
+  findField("add to", parent)
+}
+  case update: UpdateColumnType =>
+val field = findField("update", update.fieldNames)
+if (!Cast.canUpCast(field.dataType, update.newDataType)) {
 
 Review comment:
   There is a test for upcast failure. See "AlterTable: update column type must 
be compatible".
   
   Why should altering a nested column fail? As long as the new column is 
optional, there is no problem.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org