You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2020/09/28 13:09:01 UTC

[GitHub] [spark] yaooqinn opened a new pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

yaooqinn opened a new pull request #29893:
URL: https://github.com/apache/spark/pull/29893


   <!--
   Thanks for sending a pull request!  Here are some tips for you:
     1. If this is your first time, please read our contributor guidelines: https://spark.apache.org/contributing.html
     2. Ensure you have added or run the appropriate tests for your PR: https://spark.apache.org/developer-tools.html
     3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][SPARK-XXXX] Your PR title ...'.
     4. Be sure to keep the PR description updated to reflect all changes.
     5. Please write your PR title to summarize what this PR proposes.
     6. If possible, provide a concise example to reproduce the issue for a faster review.
     7. If you want to add a new configuration, please read the guideline first for naming configurations in
        'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'.
   -->
   
   ### What changes were proposed in this pull request?
   INSERT currently does not support named column lists.  
   
   INSERT INTO <table> (col1, col2,…) VALUES( 'val1', 'val2', … )
   
   Note, we assume the column list contains all the column names. Issue an exception if the list is not complete. The column order could be different from the column order defined in the table definition.
   
   
   ### Why are the changes needed?
   <!--
   Please clarify why the changes are needed. For instance,
     1. If you propose a new API, clarify the use case for a new API.
     2. If you fix a bug, you can clarify why it is a bug.
   -->
   
   
   ### Does this PR introduce _any_ user-facing change?
   <!--
   Note that it means *any* user-facing change including all aspects such as the documentation fix.
   If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible.
   If possible, please also clarify if this is a user-facing change compared to the released Spark versions or within the unreleased branches such as master.
   If no, write 'No'.
   -->
   
   
   ### How was this patch tested?
   <!--
   If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
   If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
   If tests were not added, please describe why they were not added and/or why it was difficult to add.
   -->
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-734269319






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] yaooqinn commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-733580257


   retest this please


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700012113


   Build finished. Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-733868465


   **[Test build #131796 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131796/testReport)** for PR 29893 at commit [`86d0032`](https://github.com/apache/spark/commit/86d0032ad00f7a1f10e1963070e39a24e640998d).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-734790187






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29893:
URL: https://github.com/apache/spark/pull/29893#discussion_r531418233



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -3104,6 +3106,62 @@ class Analyzer(override val catalogManager: CatalogManager)
     }
   }
 
+  object ResolveUserSpecifiedColumns extends Rule[LogicalPlan] {
+    override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperators {
+      case i: InsertIntoStatement if i.table.resolved && i.query.resolved &&
+          i.userSpecifiedCols.nonEmpty =>
+        val resolved = resolveUserSpecifiedColumns(i)
+        val projection = addColumnListOnQuery(i.table.output, resolved, i.query)
+        i.copy(userSpecifiedCols = Nil, query = projection)
+    }
+
+    private def resolveUserSpecifiedColumns(i: InsertIntoStatement): Seq[NamedExpression] = {
+      SchemaUtils.checkColumnNameDuplication(
+        i.userSpecifiedCols, "in the column list", resolver)
+
+      i.userSpecifiedCols.map { col =>
+          i.table.resolve(Seq(col), resolver)
+            .getOrElse(i.table.failAnalysis(s"Cannot resolve column name $col"))
+      }
+    }
+
+    private def addColumnListOnQuery(
+        tableOutput: Seq[Attribute],
+        cols: Seq[NamedExpression],
+        query: LogicalPlan): LogicalPlan = {
+      val errors = new mutable.ArrayBuffer[String]()
+
+      def failAdd(): Unit = {
+        val errMsg = if (errors.nonEmpty) errors.mkString("\n- ", "\n- ", "") else ""
+        query.failAnalysis(
+          s"""Cannot write to table due to mismatched user specified columns and data columns:
+             |Specified columns: ${cols.map(c => s"'${c.name}'").mkString(", ")}
+             |Data columns: ${query.output.map(c => s"'${c.name}'").mkString(", ")}$errMsg"""
+            .stripMargin)
+      }
+
+      if (cols.size != query.output.size) failAdd()
+
+      val nameToQueryExpr = cols.zip(query.output).toMap
+      val resolved = tableOutput.flatMap { tableAttr =>
+        if (nameToQueryExpr.contains(tableAttr)) {
+          TableOutputResolver.checkField(
+            tableAttr, nameToQueryExpr(tableAttr), byName = false, conf, err => errors += err)
+        } else {
+          None

Review comment:
       when can we go to this branch?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-732960610


   **[Test build #131661 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131661/testReport)** for PR 29893 at commit [`c43b964`](https://github.com/apache/spark/commit/c43b96404cbdfeb859784304f2174fc28f66b357).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] yaooqinn commented on a change in pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #29893:
URL: https://github.com/apache/spark/pull/29893#discussion_r545940081



##########
File path: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSQLInsertTestSuite.scala
##########
@@ -0,0 +1,25 @@
+/*
+ * 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.hive
+
+import org.apache.spark.sql.SQLInsertTestSuite
+import org.apache.spark.sql.hive.test.TestHiveSingleton
+
+class HiveSQLInsertTestSuite extends SQLInsertTestSuite with TestHiveSingleton {

Review comment:
       I have submitted a followup, thanks




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-702002639


   Merged build finished. Test PASSed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-706978106


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700783880






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700605102


   **[Test build #129232 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129232/testReport)** for PR 29893 at commit [`086cfa8`](https://github.com/apache/spark/commit/086cfa8260b1aba1bab774c80f8c3ff5c6d2f31a).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29893:
URL: https://github.com/apache/spark/pull/29893#discussion_r530471201



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
##########
@@ -3718,6 +3718,49 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark
       }
     }
   }
+
+  test("SPARK-32976: Support column list specification in insert into statement") {

Review comment:
       can we create a `insert.sql` test file and move this test there?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29893:
URL: https://github.com/apache/spark/pull/29893#discussion_r530465028



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala
##########
@@ -290,6 +291,7 @@ case class DropViewStatement(
 case class InsertIntoStatement(
     table: LogicalPlan,
     partitionSpec: Map[String, Option[String]],
+    userSpecifiedCols: Seq[Attribute],

Review comment:
       I'm not sure if this should be `Attribute` not `String`. `InsertIntoStatement`'s child is `query`, not `table`, which means these attributes will be resolved based on `query`, while they should be resolved by `table`.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700618031


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/33845/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29893:
URL: https://github.com/apache/spark/pull/29893#discussion_r531419671



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala
##########
@@ -0,0 +1,244 @@
+/*
+ * 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
+
+import org.apache.spark.SparkConf
+import org.apache.spark.sql.connector.InMemoryPartitionTableCatalog
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils}
+
+/**
+ * The base trait for DML - insert syntax
+ */
+trait SQLInsertTestSuite extends QueryTest with SQLTestUtils {
+
+  import testImplicits._
+
+  def format: String
+
+  protected def createTable(
+      table: String,
+      cols: Seq[String],
+      colTypes: Seq[String],
+      partCols: Seq[String] = Nil): Unit = {
+    val values = cols.zip(colTypes).map(tuple => tuple._1 + " " + tuple._2).mkString("(", ", ", ")")
+    val partitionSpec = if (partCols.nonEmpty) {
+      partCols.mkString("PARTITIONED BY (", ",", ")")
+    } else ""
+    sql(s"CREATE TABLE $table$values USING $format $partitionSpec")
+  }
+
+  protected def processInsert(
+      tableName: String,
+      insert: DataFrame,
+      cols: Seq[String] = Nil,
+      partitionExprs: Seq[String] = Nil,
+      mode: SaveMode): Unit = {
+    val tmpView = "tmp_view"
+    val columnList = if (cols.nonEmpty) cols.mkString("(", ",", ")") else ""
+    val partitionList = if (partitionExprs.nonEmpty) {
+      partitionExprs.mkString("PARTITION (", ",", ")")
+    } else ""
+    withTempView(tmpView) {
+      insert.createOrReplaceTempView(tmpView)
+      val overwrite = if (mode == SaveMode.Overwrite) "OVERWRITE" else "INTO"
+      sql(s"INSERT $overwrite TABLE $tableName $partitionList $columnList SELECT * FROM $tmpView")
+    }
+  }
+
+  protected def verifyTable(tableName: String, expected: DataFrame): Unit = {
+    checkAnswer(spark.table(tableName), expected)
+  }
+
+  test("insert with column list - follow table output order") {
+    val t1 = "t1"

Review comment:
       we can hardcode table name if it's very simple like `t1`.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700388776


   **[Test build #129203 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129203/testReport)** for PR 29893 at commit [`40e2ffd`](https://github.com/apache/spark/commit/40e2ffd43ff7c41dcf6c162daf0d9bc672a86996).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] gatorsmile commented on a change in pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
gatorsmile commented on a change in pull request #29893:
URL: https://github.com/apache/spark/pull/29893#discussion_r545307464



##########
File path: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSQLInsertTestSuite.scala
##########
@@ -0,0 +1,25 @@
+/*
+ * 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.hive
+
+import org.apache.spark.sql.SQLInsertTestSuite
+import org.apache.spark.sql.hive.test.TestHiveSingleton
+
+class HiveSQLInsertTestSuite extends SQLInsertTestSuite with TestHiveSingleton {

Review comment:
       How about something like
   
   ```
     protected override def beforeAll(): Unit = {
       spark.conf.set("hive.exec.dynamic.partition.mode", "nonstrict")
       super.beforeAll()
     }
   
     protected override def afterAll(): Unit = {
       try {
         spark.conf.set("hive.exec.dynamic.partition.mode", "strict")
       } finally {
         super.afterAll()
       }
     }
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700007406


   **[Test build #129185 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129185/testReport)** for PR 29893 at commit [`99056c4`](https://github.com/apache/spark/commit/99056c47af4376467d1117ee0ef5018c6bf8985c).
    * This patch **fails to build**.
    * This patch **does not merge cleanly**.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700698882


   **[Test build #129221 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129221/testReport)** for PR 29893 at commit [`e756a55`](https://github.com/apache/spark/commit/e756a558d89a077108d50103af441dffa4ba4263).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] yaooqinn commented on a change in pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #29893:
URL: https://github.com/apache/spark/pull/29893#discussion_r530540006



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -3104,6 +3106,66 @@ class Analyzer(override val catalogManager: CatalogManager)
     }
   }
 
+  object ResolveUserSpecifiedColumns extends Rule[LogicalPlan] {
+    override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperators {
+      case i: InsertIntoStatement if i.table.resolved && i.query.resolved &&
+          i.userSpecifiedCols.nonEmpty =>
+        val resolved = resolveUserSpecifiedColumns(i)
+        val projection = addColumnListOnQuery(i.table.output, resolved, i.query)
+        i.copy(userSpecifiedCols = Nil, query = projection)
+    }
+
+    private def resolveUserSpecifiedColumns(i: InsertIntoStatement): Seq[Attribute] = {
+      val resolved = i.userSpecifiedCols.map {
+        case u: UnresolvedAttribute => withPosition(u) {

Review comment:
       yea, it's wired. the column name should have only one part




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700647328


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/33849/
   Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700620031


   **[Test build #129233 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129233/testReport)** for PR 29893 at commit [`3a02681`](https://github.com/apache/spark/commit/3a0268107996abf3bae6a969181b269b1fc19481).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-734132558


   **[Test build #131826 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131826/testReport)** for PR 29893 at commit [`713b5b9`](https://github.com/apache/spark/commit/713b5b9a1d47ca639f36b690a2435363e9eb4fff).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-733032201


   **[Test build #131673 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131673/testReport)** for PR 29893 at commit [`c74d4d7`](https://github.com/apache/spark/commit/c74d4d7a0e0768fa5d9970a9e7ddd109306d6caa).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] gatorsmile commented on a change in pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
gatorsmile commented on a change in pull request #29893:
URL: https://github.com/apache/spark/pull/29893#discussion_r545307464



##########
File path: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSQLInsertTestSuite.scala
##########
@@ -0,0 +1,25 @@
+/*
+ * 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.hive
+
+import org.apache.spark.sql.SQLInsertTestSuite
+import org.apache.spark.sql.hive.test.TestHiveSingleton
+
+class HiveSQLInsertTestSuite extends SQLInsertTestSuite with TestHiveSingleton {

Review comment:
       How about something like
   
   ```Scala
     private var originalHiveDynamicPartitionMode: String = _
   
     protected override def beforeAll(): Unit = {
       originalHiveDynamicPartitionMode = spark.conf.get("hive.exec.dynamic.partition.mode", "strict")
       spark.conf.set("hive.exec.dynamic.partition.mode", "nonstrict")
       super.beforeAll()
     }
   
     protected override def afterAll(): Unit = {
       try {
         spark.conf.set("hive.exec.dynamic.partition.mode", originalHiveDynamicPartitionMode)
       } finally {
         super.afterAll()
       }
     }
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29893:
URL: https://github.com/apache/spark/pull/29893#discussion_r531487820



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala
##########
@@ -0,0 +1,224 @@
+/*
+ * 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
+
+import org.apache.spark.SparkConf
+import org.apache.spark.sql.connector.InMemoryPartitionTableCatalog
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils}
+
+/**
+ * The base trait for DML - insert syntax
+ */
+trait SQLInsertTestSuite extends QueryTest with SQLTestUtils {
+
+  import testImplicits._
+
+  def format: String
+
+  protected def createTable(
+      table: String,
+      cols: Seq[String],
+      colTypes: Seq[String],
+      partCols: Seq[String] = Nil): Unit = {
+    val values = cols.zip(colTypes).map(tuple => tuple._1 + " " + tuple._2).mkString("(", ", ", ")")
+    val partitionSpec = if (partCols.nonEmpty) {
+      partCols.mkString("PARTITIONED BY (", ",", ")")
+    } else ""
+    sql(s"CREATE TABLE $table$values USING $format $partitionSpec")
+  }
+
+  protected def processInsert(
+      tableName: String,
+      insert: DataFrame,
+      cols: Seq[String] = Nil,
+      partitionExprs: Seq[String] = Nil,
+      overwrite: Boolean): Unit = {
+    val tmpView = "tmp_view"
+    val columnList = if (cols.nonEmpty) cols.mkString("(", ",", ")") else ""
+    val partitionList = if (partitionExprs.nonEmpty) {
+      partitionExprs.mkString("PARTITION (", ",", ")")
+    } else ""
+    withTempView(tmpView) {
+      insert.createOrReplaceTempView(tmpView)
+      val overwriteStr = if (overwrite) "OVERWRITE" else "INTO"
+      sql(
+        s"INSERT $overwriteStr TABLE $tableName $partitionList $columnList SELECT * FROM $tmpView")
+    }
+  }
+
+  protected def verifyTable(tableName: String, expected: DataFrame): Unit = {
+    checkAnswer(spark.table(tableName), expected)
+  }
+
+  test("insert with column list - follow table output order") {
+    withTable("t1") {
+      val df = Seq((1, 2L, "3")).toDF()
+      val cols = Seq("c1", "c2", "c3")
+      createTable("t1", cols, Seq("int", "long", "string"))
+      Seq(false, true).foreach { m =>
+        processInsert("t1", df, cols, overwrite = m)
+        verifyTable("t1", df)
+      }
+    }
+  }
+
+  test("insert with column list - follow table output order + partitioned table") {
+    val cols = Seq("c1", "c2", "c3", "c4")
+    val df = Seq((1, 2, 3, 4)).toDF(cols: _*)
+    withTable("t1") {
+      createTable("t1", cols, Seq("int", "int", "int", "int"), cols.takeRight(2))
+      Seq(false, true).foreach { m =>
+        processInsert("t1", df, cols, overwrite = m)
+        verifyTable("t1", df)
+      }
+    }
+
+    withTable("t1") {
+      createTable("t1", cols, Seq("int", "int", "int", "int"), cols.takeRight(2))
+      Seq(false, true).foreach { m =>
+        processInsert(
+          "t1", df.selectExpr("c1", "c2"), cols.take(2), Seq("c3=3", "c4=4"), overwrite = m)
+        verifyTable("t1", df)
+      }
+    }
+
+    withTable("t1") {
+      createTable("t1", cols, Seq("int", "int", "int", "int"), cols.takeRight(2))
+      Seq(false, true).foreach { m =>
+        processInsert("t1", df.selectExpr("c1", "c2", "c4"),
+          cols.filterNot(_ == "c3"), Seq("c3=3", "c4"), overwrite = m)
+        verifyTable("t1", df)
+      }
+    }
+  }
+
+  test("insert with column list - table output reorder") {
+    withTable("t1") {
+      val cols = Seq("c1", "c2", "c3")
+      val df = Seq((1, 2, 3)).toDF(cols: _*)
+      createTable("t1", cols, Seq("int", "int", "int"))
+      Seq(false, true).foreach { m =>
+        processInsert("t1", df, cols.reverse, overwrite = m)
+        verifyTable("t1", df.selectExpr(cols.reverse: _*))
+      }
+    }
+  }
+
+  test("insert with column list - table output reorder  + partitioned table") {

Review comment:
       nit: remove one space before `+`




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-735033393


   **[Test build #131889 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131889/testReport)** for PR 29893 at commit [`17d0564`](https://github.com/apache/spark/commit/17d056467751785664b6fd5c89c602f7c3e07e94).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-706886780


   **[Test build #129666 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129666/testReport)** for PR 29893 at commit [`c76a2aa`](https://github.com/apache/spark/commit/c76a2aacb86411ef9d0c58ac996b274dce49beb7).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-706923002


   **[Test build #129673 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129673/testReport)** for PR 29893 at commit [`f7e8d18`](https://github.com/apache/spark/commit/f7e8d186320aa805d679942937b12b3a5141fb89).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-734763381






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-728078000


   Sorry for the late review. I have a simpler idea about how to implement it:
   1. Add a new field `userSpecifiedColums` to `InsertIntoStatement`.
   2. Add a new analyzer rule to resolve `InsertIntoStatement` whose `userSpecifiedColums` is not Nil, by adding a project to the input query to reorder the columns. This rule will set `userSpecifiedColums` to Nil at the end, so that it's idempotent.
   3. Other rules that match `InsertIntoStatement` should wait until `userSpecifiedColums` is Nil.
   
   Thus we don't need to change the v2 write commands.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-733324380






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-734362145


   **[Test build #131844 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131844/testReport)** for PR 29893 at commit [`3afb4c0`](https://github.com/apache/spark/commit/3afb4c0f92fc9f549d3dbd962c263518a65f6915).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700571366


   **[Test build #129229 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129229/testReport)** for PR 29893 at commit [`2b164ed`](https://github.com/apache/spark/commit/2b164eddd80145e9104546a7e7b41d340fd9481f).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-734064221






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-732981931


   **[Test build #131661 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131661/testReport)** for PR 29893 at commit [`c43b964`](https://github.com/apache/spark/commit/c43b96404cbdfeb859784304f2174fc28f66b357).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-733749615


   **[Test build #131766 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131766/testReport)** for PR 29893 at commit [`475e790`](https://github.com/apache/spark/commit/475e790f7eeae80c4123f93c74952eb24f056897).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-734729717






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-732960610


   **[Test build #131661 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131661/testReport)** for PR 29893 at commit [`c43b964`](https://github.com/apache/spark/commit/c43b96404cbdfeb859784304f2174fc28f66b357).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-735105008


   **[Test build #131889 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131889/testReport)** for PR 29893 at commit [`17d0564`](https://github.com/apache/spark/commit/17d056467751785664b6fd5c89c602f7c3e07e94).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-733282468






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-733074321


   **[Test build #131678 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131678/testReport)** for PR 29893 at commit [`475e790`](https://github.com/apache/spark/commit/475e790f7eeae80c4123f93c74952eb24f056897).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700388776


   **[Test build #129203 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129203/testReport)** for PR 29893 at commit [`40e2ffd`](https://github.com/apache/spark/commit/40e2ffd43ff7c41dcf6c162daf0d9bc672a86996).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700007598






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] wangyum commented on a change in pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
wangyum commented on a change in pull request #29893:
URL: https://github.com/apache/spark/pull/29893#discussion_r502146835



##########
File path: sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
##########
@@ -337,8 +337,8 @@ query
     ;
 
 insertInto
-    : INSERT OVERWRITE TABLE? multipartIdentifier (partitionSpec (IF NOT EXISTS)?)?                         #insertOverwriteTable
-    | INSERT INTO TABLE? multipartIdentifier partitionSpec? (IF NOT EXISTS)?                                #insertIntoTable
+    : INSERT OVERWRITE TABLE? multipartIdentifier identifierList?  (partitionSpec (IF NOT EXISTS)?)?        #insertOverwriteTable
+    | INSERT INTO TABLE? multipartIdentifier identifierList?  partitionSpec? (IF NOT EXISTS)?               #insertIntoTable

Review comment:
       Move `identifierList?` to the end? This is the order of Hive:
   ```sql
   hive> SELECT version();
   2.3.7 rcb213d88304034393d68cc31a95be24f5aac62b6
   hive> CREATE TABLE spark_32976(id int, name string) PARTITIONED BY (part string);
   hive> INSERT INTO spark_32976 PARTITION (part = '1') (name) SELECT 'name1';
   hive> SELECT * FROM spark_32976;
   NULL	name1	1
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] yaooqinn commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-706885025


   Thanks for reviewing, sorry for the late reply as I spent a long holiday with my family. I've addressed the comment, please check again @gatorsmile @cloud-fan @wangyum thanks.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-702001508


   **[Test build #129299 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129299/testReport)** for PR 29893 at commit [`a76c9ba`](https://github.com/apache/spark/commit/a76c9ba5e2c41b9aa68b6a5e170f7d2750af7514).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700577119






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-701236003






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-734853888






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-733899415






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-734705367


   **[Test build #131871 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131871/testReport)** for PR 29893 at commit [`7680da2`](https://github.com/apache/spark/commit/7680da271c3884093a95f7352fe6e9067c537430).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-706911985


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34270/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-701235984


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/33882/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700849692


   **[Test build #129233 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129233/testReport)** for PR 29893 at commit [`3a02681`](https://github.com/apache/spark/commit/3a0268107996abf3bae6a969181b269b1fc19481).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-734046895


   **[Test build #131826 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131826/testReport)** for PR 29893 at commit [`713b5b9`](https://github.com/apache/spark/commit/713b5b9a1d47ca639f36b690a2435363e9eb4fff).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-733798906






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700625966






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-733074321


   **[Test build #131678 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131678/testReport)** for PR 29893 at commit [`475e790`](https://github.com/apache/spark/commit/475e790f7eeae80c4123f93c74952eb24f056897).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29893:
URL: https://github.com/apache/spark/pull/29893#discussion_r530475127



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -3104,6 +3106,66 @@ class Analyzer(override val catalogManager: CatalogManager)
     }
   }
 
+  object ResolveUserSpecifiedColumns extends Rule[LogicalPlan] {
+    override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperators {
+      case i: InsertIntoStatement if i.table.resolved && i.query.resolved &&
+          i.userSpecifiedCols.nonEmpty =>
+        val resolved = resolveUserSpecifiedColumns(i)
+        val projection = addColumnListOnQuery(i.table.output, resolved, i.query)
+        i.copy(userSpecifiedCols = Nil, query = projection)
+    }
+
+    private def resolveUserSpecifiedColumns(i: InsertIntoStatement): Seq[Attribute] = {
+      val resolved = i.userSpecifiedCols.map {
+        case u: UnresolvedAttribute => withPosition(u) {

Review comment:
       Does other databases support `INSERT INTO t(t.col) ...`? Otherwise using `String` instead of `UnresolvedAttribute` is simpler.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700002465


   **[Test build #129185 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129185/testReport)** for PR 29893 at commit [`99056c4`](https://github.com/apache/spark/commit/99056c47af4376467d1117ee0ef5018c6bf8985c).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-734790387


   **[Test build #131868 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131868/testReport)** for PR 29893 at commit [`6f9b42e`](https://github.com/apache/spark/commit/6f9b42e963b46fb8fa049c4c88909d048b5b1b66).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] wangyum commented on a change in pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
wangyum commented on a change in pull request #29893:
URL: https://github.com/apache/spark/pull/29893#discussion_r502146835



##########
File path: sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
##########
@@ -337,8 +337,8 @@ query
     ;
 
 insertInto
-    : INSERT OVERWRITE TABLE? multipartIdentifier (partitionSpec (IF NOT EXISTS)?)?                         #insertOverwriteTable
-    | INSERT INTO TABLE? multipartIdentifier partitionSpec? (IF NOT EXISTS)?                                #insertIntoTable
+    : INSERT OVERWRITE TABLE? multipartIdentifier identifierList?  (partitionSpec (IF NOT EXISTS)?)?        #insertOverwriteTable
+    | INSERT INTO TABLE? multipartIdentifier identifierList?  partitionSpec? (IF NOT EXISTS)?               #insertIntoTable

Review comment:
       Move `identifierList?` to the end? This is the order of Hive:
   ```sql
   hive> SELECT version();
   2.3.7 rcb213d88304034393d68cc31a95be24f5aac62b6
   hive> CREATE TABLE spark_32976(id int, name string) PARTITIONED BY (part string);
   hive> INSERT INTO spark_32976 PARTITION (part = '1') (name) SELECT 'name1';
   hive> SELECT * FROM spark_32976;
   NULL	name1	1
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-734141724






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-704259569


   can we enrich the PR description to explain how this feature is implemented in the analyzer phase? e.g. how we resolve logical plans for insertion.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] yaooqinn commented on a change in pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #29893:
URL: https://github.com/apache/spark/pull/29893#discussion_r531496514



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -3104,6 +3106,48 @@ class Analyzer(override val catalogManager: CatalogManager)
     }
   }
 
+  object ResolveUserSpecifiedColumns extends Rule[LogicalPlan] {
+    override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperators {
+      case i: InsertIntoStatement if i.table.resolved && i.query.resolved &&
+          i.userSpecifiedCols.nonEmpty =>
+        val resolved = resolveUserSpecifiedColumns(i)
+        val projection = addColumnListOnQuery(i.table.output, resolved, i.query)
+        i.copy(userSpecifiedCols = Nil, query = projection)
+    }
+
+    private def resolveUserSpecifiedColumns(i: InsertIntoStatement): Seq[NamedExpression] = {
+      SchemaUtils.checkColumnNameDuplication(
+        i.userSpecifiedCols, "in the column list", resolver)
+
+      i.userSpecifiedCols.map { col =>
+          i.table.resolve(Seq(col), resolver)
+            .getOrElse(i.table.failAnalysis(s"Cannot resolve column name $col"))
+      }
+    }
+
+    private def addColumnListOnQuery(
+        tableOutput: Seq[Attribute],
+        cols: Seq[NamedExpression],
+        query: LogicalPlan): LogicalPlan = {
+      if (cols.size != query.output.size) {
+        query.failAnalysis(
+          s"""Cannot write to table due to mismatched user specified columns and data columns:
+             |Specified columns: ${cols.map(c => s"'${c.name}'").mkString(", ")}
+             |Data columns: ${query.output.map(c => s"'${c.name}'").mkString(", ")}"""

Review comment:
       OK, btw, org.apache.spark.sql.catalyst.analysis.TableOutputResolver#resolveOutputColumns seems to have the same issuse?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700409574


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29893:
URL: https://github.com/apache/spark/pull/29893#discussion_r531447037



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala
##########
@@ -0,0 +1,230 @@
+/*
+ * 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
+
+import org.apache.spark.SparkConf
+import org.apache.spark.sql.connector.InMemoryPartitionTableCatalog
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils}
+
+/**
+ * The base trait for DML - insert syntax
+ */
+trait SQLInsertTestSuite extends QueryTest with SQLTestUtils {
+
+  import testImplicits._
+
+  def format: String
+
+  protected def createTable(
+      table: String,
+      cols: Seq[String],
+      colTypes: Seq[String],
+      partCols: Seq[String] = Nil): Unit = {
+    val values = cols.zip(colTypes).map(tuple => tuple._1 + " " + tuple._2).mkString("(", ", ", ")")
+    val partitionSpec = if (partCols.nonEmpty) {
+      partCols.mkString("PARTITIONED BY (", ",", ")")
+    } else ""
+    sql(s"CREATE TABLE $table$values USING $format $partitionSpec")
+  }
+
+  protected def processInsert(
+      tableName: String,
+      insert: DataFrame,
+      cols: Seq[String] = Nil,
+      partitionExprs: Seq[String] = Nil,
+      overwrite: Boolean): Unit = {
+    val tmpView = "tmp_view"
+    val columnList = if (cols.nonEmpty) cols.mkString("(", ",", ")") else ""
+    val partitionList = if (partitionExprs.nonEmpty) {
+      partitionExprs.mkString("PARTITION (", ",", ")")
+    } else ""
+    withTempView(tmpView) {
+      insert.createOrReplaceTempView(tmpView)
+      val overwriteStr = if (overwrite) "OVERWRITE" else "INTO"
+      sql(
+        s"INSERT $overwriteStr TABLE $tableName $partitionList $columnList SELECT * FROM $tmpView")
+    }
+  }
+
+  protected def verifyTable(tableName: String, expected: DataFrame): Unit = {
+    checkAnswer(spark.table(tableName), expected)
+  }
+
+  test("insert with column list - follow table output order") {
+    withTable("t1") {
+      val df = Seq((1, 2L, "3")).toDF()
+      val cols = Seq("c1", "c2", "c3")
+      createTable("t1", cols, Seq("int", "long", "string"))
+      Seq(false, true).foreach { m =>
+        processInsert("t1", df, cols, overwrite = m)
+        verifyTable("t1", df)
+      }
+    }
+
+    val cols = Seq("c1", "c2", "c3", "c4")
+    val df = Seq((1, 2, 3, 4)).toDF(cols: _*)
+    withTable("t1") {
+      createTable("t1", cols, Seq("int", "int", "int", "int"), cols.takeRight(2))
+      Seq(false, true).foreach { m =>
+        processInsert("t1", df, cols, overwrite = m)
+        verifyTable("t1", df)
+      }
+    }
+  }
+
+  test("insert with column list - follow table output order + partitioned table") {
+    val cols = Seq("c1", "c2", "c3", "c4")
+    val df = Seq((1, 2, 3, 4)).toDF(cols: _*)
+
+    withTable("t1") {
+      createTable("t1", cols, Seq("int", "int", "int", "int"), cols.takeRight(2))
+      Seq(false, true).foreach { m =>
+        processInsert(
+          "t1", df.selectExpr("c1", "c2"), cols.take(2), Seq("c3=3", "c4=4"), overwrite = m)
+        verifyTable("t1", df)
+      }
+    }
+
+    withTable("t1") {
+      createTable("t1", cols, Seq("int", "int", "int", "int"), cols.takeRight(2))
+      Seq(false, true).foreach { m =>
+        processInsert("t1", df.selectExpr("c1", "c2", "c4"),
+          cols.filterNot(_ == "c3"), Seq("c3=3", "c4"), overwrite = m)
+        verifyTable("t1", df)
+      }
+    }
+  }
+
+  test("insert with column list - table output reorder") {
+    withTable("t1") {
+      val cols = Seq("c1", "c2", "c3")
+      val df = Seq((1, 2, 3)).toDF(cols: _*)
+      createTable("t1", cols, Seq("int", "int", "int"))
+      Seq(false, true).foreach { m =>
+        processInsert("t1", df, cols.reverse, overwrite = m)
+        verifyTable("t1", df.selectExpr(cols.reverse: _*))
+      }
+    }
+
+    val cols = Seq("c1", "c2", "c3", "c4")
+    val df = Seq((1, 2, 3, 4)).toDF(cols: _*)
+    withTable("t1") {
+      createTable("t1", cols, Seq("int", "int", "int", "int"), cols.takeRight(2))
+      Seq(false, true).foreach { m =>
+        processInsert("t1", df, cols.reverse, overwrite = m)
+        verifyTable("t1", df.selectExpr(cols.reverse: _*))
+      }
+    }
+  }
+
+  test("insert with column list - table output reorder  + partitioned table") {
+    val cols = Seq("c1", "c2", "c3", "c4")
+    val df = Seq((1, 2, 3, 4)).toDF(cols: _*)
+
+    withTable("t1") {
+      createTable("t1", cols, Seq("int", "int", "int", "int"), cols.takeRight(2))
+      Seq(false, true).foreach { m =>
+        processInsert(
+          "t1", df.selectExpr("c1", "c2"), cols.take(2).reverse, Seq("c3=3", "c4=4"), overwrite = m)
+        verifyTable("t1", df.selectExpr("c2", "c1", "c3", "c4"))
+      }
+    }
+
+    withTable("t1") {
+      createTable("t1", cols, Seq("int", "int", "int", "int"), cols.takeRight(2))
+      Seq(false, true).foreach { m =>
+        processInsert("t1",
+          df.selectExpr("c1", "c2", "c4"), Seq("c4", "c2", "c1"), Seq("c3=3", "c4"), overwrite = m)
+        verifyTable("t1", df.selectExpr("c4", "c2", "c3", "c1"))
+      }
+    }
+  }
+
+  test("insert with column list - duplicated columns") {
+    val t1 = "t1"
+    withTable(t1) {
+      val cols = Seq("c1", "c2", "c3")
+      createTable(t1, cols, Seq("int", "long", "string"))
+      val e1 = intercept[AnalysisException](sql(s"INSERT INTO $t1 (c1, c2, c2) values(1, 2, 3)"))
+      assert(e1.getMessage === "Found duplicate column(s) in the column list: `c2`;")
+    }
+  }
+
+  test("insert with column list - invalid columns") {
+    val t1 = "t1"
+    withTable(t1) {
+      val cols = Seq("c1", "c2", "c3")
+      createTable(t1, cols, Seq("int", "long", "string"))
+      val e1 = intercept[AnalysisException](sql(s"INSERT INTO $t1 (c1, c2, c4) values(1, 2, 3)"))
+      assert(e1.getMessage === "Cannot resolve column name c4;")
+    }
+  }
+
+  test("insert with column list - mismatched column list size") {
+    val t1 = "t1"
+    val msg = "Cannot write to table due to mismatched user specified columns and data columns"
+    withTable(t1) {
+      val cols = Seq("c1", "c2", "c3")
+      createTable(t1, cols, Seq("int", "long", "string"))
+      val e1 = intercept[AnalysisException](sql(s"INSERT INTO $t1 (c1, c2) values(1, 2, 3)"))

Review comment:
       how about  `INSERT INTO $t1 (c1, c2) values(1, 2, 3)`

##########
File path: sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala
##########
@@ -0,0 +1,230 @@
+/*
+ * 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
+
+import org.apache.spark.SparkConf
+import org.apache.spark.sql.connector.InMemoryPartitionTableCatalog
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils}
+
+/**
+ * The base trait for DML - insert syntax
+ */
+trait SQLInsertTestSuite extends QueryTest with SQLTestUtils {
+
+  import testImplicits._
+
+  def format: String
+
+  protected def createTable(
+      table: String,
+      cols: Seq[String],
+      colTypes: Seq[String],
+      partCols: Seq[String] = Nil): Unit = {
+    val values = cols.zip(colTypes).map(tuple => tuple._1 + " " + tuple._2).mkString("(", ", ", ")")
+    val partitionSpec = if (partCols.nonEmpty) {
+      partCols.mkString("PARTITIONED BY (", ",", ")")
+    } else ""
+    sql(s"CREATE TABLE $table$values USING $format $partitionSpec")
+  }
+
+  protected def processInsert(
+      tableName: String,
+      insert: DataFrame,
+      cols: Seq[String] = Nil,
+      partitionExprs: Seq[String] = Nil,
+      overwrite: Boolean): Unit = {
+    val tmpView = "tmp_view"
+    val columnList = if (cols.nonEmpty) cols.mkString("(", ",", ")") else ""
+    val partitionList = if (partitionExprs.nonEmpty) {
+      partitionExprs.mkString("PARTITION (", ",", ")")
+    } else ""
+    withTempView(tmpView) {
+      insert.createOrReplaceTempView(tmpView)
+      val overwriteStr = if (overwrite) "OVERWRITE" else "INTO"
+      sql(
+        s"INSERT $overwriteStr TABLE $tableName $partitionList $columnList SELECT * FROM $tmpView")
+    }
+  }
+
+  protected def verifyTable(tableName: String, expected: DataFrame): Unit = {
+    checkAnswer(spark.table(tableName), expected)
+  }
+
+  test("insert with column list - follow table output order") {
+    withTable("t1") {
+      val df = Seq((1, 2L, "3")).toDF()
+      val cols = Seq("c1", "c2", "c3")
+      createTable("t1", cols, Seq("int", "long", "string"))
+      Seq(false, true).foreach { m =>
+        processInsert("t1", df, cols, overwrite = m)
+        verifyTable("t1", df)
+      }
+    }
+
+    val cols = Seq("c1", "c2", "c3", "c4")
+    val df = Seq((1, 2, 3, 4)).toDF(cols: _*)
+    withTable("t1") {
+      createTable("t1", cols, Seq("int", "int", "int", "int"), cols.takeRight(2))
+      Seq(false, true).foreach { m =>
+        processInsert("t1", df, cols, overwrite = m)
+        verifyTable("t1", df)
+      }
+    }
+  }
+
+  test("insert with column list - follow table output order + partitioned table") {
+    val cols = Seq("c1", "c2", "c3", "c4")
+    val df = Seq((1, 2, 3, 4)).toDF(cols: _*)
+
+    withTable("t1") {
+      createTable("t1", cols, Seq("int", "int", "int", "int"), cols.takeRight(2))
+      Seq(false, true).foreach { m =>
+        processInsert(
+          "t1", df.selectExpr("c1", "c2"), cols.take(2), Seq("c3=3", "c4=4"), overwrite = m)
+        verifyTable("t1", df)
+      }
+    }
+
+    withTable("t1") {
+      createTable("t1", cols, Seq("int", "int", "int", "int"), cols.takeRight(2))
+      Seq(false, true).foreach { m =>
+        processInsert("t1", df.selectExpr("c1", "c2", "c4"),
+          cols.filterNot(_ == "c3"), Seq("c3=3", "c4"), overwrite = m)
+        verifyTable("t1", df)
+      }
+    }
+  }
+
+  test("insert with column list - table output reorder") {
+    withTable("t1") {
+      val cols = Seq("c1", "c2", "c3")
+      val df = Seq((1, 2, 3)).toDF(cols: _*)
+      createTable("t1", cols, Seq("int", "int", "int"))
+      Seq(false, true).foreach { m =>
+        processInsert("t1", df, cols.reverse, overwrite = m)
+        verifyTable("t1", df.selectExpr(cols.reverse: _*))
+      }
+    }
+
+    val cols = Seq("c1", "c2", "c3", "c4")
+    val df = Seq((1, 2, 3, 4)).toDF(cols: _*)
+    withTable("t1") {
+      createTable("t1", cols, Seq("int", "int", "int", "int"), cols.takeRight(2))
+      Seq(false, true).foreach { m =>
+        processInsert("t1", df, cols.reverse, overwrite = m)
+        verifyTable("t1", df.selectExpr(cols.reverse: _*))
+      }
+    }
+  }
+
+  test("insert with column list - table output reorder  + partitioned table") {
+    val cols = Seq("c1", "c2", "c3", "c4")
+    val df = Seq((1, 2, 3, 4)).toDF(cols: _*)
+
+    withTable("t1") {
+      createTable("t1", cols, Seq("int", "int", "int", "int"), cols.takeRight(2))
+      Seq(false, true).foreach { m =>
+        processInsert(
+          "t1", df.selectExpr("c1", "c2"), cols.take(2).reverse, Seq("c3=3", "c4=4"), overwrite = m)
+        verifyTable("t1", df.selectExpr("c2", "c1", "c3", "c4"))
+      }
+    }
+
+    withTable("t1") {
+      createTable("t1", cols, Seq("int", "int", "int", "int"), cols.takeRight(2))
+      Seq(false, true).foreach { m =>
+        processInsert("t1",
+          df.selectExpr("c1", "c2", "c4"), Seq("c4", "c2", "c1"), Seq("c3=3", "c4"), overwrite = m)
+        verifyTable("t1", df.selectExpr("c4", "c2", "c3", "c1"))
+      }
+    }
+  }
+
+  test("insert with column list - duplicated columns") {
+    val t1 = "t1"
+    withTable(t1) {
+      val cols = Seq("c1", "c2", "c3")
+      createTable(t1, cols, Seq("int", "long", "string"))
+      val e1 = intercept[AnalysisException](sql(s"INSERT INTO $t1 (c1, c2, c2) values(1, 2, 3)"))
+      assert(e1.getMessage === "Found duplicate column(s) in the column list: `c2`;")
+    }
+  }
+
+  test("insert with column list - invalid columns") {
+    val t1 = "t1"
+    withTable(t1) {
+      val cols = Seq("c1", "c2", "c3")
+      createTable(t1, cols, Seq("int", "long", "string"))
+      val e1 = intercept[AnalysisException](sql(s"INSERT INTO $t1 (c1, c2, c4) values(1, 2, 3)"))
+      assert(e1.getMessage === "Cannot resolve column name c4;")
+    }
+  }
+
+  test("insert with column list - mismatched column list size") {
+    val t1 = "t1"
+    val msg = "Cannot write to table due to mismatched user specified columns and data columns"
+    withTable(t1) {
+      val cols = Seq("c1", "c2", "c3")
+      createTable(t1, cols, Seq("int", "long", "string"))
+      val e1 = intercept[AnalysisException](sql(s"INSERT INTO $t1 (c1, c2) values(1, 2, 3)"))

Review comment:
       how about  `INSERT INTO $t1 (c1, c2) values(1, 2)`?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-733606715


   **[Test build #131766 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131766/testReport)** for PR 29893 at commit [`475e790`](https://github.com/apache/spark/commit/475e790f7eeae80c4123f93c74952eb24f056897).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-701236003






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-734382224






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] maropu commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
maropu commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-735604940


   Nice feature, @yaooqinn ! It would be nice to update the SQL doc, too, as you said above.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-702002653


   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/129299/
   Test PASSed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-707067125






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-734839504


   **[Test build #131877 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131877/testReport)** for PR 29893 at commit [`833ac41`](https://github.com/apache/spark/commit/833ac416aac680059ce95fffc09c3234e87e0757).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-706978106






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29893:
URL: https://github.com/apache/spark/pull/29893#discussion_r531418779



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala
##########
@@ -522,7 +523,8 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) {
         if (dynamicPartitionOverwrite) {
           OverwritePartitionsDynamic.byPosition(table, df.logicalPlan, extraOptions.toMap)
         } else {
-          OverwriteByExpression.byPosition(table, df.logicalPlan, Literal(true), extraOptions.toMap)
+          OverwriteByExpression.byPosition(
+            table, df.logicalPlan, Literal(true), extraOptions.toMap)

Review comment:
       unnecessry 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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700625947


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/33845/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-734269319






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29893:
URL: https://github.com/apache/spark/pull/29893#discussion_r531487331



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala
##########
@@ -0,0 +1,224 @@
+/*
+ * 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
+
+import org.apache.spark.SparkConf
+import org.apache.spark.sql.connector.InMemoryPartitionTableCatalog
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils}
+
+/**
+ * The base trait for DML - insert syntax
+ */
+trait SQLInsertTestSuite extends QueryTest with SQLTestUtils {
+
+  import testImplicits._
+
+  def format: String
+
+  protected def createTable(
+      table: String,
+      cols: Seq[String],
+      colTypes: Seq[String],
+      partCols: Seq[String] = Nil): Unit = {
+    val values = cols.zip(colTypes).map(tuple => tuple._1 + " " + tuple._2).mkString("(", ", ", ")")
+    val partitionSpec = if (partCols.nonEmpty) {
+      partCols.mkString("PARTITIONED BY (", ",", ")")
+    } else ""
+    sql(s"CREATE TABLE $table$values USING $format $partitionSpec")
+  }
+
+  protected def processInsert(
+      tableName: String,
+      insert: DataFrame,

Review comment:
       nit: `input`




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-733236746






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700647319






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-734790187






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700015609


   **[Test build #129186 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129186/testReport)** for PR 29893 at commit [`c0643f6`](https://github.com/apache/spark/commit/c0643f6cbd24b561e079172d1d8953ba9af9937d).
    * This patch **fails to build**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700644807


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700406362






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] gatorsmile commented on a change in pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
gatorsmile commented on a change in pull request #29893:
URL: https://github.com/apache/spark/pull/29893#discussion_r545303780



##########
File path: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSQLInsertTestSuite.scala
##########
@@ -0,0 +1,25 @@
+/*
+ * 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.hive
+
+import org.apache.spark.sql.SQLInsertTestSuite
+import org.apache.spark.sql.hive.test.TestHiveSingleton
+
+class HiveSQLInsertTestSuite extends SQLInsertTestSuite with TestHiveSingleton {

Review comment:
       We need to set ` spark.conf.set("hive.exec.dynamic.partition.mode", "nonstrict")` before executing this suite; otherwise, `test("insert with column list - follow table output order + partitioned table")` will fail. 
   
   The reason why it does not fail because some test cases [running before this suite] do not change the default value of `hive.exec.dynamic.partition.mode` back to `strict`. However, the order of test suite execution is not deterministic. 




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700644786


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/33848/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-734729717






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700639000


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/33849/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700773353






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700644807






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29893:
URL: https://github.com/apache/spark/pull/29893#discussion_r531446155



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala
##########
@@ -0,0 +1,230 @@
+/*
+ * 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
+
+import org.apache.spark.SparkConf
+import org.apache.spark.sql.connector.InMemoryPartitionTableCatalog
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils}
+
+/**
+ * The base trait for DML - insert syntax
+ */
+trait SQLInsertTestSuite extends QueryTest with SQLTestUtils {
+
+  import testImplicits._
+
+  def format: String
+
+  protected def createTable(
+      table: String,
+      cols: Seq[String],
+      colTypes: Seq[String],
+      partCols: Seq[String] = Nil): Unit = {
+    val values = cols.zip(colTypes).map(tuple => tuple._1 + " " + tuple._2).mkString("(", ", ", ")")
+    val partitionSpec = if (partCols.nonEmpty) {
+      partCols.mkString("PARTITIONED BY (", ",", ")")
+    } else ""
+    sql(s"CREATE TABLE $table$values USING $format $partitionSpec")
+  }
+
+  protected def processInsert(
+      tableName: String,
+      insert: DataFrame,
+      cols: Seq[String] = Nil,
+      partitionExprs: Seq[String] = Nil,
+      overwrite: Boolean): Unit = {
+    val tmpView = "tmp_view"
+    val columnList = if (cols.nonEmpty) cols.mkString("(", ",", ")") else ""
+    val partitionList = if (partitionExprs.nonEmpty) {
+      partitionExprs.mkString("PARTITION (", ",", ")")
+    } else ""
+    withTempView(tmpView) {
+      insert.createOrReplaceTempView(tmpView)
+      val overwriteStr = if (overwrite) "OVERWRITE" else "INTO"
+      sql(
+        s"INSERT $overwriteStr TABLE $tableName $partitionList $columnList SELECT * FROM $tmpView")
+    }
+  }
+
+  protected def verifyTable(tableName: String, expected: DataFrame): Unit = {
+    checkAnswer(spark.table(tableName), expected)
+  }
+
+  test("insert with column list - follow table output order") {
+    withTable("t1") {
+      val df = Seq((1, 2L, "3")).toDF()
+      val cols = Seq("c1", "c2", "c3")
+      createTable("t1", cols, Seq("int", "long", "string"))
+      Seq(false, true).foreach { m =>
+        processInsert("t1", df, cols, overwrite = m)
+        verifyTable("t1", df)
+      }
+    }
+
+    val cols = Seq("c1", "c2", "c3", "c4")
+    val df = Seq((1, 2, 3, 4)).toDF(cols: _*)
+    withTable("t1") {
+      createTable("t1", cols, Seq("int", "int", "int", "int"), cols.takeRight(2))
+      Seq(false, true).foreach { m =>
+        processInsert("t1", df, cols, overwrite = m)
+        verifyTable("t1", df)
+      }
+    }
+  }
+
+  test("insert with column list - follow table output order + partitioned table") {
+    val cols = Seq("c1", "c2", "c3", "c4")
+    val df = Seq((1, 2, 3, 4)).toDF(cols: _*)
+
+    withTable("t1") {
+      createTable("t1", cols, Seq("int", "int", "int", "int"), cols.takeRight(2))
+      Seq(false, true).foreach { m =>
+        processInsert(
+          "t1", df.selectExpr("c1", "c2"), cols.take(2), Seq("c3=3", "c4=4"), overwrite = m)
+        verifyTable("t1", df)
+      }
+    }
+
+    withTable("t1") {
+      createTable("t1", cols, Seq("int", "int", "int", "int"), cols.takeRight(2))
+      Seq(false, true).foreach { m =>
+        processInsert("t1", df.selectExpr("c1", "c2", "c4"),
+          cols.filterNot(_ == "c3"), Seq("c3=3", "c4"), overwrite = m)
+        verifyTable("t1", df)
+      }
+    }
+  }
+
+  test("insert with column list - table output reorder") {
+    withTable("t1") {
+      val cols = Seq("c1", "c2", "c3")
+      val df = Seq((1, 2, 3)).toDF(cols: _*)
+      createTable("t1", cols, Seq("int", "int", "int"))
+      Seq(false, true).foreach { m =>
+        processInsert("t1", df, cols.reverse, overwrite = m)
+        verifyTable("t1", df.selectExpr(cols.reverse: _*))
+      }
+    }
+
+    val cols = Seq("c1", "c2", "c3", "c4")
+    val df = Seq((1, 2, 3, 4)).toDF(cols: _*)
+    withTable("t1") {
+      createTable("t1", cols, Seq("int", "int", "int", "int"), cols.takeRight(2))

Review comment:
       ditto




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700022214


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/33802/
   Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700015643


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/129186/
   Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700634132


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/33848/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700625966






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-706913662


   **[Test build #129666 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129666/testReport)** for PR 29893 at commit [`c76a2aa`](https://github.com/apache/spark/commit/c76a2aacb86411ef9d0c58ac996b274dce49beb7).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-706913968


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-701240521


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/33883/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-733110226






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-733572124






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan closed pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
cloud-fan closed pull request #29893:
URL: https://github.com/apache/spark/pull/29893


   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29893:
URL: https://github.com/apache/spark/pull/29893#discussion_r531445867



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala
##########
@@ -0,0 +1,230 @@
+/*
+ * 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
+
+import org.apache.spark.SparkConf
+import org.apache.spark.sql.connector.InMemoryPartitionTableCatalog
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils}
+
+/**
+ * The base trait for DML - insert syntax
+ */
+trait SQLInsertTestSuite extends QueryTest with SQLTestUtils {
+
+  import testImplicits._
+
+  def format: String
+
+  protected def createTable(
+      table: String,
+      cols: Seq[String],
+      colTypes: Seq[String],
+      partCols: Seq[String] = Nil): Unit = {
+    val values = cols.zip(colTypes).map(tuple => tuple._1 + " " + tuple._2).mkString("(", ", ", ")")
+    val partitionSpec = if (partCols.nonEmpty) {
+      partCols.mkString("PARTITIONED BY (", ",", ")")
+    } else ""
+    sql(s"CREATE TABLE $table$values USING $format $partitionSpec")
+  }
+
+  protected def processInsert(
+      tableName: String,
+      insert: DataFrame,
+      cols: Seq[String] = Nil,
+      partitionExprs: Seq[String] = Nil,
+      overwrite: Boolean): Unit = {
+    val tmpView = "tmp_view"
+    val columnList = if (cols.nonEmpty) cols.mkString("(", ",", ")") else ""
+    val partitionList = if (partitionExprs.nonEmpty) {
+      partitionExprs.mkString("PARTITION (", ",", ")")
+    } else ""
+    withTempView(tmpView) {
+      insert.createOrReplaceTempView(tmpView)
+      val overwriteStr = if (overwrite) "OVERWRITE" else "INTO"
+      sql(
+        s"INSERT $overwriteStr TABLE $tableName $partitionList $columnList SELECT * FROM $tmpView")
+    }
+  }
+
+  protected def verifyTable(tableName: String, expected: DataFrame): Unit = {
+    checkAnswer(spark.table(tableName), expected)
+  }
+
+  test("insert with column list - follow table output order") {
+    withTable("t1") {
+      val df = Seq((1, 2L, "3")).toDF()
+      val cols = Seq("c1", "c2", "c3")
+      createTable("t1", cols, Seq("int", "long", "string"))
+      Seq(false, true).foreach { m =>
+        processInsert("t1", df, cols, overwrite = m)
+        verifyTable("t1", df)
+      }
+    }
+
+    val cols = Seq("c1", "c2", "c3", "c4")
+    val df = Seq((1, 2, 3, 4)).toDF(cols: _*)
+    withTable("t1") {
+      createTable("t1", cols, Seq("int", "int", "int", "int"), cols.takeRight(2))

Review comment:
       this is a partitioned table, should be put in the test  `insert with column list - follow table output order + partitioned table`




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29893:
URL: https://github.com/apache/spark/pull/29893#discussion_r531420156



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala
##########
@@ -0,0 +1,244 @@
+/*
+ * 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
+
+import org.apache.spark.SparkConf
+import org.apache.spark.sql.connector.InMemoryPartitionTableCatalog
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils}
+
+/**
+ * The base trait for DML - insert syntax
+ */
+trait SQLInsertTestSuite extends QueryTest with SQLTestUtils {
+
+  import testImplicits._
+
+  def format: String
+
+  protected def createTable(
+      table: String,
+      cols: Seq[String],
+      colTypes: Seq[String],
+      partCols: Seq[String] = Nil): Unit = {
+    val values = cols.zip(colTypes).map(tuple => tuple._1 + " " + tuple._2).mkString("(", ", ", ")")
+    val partitionSpec = if (partCols.nonEmpty) {
+      partCols.mkString("PARTITIONED BY (", ",", ")")
+    } else ""
+    sql(s"CREATE TABLE $table$values USING $format $partitionSpec")
+  }
+
+  protected def processInsert(
+      tableName: String,
+      insert: DataFrame,
+      cols: Seq[String] = Nil,
+      partitionExprs: Seq[String] = Nil,
+      mode: SaveMode): Unit = {
+    val tmpView = "tmp_view"
+    val columnList = if (cols.nonEmpty) cols.mkString("(", ",", ")") else ""
+    val partitionList = if (partitionExprs.nonEmpty) {
+      partitionExprs.mkString("PARTITION (", ",", ")")
+    } else ""
+    withTempView(tmpView) {
+      insert.createOrReplaceTempView(tmpView)
+      val overwrite = if (mode == SaveMode.Overwrite) "OVERWRITE" else "INTO"
+      sql(s"INSERT $overwrite TABLE $tableName $partitionList $columnList SELECT * FROM $tmpView")
+    }
+  }
+
+  protected def verifyTable(tableName: String, expected: DataFrame): Unit = {
+    checkAnswer(spark.table(tableName), expected)
+  }
+
+  test("insert with column list - follow table output order") {
+    val t1 = "t1"
+    withTable(t1) {
+      val df = Seq((1, 2L, "3")).toDF()
+      val cols = Seq("c1", "c2", "c3")
+      createTable(t1, cols, Seq("int", "long", "string"))

Review comment:
       it's more readable if we pass schema string directly `createTable(t1, "c1 int, c2 long, c3 string")`




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-735108115






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-734853888






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-733236746






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] yaooqinn commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-733848715


   I will address the comment for the main code change first but leave the test part tomorrow. It’s too late now :)


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-733945204






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] yaooqinn commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-735559480


   thanks for merging. do we need to update the SQL reference doc too?


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-701338944






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29893:
URL: https://github.com/apache/spark/pull/29893#discussion_r545308284



##########
File path: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSQLInsertTestSuite.scala
##########
@@ -0,0 +1,25 @@
+/*
+ * 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.hive
+
+import org.apache.spark.sql.SQLInsertTestSuite
+import org.apache.spark.sql.hive.test.TestHiveSingleton
+
+class HiveSQLInsertTestSuite extends SQLInsertTestSuite with TestHiveSingleton {

Review comment:
       Good catch! We can follow `HiveCharVarcharTestSuite` to set/restore this conf.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29893:
URL: https://github.com/apache/spark/pull/29893#discussion_r531486612



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -3104,6 +3106,48 @@ class Analyzer(override val catalogManager: CatalogManager)
     }
   }
 
+  object ResolveUserSpecifiedColumns extends Rule[LogicalPlan] {
+    override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperators {
+      case i: InsertIntoStatement if i.table.resolved && i.query.resolved &&
+          i.userSpecifiedCols.nonEmpty =>
+        val resolved = resolveUserSpecifiedColumns(i)
+        val projection = addColumnListOnQuery(i.table.output, resolved, i.query)
+        i.copy(userSpecifiedCols = Nil, query = projection)
+    }
+
+    private def resolveUserSpecifiedColumns(i: InsertIntoStatement): Seq[NamedExpression] = {
+      SchemaUtils.checkColumnNameDuplication(
+        i.userSpecifiedCols, "in the column list", resolver)
+
+      i.userSpecifiedCols.map { col =>
+          i.table.resolve(Seq(col), resolver)
+            .getOrElse(i.table.failAnalysis(s"Cannot resolve column name $col"))
+      }
+    }
+
+    private def addColumnListOnQuery(
+        tableOutput: Seq[Attribute],
+        cols: Seq[NamedExpression],
+        query: LogicalPlan): LogicalPlan = {
+      if (cols.size != query.output.size) {
+        query.failAnalysis(
+          s"""Cannot write to table due to mismatched user specified columns and data columns:
+             |Specified columns: ${cols.map(c => s"'${c.name}'").mkString(", ")}
+             |Data columns: ${query.output.map(c => s"'${c.name}'").mkString(", ")}"""

Review comment:
       The data column names don't need to match the user-specified column names, e.g. `INSERT INTO t(a, b) SELECT 1, 2`. The thing only matters is the number of columns. Can we refine the error message?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700771475


   **[Test build #129229 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129229/testReport)** for PR 29893 at commit [`2b164ed`](https://github.com/apache/spark/commit/2b164eddd80145e9104546a7e7b41d340fd9481f).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] yaooqinn commented on a change in pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #29893:
URL: https://github.com/apache/spark/pull/29893#discussion_r530542829



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala
##########
@@ -290,6 +291,7 @@ case class DropViewStatement(
 case class InsertIntoStatement(
     table: LogicalPlan,
     partitionSpec: Map[String, Option[String]],
+    userSpecifiedCols: Seq[Attribute],

Review comment:
       ah, `String` here is much simpler and more rational.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] yaooqinn commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-735626164


   OK~, I'll make a followup very very soon


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-733068657






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29893:
URL: https://github.com/apache/spark/pull/29893#discussion_r531419418



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala
##########
@@ -0,0 +1,244 @@
+/*
+ * 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
+
+import org.apache.spark.SparkConf
+import org.apache.spark.sql.connector.InMemoryPartitionTableCatalog
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils}
+
+/**
+ * The base trait for DML - insert syntax
+ */
+trait SQLInsertTestSuite extends QueryTest with SQLTestUtils {
+
+  import testImplicits._
+
+  def format: String
+
+  protected def createTable(
+      table: String,
+      cols: Seq[String],
+      colTypes: Seq[String],
+      partCols: Seq[String] = Nil): Unit = {
+    val values = cols.zip(colTypes).map(tuple => tuple._1 + " " + tuple._2).mkString("(", ", ", ")")
+    val partitionSpec = if (partCols.nonEmpty) {
+      partCols.mkString("PARTITIONED BY (", ",", ")")
+    } else ""
+    sql(s"CREATE TABLE $table$values USING $format $partitionSpec")
+  }
+
+  protected def processInsert(
+      tableName: String,
+      insert: DataFrame,
+      cols: Seq[String] = Nil,
+      partitionExprs: Seq[String] = Nil,
+      mode: SaveMode): Unit = {
+    val tmpView = "tmp_view"
+    val columnList = if (cols.nonEmpty) cols.mkString("(", ",", ")") else ""
+    val partitionList = if (partitionExprs.nonEmpty) {
+      partitionExprs.mkString("PARTITION (", ",", ")")
+    } else ""
+    withTempView(tmpView) {
+      insert.createOrReplaceTempView(tmpView)
+      val overwrite = if (mode == SaveMode.Overwrite) "OVERWRITE" else "INTO"

Review comment:
       It's weird to see `SaveMode` in SQL test. Can we use `overwrite: 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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-735033393


   **[Test build #131889 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131889/testReport)** for PR 29893 at commit [`17d0564`](https://github.com/apache/spark/commit/17d056467751785664b6fd5c89c602f7c3e07e94).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-735553015


   thanks, merging to master!


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-734064221






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] yaooqinn commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-733501529






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] gatorsmile commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
gatorsmile commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-701877925


   Thanks for working on it! Happy Holiday! 


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-701875342






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-701210256


   **[Test build #129266 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129266/testReport)** for PR 29893 at commit [`ccd11f3`](https://github.com/apache/spark/commit/ccd11f36a1cbc32d9f123e8a821bd0d2169ca949).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-701240547






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] yaooqinn commented on a change in pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #29893:
URL: https://github.com/apache/spark/pull/29893#discussion_r545940081



##########
File path: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSQLInsertTestSuite.scala
##########
@@ -0,0 +1,25 @@
+/*
+ * 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.hive
+
+import org.apache.spark.sql.SQLInsertTestSuite
+import org.apache.spark.sql.hive.test.TestHiveSingleton
+
+class HiveSQLInsertTestSuite extends SQLInsertTestSuite with TestHiveSingleton {

Review comment:
       I have submitted a followup




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-701338213


   **[Test build #129265 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129265/testReport)** for PR 29893 at commit [`4af3a63`](https://github.com/apache/spark/commit/4af3a63aee8a2526f36b42702bcae85b18b44a9a).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700406369


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/33818/
   Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29893:
URL: https://github.com/apache/spark/pull/29893#discussion_r531421389



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala
##########
@@ -0,0 +1,244 @@
+/*
+ * 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
+
+import org.apache.spark.SparkConf
+import org.apache.spark.sql.connector.InMemoryPartitionTableCatalog
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils}
+
+/**
+ * The base trait for DML - insert syntax
+ */
+trait SQLInsertTestSuite extends QueryTest with SQLTestUtils {
+
+  import testImplicits._
+
+  def format: String
+
+  protected def createTable(
+      table: String,
+      cols: Seq[String],
+      colTypes: Seq[String],
+      partCols: Seq[String] = Nil): Unit = {
+    val values = cols.zip(colTypes).map(tuple => tuple._1 + " " + tuple._2).mkString("(", ", ", ")")
+    val partitionSpec = if (partCols.nonEmpty) {
+      partCols.mkString("PARTITIONED BY (", ",", ")")
+    } else ""
+    sql(s"CREATE TABLE $table$values USING $format $partitionSpec")
+  }
+
+  protected def processInsert(
+      tableName: String,
+      insert: DataFrame,
+      cols: Seq[String] = Nil,
+      partitionExprs: Seq[String] = Nil,
+      mode: SaveMode): Unit = {
+    val tmpView = "tmp_view"
+    val columnList = if (cols.nonEmpty) cols.mkString("(", ",", ")") else ""
+    val partitionList = if (partitionExprs.nonEmpty) {
+      partitionExprs.mkString("PARTITION (", ",", ")")
+    } else ""
+    withTempView(tmpView) {
+      insert.createOrReplaceTempView(tmpView)
+      val overwrite = if (mode == SaveMode.Overwrite) "OVERWRITE" else "INTO"
+      sql(s"INSERT $overwrite TABLE $tableName $partitionList $columnList SELECT * FROM $tmpView")
+    }
+  }
+
+  protected def verifyTable(tableName: String, expected: DataFrame): Unit = {
+    checkAnswer(spark.table(tableName), expected)
+  }
+
+  test("insert with column list - follow table output order") {
+    val t1 = "t1"
+    withTable(t1) {
+      val df = Seq((1, 2L, "3")).toDF()
+      val cols = Seq("c1", "c2", "c3")
+      createTable(t1, cols, Seq("int", "long", "string"))
+      Seq(SaveMode.Append, SaveMode.Overwrite).foreach { m =>
+        processInsert(t1, df, cols, mode = m)
+        verifyTable(t1, df)
+      }
+    }
+
+    val cols = Seq("c1", "c2", "c3", "c4")
+    val df = Seq((1, 2, 3, 4)).toDF(cols: _*)
+    withTable(t1) {
+      createTable(t1, cols, Seq("int", "int", "int", "int"), cols.takeRight(2))

Review comment:
       we should keep each test case small. This should be a new test case: `insert with column list - follow table output order + partitioned table`




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-735037962






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700559640


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/33837/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700012113






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-734761624






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-706923142






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700012122






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-701338105


   **[Test build #129266 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129266/testReport)** for PR 29893 at commit [`ccd11f3`](https://github.com/apache/spark/commit/ccd11f36a1cbc32d9f123e8a821bd0d2169ca949).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-701226871


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/33882/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-706978116


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/34277/
   Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-733868465


   **[Test build #131796 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131796/testReport)** for PR 29893 at commit [`86d0032`](https://github.com/apache/spark/commit/86d0032ad00f7a1f10e1963070e39a24e640998d).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29893:
URL: https://github.com/apache/spark/pull/29893#discussion_r531421673



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala
##########
@@ -0,0 +1,244 @@
+/*
+ * 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
+
+import org.apache.spark.SparkConf
+import org.apache.spark.sql.connector.InMemoryPartitionTableCatalog
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils}
+
+/**
+ * The base trait for DML - insert syntax
+ */
+trait SQLInsertTestSuite extends QueryTest with SQLTestUtils {
+
+  import testImplicits._
+
+  def format: String
+
+  protected def createTable(
+      table: String,
+      cols: Seq[String],
+      colTypes: Seq[String],
+      partCols: Seq[String] = Nil): Unit = {
+    val values = cols.zip(colTypes).map(tuple => tuple._1 + " " + tuple._2).mkString("(", ", ", ")")
+    val partitionSpec = if (partCols.nonEmpty) {
+      partCols.mkString("PARTITIONED BY (", ",", ")")
+    } else ""
+    sql(s"CREATE TABLE $table$values USING $format $partitionSpec")
+  }
+
+  protected def processInsert(
+      tableName: String,
+      insert: DataFrame,
+      cols: Seq[String] = Nil,
+      partitionExprs: Seq[String] = Nil,
+      mode: SaveMode): Unit = {
+    val tmpView = "tmp_view"
+    val columnList = if (cols.nonEmpty) cols.mkString("(", ",", ")") else ""
+    val partitionList = if (partitionExprs.nonEmpty) {
+      partitionExprs.mkString("PARTITION (", ",", ")")
+    } else ""
+    withTempView(tmpView) {
+      insert.createOrReplaceTempView(tmpView)
+      val overwrite = if (mode == SaveMode.Overwrite) "OVERWRITE" else "INTO"
+      sql(s"INSERT $overwrite TABLE $tableName $partitionList $columnList SELECT * FROM $tmpView")
+    }
+  }
+
+  protected def verifyTable(tableName: String, expected: DataFrame): Unit = {
+    checkAnswer(spark.table(tableName), expected)
+  }
+
+  test("insert with column list - follow table output order") {
+    val t1 = "t1"
+    withTable(t1) {
+      val df = Seq((1, 2L, "3")).toDF()
+      val cols = Seq("c1", "c2", "c3")
+      createTable(t1, cols, Seq("int", "long", "string"))
+      Seq(SaveMode.Append, SaveMode.Overwrite).foreach { m =>
+        processInsert(t1, df, cols, mode = m)
+        verifyTable(t1, df)
+      }
+    }
+
+    val cols = Seq("c1", "c2", "c3", "c4")
+    val df = Seq((1, 2, 3, 4)).toDF(cols: _*)
+    withTable(t1) {
+      createTable(t1, cols, Seq("int", "int", "int", "int"), cols.takeRight(2))
+      Seq(SaveMode.Append, SaveMode.Overwrite).foreach { m =>
+        processInsert(t1, df, cols, mode = m)
+        verifyTable(t1, df)
+      }
+    }
+
+    withTable(t1) {

Review comment:
       ditto, `insert with column list - follow table output order + partitioned table and write with static partitions`




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-734689757


   **[Test build #131868 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131868/testReport)** for PR 29893 at commit [`6f9b42e`](https://github.com/apache/spark/commit/6f9b42e963b46fb8fa049c4c88909d048b5b1b66).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-734791143






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700851376






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700022203


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] yaooqinn commented on a change in pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #29893:
URL: https://github.com/apache/spark/pull/29893#discussion_r530538735



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
##########
@@ -299,13 +301,16 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg
   override def visitInsertIntoTable(
       ctx: InsertIntoTableContext): InsertTableParams = withOrigin(ctx) {
     val tableIdent = visitMultipartIdentifier(ctx.multipartIdentifier)
+    val cols = Option(ctx.identifierList())
+      .map(visitIdentifierList)

Review comment:
       unnecessary, it comes from a grammar that accepts a one-part name




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-733761484






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] yaooqinn commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-735030004


   retest this please


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700002465


   **[Test build #129185 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129185/testReport)** for PR 29893 at commit [`99056c4`](https://github.com/apache/spark/commit/99056c47af4376467d1117ee0ef5018c6bf8985c).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700406079


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/33818/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-733571838


   **[Test build #131750 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131750/testReport)** for PR 29893 at commit [`475e790`](https://github.com/apache/spark/commit/475e790f7eeae80c4123f93c74952eb24f056897).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-734046895


   **[Test build #131826 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131826/testReport)** for PR 29893 at commit [`713b5b9`](https://github.com/apache/spark/commit/713b5b9a1d47ca639f36b690a2435363e9eb4fff).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-706886780


   **[Test build #129666 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129666/testReport)** for PR 29893 at commit [`c76a2aa`](https://github.com/apache/spark/commit/c76a2aacb86411ef9d0c58ac996b274dce49beb7).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-735037962






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700409574






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-701210256






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700015631






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-706923120


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34270/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700007598


   Build finished. Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700022203






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-701837571


   **[Test build #129299 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129299/testReport)** for PR 29893 at commit [`a76c9ba`](https://github.com/apache/spark/commit/a76c9ba5e2c41b9aa68b6a5e170f7d2750af7514).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-701870023


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/33915/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-734689757


   **[Test build #131868 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131868/testReport)** for PR 29893 at commit [`6f9b42e`](https://github.com/apache/spark/commit/6f9b42e963b46fb8fa049c4c88909d048b5b1b66).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-732991019






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-707066215


   **[Test build #129673 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129673/testReport)** for PR 29893 at commit [`f7e8d18`](https://github.com/apache/spark/commit/f7e8d186320aa805d679942937b12b3a5141fb89).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-733899147


   **[Test build #131796 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131796/testReport)** for PR 29893 at commit [`86d0032`](https://github.com/apache/spark/commit/86d0032ad00f7a1f10e1963070e39a24e640998d).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-702002639






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-734763034


   **[Test build #131871 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131871/testReport)** for PR 29893 at commit [`7680da2`](https://github.com/apache/spark/commit/7680da271c3884093a95f7352fe6e9067c537430).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-732991019






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29893:
URL: https://github.com/apache/spark/pull/29893#discussion_r531448088



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala
##########
@@ -0,0 +1,230 @@
+/*
+ * 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
+
+import org.apache.spark.SparkConf
+import org.apache.spark.sql.connector.InMemoryPartitionTableCatalog
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils}
+
+/**
+ * The base trait for DML - insert syntax
+ */
+trait SQLInsertTestSuite extends QueryTest with SQLTestUtils {
+
+  import testImplicits._
+
+  def format: String
+
+  protected def createTable(
+      table: String,
+      cols: Seq[String],
+      colTypes: Seq[String],
+      partCols: Seq[String] = Nil): Unit = {
+    val values = cols.zip(colTypes).map(tuple => tuple._1 + " " + tuple._2).mkString("(", ", ", ")")
+    val partitionSpec = if (partCols.nonEmpty) {
+      partCols.mkString("PARTITIONED BY (", ",", ")")
+    } else ""
+    sql(s"CREATE TABLE $table$values USING $format $partitionSpec")
+  }
+
+  protected def processInsert(
+      tableName: String,
+      insert: DataFrame,
+      cols: Seq[String] = Nil,
+      partitionExprs: Seq[String] = Nil,
+      overwrite: Boolean): Unit = {
+    val tmpView = "tmp_view"
+    val columnList = if (cols.nonEmpty) cols.mkString("(", ",", ")") else ""
+    val partitionList = if (partitionExprs.nonEmpty) {
+      partitionExprs.mkString("PARTITION (", ",", ")")
+    } else ""
+    withTempView(tmpView) {
+      insert.createOrReplaceTempView(tmpView)
+      val overwriteStr = if (overwrite) "OVERWRITE" else "INTO"
+      sql(
+        s"INSERT $overwriteStr TABLE $tableName $partitionList $columnList SELECT * FROM $tmpView")
+    }
+  }
+
+  protected def verifyTable(tableName: String, expected: DataFrame): Unit = {
+    checkAnswer(spark.table(tableName), expected)
+  }
+
+  test("insert with column list - follow table output order") {
+    withTable("t1") {
+      val df = Seq((1, 2L, "3")).toDF()
+      val cols = Seq("c1", "c2", "c3")
+      createTable("t1", cols, Seq("int", "long", "string"))
+      Seq(false, true).foreach { m =>
+        processInsert("t1", df, cols, overwrite = m)
+        verifyTable("t1", df)
+      }
+    }
+
+    val cols = Seq("c1", "c2", "c3", "c4")
+    val df = Seq((1, 2, 3, 4)).toDF(cols: _*)
+    withTable("t1") {
+      createTable("t1", cols, Seq("int", "int", "int", "int"), cols.takeRight(2))
+      Seq(false, true).foreach { m =>
+        processInsert("t1", df, cols, overwrite = m)
+        verifyTable("t1", df)
+      }
+    }
+  }
+
+  test("insert with column list - follow table output order + partitioned table") {
+    val cols = Seq("c1", "c2", "c3", "c4")
+    val df = Seq((1, 2, 3, 4)).toDF(cols: _*)
+
+    withTable("t1") {
+      createTable("t1", cols, Seq("int", "int", "int", "int"), cols.takeRight(2))
+      Seq(false, true).foreach { m =>
+        processInsert(
+          "t1", df.selectExpr("c1", "c2"), cols.take(2), Seq("c3=3", "c4=4"), overwrite = m)
+        verifyTable("t1", df)
+      }
+    }
+
+    withTable("t1") {
+      createTable("t1", cols, Seq("int", "int", "int", "int"), cols.takeRight(2))
+      Seq(false, true).foreach { m =>
+        processInsert("t1", df.selectExpr("c1", "c2", "c4"),
+          cols.filterNot(_ == "c3"), Seq("c3=3", "c4"), overwrite = m)
+        verifyTable("t1", df)
+      }
+    }
+  }
+
+  test("insert with column list - table output reorder") {
+    withTable("t1") {
+      val cols = Seq("c1", "c2", "c3")
+      val df = Seq((1, 2, 3)).toDF(cols: _*)
+      createTable("t1", cols, Seq("int", "int", "int"))
+      Seq(false, true).foreach { m =>
+        processInsert("t1", df, cols.reverse, overwrite = m)
+        verifyTable("t1", df.selectExpr(cols.reverse: _*))
+      }
+    }
+
+    val cols = Seq("c1", "c2", "c3", "c4")
+    val df = Seq((1, 2, 3, 4)).toDF(cols: _*)
+    withTable("t1") {
+      createTable("t1", cols, Seq("int", "int", "int", "int"), cols.takeRight(2))
+      Seq(false, true).foreach { m =>
+        processInsert("t1", df, cols.reverse, overwrite = m)
+        verifyTable("t1", df.selectExpr(cols.reverse: _*))
+      }
+    }
+  }
+
+  test("insert with column list - table output reorder  + partitioned table") {
+    val cols = Seq("c1", "c2", "c3", "c4")
+    val df = Seq((1, 2, 3, 4)).toDF(cols: _*)
+
+    withTable("t1") {
+      createTable("t1", cols, Seq("int", "int", "int", "int"), cols.takeRight(2))
+      Seq(false, true).foreach { m =>
+        processInsert(
+          "t1", df.selectExpr("c1", "c2"), cols.take(2).reverse, Seq("c3=3", "c4=4"), overwrite = m)
+        verifyTable("t1", df.selectExpr("c2", "c1", "c3", "c4"))
+      }
+    }
+
+    withTable("t1") {
+      createTable("t1", cols, Seq("int", "int", "int", "int"), cols.takeRight(2))
+      Seq(false, true).foreach { m =>
+        processInsert("t1",
+          df.selectExpr("c1", "c2", "c4"), Seq("c4", "c2", "c1"), Seq("c3=3", "c4"), overwrite = m)
+        verifyTable("t1", df.selectExpr("c4", "c2", "c3", "c1"))
+      }
+    }
+  }
+
+  test("insert with column list - duplicated columns") {
+    val t1 = "t1"
+    withTable(t1) {
+      val cols = Seq("c1", "c2", "c3")
+      createTable(t1, cols, Seq("int", "long", "string"))
+      val e1 = intercept[AnalysisException](sql(s"INSERT INTO $t1 (c1, c2, c2) values(1, 2, 3)"))
+      assert(e1.getMessage === "Found duplicate column(s) in the column list: `c2`;")
+    }
+  }
+
+  test("insert with column list - invalid columns") {
+    val t1 = "t1"
+    withTable(t1) {
+      val cols = Seq("c1", "c2", "c3")
+      createTable(t1, cols, Seq("int", "long", "string"))
+      val e1 = intercept[AnalysisException](sql(s"INSERT INTO $t1 (c1, c2, c4) values(1, 2, 3)"))
+      assert(e1.getMessage === "Cannot resolve column name c4;")
+    }
+  }
+
+  test("insert with column list - mismatched column list size") {
+    val t1 = "t1"
+    val msg = "Cannot write to table due to mismatched user specified columns and data columns"
+    withTable(t1) {
+      val cols = Seq("c1", "c2", "c3")
+      createTable(t1, cols, Seq("int", "long", "string"))
+      val e1 = intercept[AnalysisException](sql(s"INSERT INTO $t1 (c1, c2) values(1, 2, 3)"))
+      assert(e1.getMessage.contains(msg))
+      val e2 = intercept[AnalysisException](sql(s"INSERT INTO $t1 (c1, c2, c3) values(1, 2)"))
+      assert(e2.getMessage.contains(msg))
+    }
+  }
+
+  test("insert with column list - mismatched target table out size after rewritten query") {
+    val v2Msg = "Cannot write to 'testcat.t12', not enough data columns:"
+    val cols = Seq("c1", "c2", "c3", "c4")
+
+    withTable("t12") {

Review comment:
       can we use `t1`?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] yaooqinn commented on a change in pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #29893:
URL: https://github.com/apache/spark/pull/29893#discussion_r530958932



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
##########
@@ -3718,6 +3718,49 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark
       }
     }
   }
+
+  test("SPARK-32976: Support column list specification in insert into statement") {

Review comment:
       Based on this requirement, I create the base trait and 3 concrete test suites, but w/ `V1SQLInsertTestSuite` trait for bypassing different error handling between v2/v1 tables, which I guess is not the right thing to do in this PR.
   
   FYI, when the final query's output mismatches w/ the target table output, v1 throws errors at org.apache.spark.sql.execution.datasources.DataSourceAnalysis#convertStaticPartitions, while v2 at org.apache.spark.sql.catalyst.analysis.TableOutputResolver#resolveOutputColumns
   
   To merge these two parts rashly would introduce a large number of LOC in other test cases and may
    make this PR hard to review.
   
    




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-734869682


   **[Test build #131880 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131880/testReport)** for PR 29893 at commit [`17d0564`](https://github.com/apache/spark/commit/17d056467751785664b6fd5c89c602f7c3e07e94).
    * This patch **fails SparkR unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700515838


   **[Test build #129221 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129221/testReport)** for PR 29893 at commit [`e756a55`](https://github.com/apache/spark/commit/e756a558d89a077108d50103af441dffa4ba4263).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-706923142






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-733105453


   **[Test build #131678 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131678/testReport)** for PR 29893 at commit [`475e790`](https://github.com/apache/spark/commit/475e790f7eeae80c4123f93c74952eb24f056897).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-733068657






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-701339080






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-733512932


   **[Test build #131750 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131750/testReport)** for PR 29893 at commit [`475e790`](https://github.com/apache/spark/commit/475e790f7eeae80c4123f93c74952eb24f056897).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-734731578


   **[Test build #131877 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131877/testReport)** for PR 29893 at commit [`833ac41`](https://github.com/apache/spark/commit/833ac416aac680059ce95fffc09c3234e87e0757).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700620031


   **[Test build #129233 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129233/testReport)** for PR 29893 at commit [`3a02681`](https://github.com/apache/spark/commit/3a0268107996abf3bae6a969181b269b1fc19481).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700007607


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/129185/
   Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700647285


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/33849/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] yaooqinn commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-701834507


   cc @gatorsmile @cloud-fan @wangyum thanks


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-701206202


   **[Test build #129265 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129265/testReport)** for PR 29893 at commit [`4af3a63`](https://github.com/apache/spark/commit/4af3a63aee8a2526f36b42702bcae85b18b44a9a).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-706913984


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/129666/
   Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-706965990


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34277/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29893:
URL: https://github.com/apache/spark/pull/29893#discussion_r530471201



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
##########
@@ -3718,6 +3718,49 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark
       }
     }
   }
+
+  test("SPARK-32976: Support column list specification in insert into statement") {

Review comment:
       can we create a base trait `SQLInsertTestSuite` and move this test there?
   
   Then we can have 3 concrete test suites `FileSourceSQLInsertTestSuite`, `DSV2SQLInsertTestSuite` and `HiveSQLInsertTestSuite`




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700782473


   **[Test build #129232 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129232/testReport)** for PR 29893 at commit [`086cfa8`](https://github.com/apache/spark/commit/086cfa8260b1aba1bab774c80f8c3ff5c6d2f31a).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700409106


   **[Test build #129203 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129203/testReport)** for PR 29893 at commit [`40e2ffd`](https://github.com/apache/spark/commit/40e2ffd43ff7c41dcf6c162daf0d9bc672a86996).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-733666812






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700605102


   **[Test build #129232 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129232/testReport)** for PR 29893 at commit [`086cfa8`](https://github.com/apache/spark/commit/086cfa8260b1aba1bab774c80f8c3ff5c6d2f31a).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29893:
URL: https://github.com/apache/spark/pull/29893#discussion_r530459131



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
##########
@@ -299,13 +301,16 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg
   override def visitInsertIntoTable(
       ctx: InsertIntoTableContext): InsertTableParams = withOrigin(ctx) {
     val tableIdent = visitMultipartIdentifier(ctx.multipartIdentifier)
+    val cols = Option(ctx.identifierList())
+      .map(visitIdentifierList)

Review comment:
       Can we add a check that column name must be one part?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-733761484






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-734879648






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-733110226






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-734239060


   **[Test build #131844 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131844/testReport)** for PR 29893 at commit [`3afb4c0`](https://github.com/apache/spark/commit/3afb4c0f92fc9f549d3dbd962c263518a65f6915).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700647319


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-733032201


   **[Test build #131673 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131673/testReport)** for PR 29893 at commit [`c74d4d7`](https://github.com/apache/spark/commit/c74d4d7a0e0768fa5d9970a9e7ddd109306d6caa).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-701837571


   **[Test build #129299 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129299/testReport)** for PR 29893 at commit [`a76c9ba`](https://github.com/apache/spark/commit/a76c9ba5e2c41b9aa68b6a5e170f7d2750af7514).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-734761624






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-734239060


   **[Test build #131844 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131844/testReport)** for PR 29893 at commit [`3afb4c0`](https://github.com/apache/spark/commit/3afb4c0f92fc9f549d3dbd962c263518a65f6915).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-701875308


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/33915/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29893:
URL: https://github.com/apache/spark/pull/29893#discussion_r531422458



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala
##########
@@ -0,0 +1,244 @@
+/*
+ * 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
+
+import org.apache.spark.SparkConf
+import org.apache.spark.sql.connector.InMemoryPartitionTableCatalog
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils}
+
+/**
+ * The base trait for DML - insert syntax
+ */
+trait SQLInsertTestSuite extends QueryTest with SQLTestUtils {
+
+  import testImplicits._
+
+  def format: String
+
+  protected def createTable(
+      table: String,
+      cols: Seq[String],
+      colTypes: Seq[String],
+      partCols: Seq[String] = Nil): Unit = {
+    val values = cols.zip(colTypes).map(tuple => tuple._1 + " " + tuple._2).mkString("(", ", ", ")")
+    val partitionSpec = if (partCols.nonEmpty) {
+      partCols.mkString("PARTITIONED BY (", ",", ")")
+    } else ""
+    sql(s"CREATE TABLE $table$values USING $format $partitionSpec")
+  }
+
+  protected def processInsert(
+      tableName: String,
+      insert: DataFrame,
+      cols: Seq[String] = Nil,
+      partitionExprs: Seq[String] = Nil,
+      mode: SaveMode): Unit = {
+    val tmpView = "tmp_view"
+    val columnList = if (cols.nonEmpty) cols.mkString("(", ",", ")") else ""
+    val partitionList = if (partitionExprs.nonEmpty) {
+      partitionExprs.mkString("PARTITION (", ",", ")")
+    } else ""
+    withTempView(tmpView) {
+      insert.createOrReplaceTempView(tmpView)
+      val overwrite = if (mode == SaveMode.Overwrite) "OVERWRITE" else "INTO"
+      sql(s"INSERT $overwrite TABLE $tableName $partitionList $columnList SELECT * FROM $tmpView")
+    }
+  }
+
+  protected def verifyTable(tableName: String, expected: DataFrame): Unit = {
+    checkAnswer(spark.table(tableName), expected)
+  }
+
+  test("insert with column list - follow table output order") {
+    val t1 = "t1"
+    withTable(t1) {
+      val df = Seq((1, 2L, "3")).toDF()
+      val cols = Seq("c1", "c2", "c3")
+      createTable(t1, cols, Seq("int", "long", "string"))
+      Seq(SaveMode.Append, SaveMode.Overwrite).foreach { m =>
+        processInsert(t1, df, cols, mode = m)
+        verifyTable(t1, df)
+      }
+    }
+
+    val cols = Seq("c1", "c2", "c3", "c4")
+    val df = Seq((1, 2, 3, 4)).toDF(cols: _*)
+    withTable(t1) {
+      createTable(t1, cols, Seq("int", "int", "int", "int"), cols.takeRight(2))
+      Seq(SaveMode.Append, SaveMode.Overwrite).foreach { m =>
+        processInsert(t1, df, cols, mode = m)
+        verifyTable(t1, df)
+      }
+    }
+
+    withTable(t1) {
+      createTable(t1, cols, Seq("int", "int", "int", "int"), cols.takeRight(2))
+      Seq(SaveMode.Append, SaveMode.Overwrite).foreach { m =>
+        processInsert(t1, df.selectExpr("c1", "c2"), cols.take(2), Seq("c3=3", "c4=4"), mode = m)
+        verifyTable(t1, df)
+      }
+    }
+
+    withTable(t1) {
+      createTable(t1, cols, Seq("int", "int", "int", "int"), cols.takeRight(2))
+      Seq(SaveMode.Append, SaveMode.Overwrite).foreach { m =>
+        processInsert(t1,
+          df.selectExpr("c1", "c2", "c4"), cols.filterNot(_ == "c3"), Seq("c3=3", "c4"), mode = m)
+        verifyTable(t1, df)
+      }
+    }
+  }
+
+  test("insert with column list - table output reorder") {
+    val t1 = "t1"
+    withTable(t1) {
+      val cols = Seq("c1", "c2", "c3")
+      val df = Seq((1, 2, 3)).toDF(cols: _*)
+      createTable(t1, cols, Seq("int", "int", "int"))
+      Seq(SaveMode.Append, SaveMode.Overwrite).foreach { m =>
+        processInsert(t1, df, cols.reverse, mode = m)
+        verifyTable(t1, df.selectExpr(cols.reverse : _*))
+      }
+    }
+
+    val cols = Seq("c1", "c2", "c3", "c4")
+    val df = Seq((1, 2, 3, 4)).toDF(cols: _*)
+    withTable(t1) {
+      createTable(t1, cols, Seq("int", "int", "int", "int"), cols.takeRight(2))
+      Seq(SaveMode.Append, SaveMode.Overwrite).foreach { m =>
+        processInsert(t1, df, cols.reverse, mode = m)
+        verifyTable(t1, df.selectExpr(cols.reverse : _*))
+      }
+    }
+
+    withTable(t1) {
+      createTable(t1, cols, Seq("int", "int", "int", "int"), cols.takeRight(2))
+      Seq(SaveMode.Append, SaveMode.Overwrite).foreach { m =>
+        processInsert(
+          t1, df.selectExpr("c1", "c2"), cols.take(2).reverse, Seq("c3=3", "c4=4"), mode = m)
+        verifyTable(t1, df.selectExpr("c2", "c1", "c3", "c4"))
+      }
+    }
+
+    withTable(t1) {
+      createTable(t1, cols, Seq("int", "int", "int", "int"), cols.takeRight(2))
+      Seq(SaveMode.Append, SaveMode.Overwrite).foreach { m =>
+        processInsert(
+          t1, df.selectExpr("c1", "c2", "c4"), Seq("c4", "c2", "c1"), Seq("c3=3", "c4"), mode = m)
+        verifyTable(t1, df.selectExpr("c4", "c2", "c3", "c1"))
+      }
+    }
+  }
+
+  test("insert with column list - duplicated columns") {
+    val t1 = "t1"
+    withTable(t1) {
+      val cols = Seq("c1", "c2", "c3")
+      createTable(t1, cols, Seq("int", "long", "string"))
+      val e1 = intercept[AnalysisException](sql(s"INSERT INTO $t1 (c1, c2, c2) values(1, 2, 3)"))
+      assert(e1.getMessage === "Found duplicate column(s) in the column list: `c2`;")
+    }
+  }
+
+  test("insert with column list - invalid columns") {
+    val t1 = "t1"
+    withTable(t1) {
+      val cols = Seq("c1", "c2", "c3")
+      createTable(t1, cols, Seq("int", "long", "string"))
+      val e1 = intercept[AnalysisException](sql(s"INSERT INTO $t1 (c1, c2, c4) values(1, 2, 3)"))
+      assert(e1.getMessage === "Cannot resolve column name c4;")
+    }
+  }
+
+  test("insert with column list - mismatched column list size") {
+    val t1 = "t1"
+    val msg = "Cannot write to table due to mismatched user specified columns and data columns"
+    withTable(t1) {
+      val cols = Seq("c1", "c2", "c3")
+      createTable(t1, cols, Seq("int", "long", "string"))
+      val e1 = intercept[AnalysisException](sql(s"INSERT INTO $t1 (c1, c2) values(1, 2, 3)"))
+      assert(e1.getMessage.contains(msg))
+      val e2 = intercept[AnalysisException](sql(s"INSERT INTO $t1 (c1, c2, c3) values(1, 2)"))
+      assert(e2.getMessage.contains(msg))
+    }
+  }
+}
+
+/**
+ * TODO: Currently, v1 and v2 have different error handling in some cases, we should merge this to
+ * the base trait after unifying them
+ */
+trait V1SQLInsertTestSuite extends SQLInsertTestSuite {
+
+  test("insert with column list - mismatched target table out size w/ rewritten query") {
+    val table = "t1"
+
+    withTable(table) {
+      sql(s"CREATE TABLE $table(c1 INT, c2 INT, c3 INT) USING $format")
+      val e1 = intercept[AnalysisException](sql(s"INSERT INTO $table (c1) values(1)"))
+      assert(e1.getMessage.contains("target table has 3 column(s) but the inserted data has 1"))

Review comment:
       only error message is different? then we can put test in the base trait
   ```
   val e = intercept[AnalysisException]...
   assert(e.message.contains(DSV1Msg) || e.message.contains(DSV2Msg))
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-701875342


   Merged build finished. Test PASSed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-706913968






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700577090


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/33837/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-706923002


   **[Test build #129673 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129673/testReport)** for PR 29893 at commit [`f7e8d18`](https://github.com/apache/spark/commit/f7e8d186320aa805d679942937b12b3a5141fb89).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-701240547






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700700375






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700409576


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/129203/
   Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-701339080






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-733606715


   **[Test build #131766 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131766/testReport)** for PR 29893 at commit [`475e790`](https://github.com/apache/spark/commit/475e790f7eeae80c4123f93c74952eb24f056897).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-733798906






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700406362


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-701231971


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/33883/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700577119






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-735108115






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-734764118


   **[Test build #131880 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131880/testReport)** for PR 29893 at commit [`17d0564`](https://github.com/apache/spark/commit/17d056467751785664b6fd5c89c602f7c3e07e94).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700700375






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700783880






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700010855


   **[Test build #129186 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129186/testReport)** for PR 29893 at commit [`c0643f6`](https://github.com/apache/spark/commit/c0643f6cbd24b561e079172d1d8953ba9af9937d).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-706978078


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34277/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-733899415






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-734141724






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-707067125






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] yaooqinn commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-731582295


   HI @cloud-fan @gatorsmile,  I will focus on this next week. 


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-733572124






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-734879648






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700851376






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-734764118


   **[Test build #131880 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131880/testReport)** for PR 29893 at commit [`17d0564`](https://github.com/apache/spark/commit/17d056467751785664b6fd5c89c602f7c3e07e94).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-734705367


   **[Test build #131871 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131871/testReport)** for PR 29893 at commit [`7680da2`](https://github.com/apache/spark/commit/7680da271c3884093a95f7352fe6e9067c537430).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] gatorsmile commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
gatorsmile commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-731516438


   ping @yaooqinn  Could you help address the above comments? This is a nice feature we should have for ANSI SQL 


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700010855


   **[Test build #129186 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129186/testReport)** for PR 29893 at commit [`c0643f6`](https://github.com/apache/spark/commit/c0643f6cbd24b561e079172d1d8953ba9af9937d).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] maropu commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
maropu commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-749381388


   @yaooqinn kindly ping: I've filed jira so that we don't forget to do it. https://issues.apache.org/jira/browse/SPARK-33877 `branch-3.1` includes this commit, so I think its better to document it until v3.1.0 released. cc: @HyukjinKwon 


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700773353






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-733055577


   **[Test build #131673 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131673/testReport)** for PR 29893 at commit [`c74d4d7`](https://github.com/apache/spark/commit/c74d4d7a0e0768fa5d9970a9e7ddd109306d6caa).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-734731578


   **[Test build #131877 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131877/testReport)** for PR 29893 at commit [`833ac41`](https://github.com/apache/spark/commit/833ac416aac680059ce95fffc09c3234e87e0757).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-701875563


   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/33915/
   Test PASSed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-733512932


   **[Test build #131750 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131750/testReport)** for PR 29893 at commit [`475e790`](https://github.com/apache/spark/commit/475e790f7eeae80c4123f93c74952eb24f056897).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700571366


   **[Test build #129229 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129229/testReport)** for PR 29893 at commit [`2b164ed`](https://github.com/apache/spark/commit/2b164eddd80145e9104546a7e7b41d340fd9481f).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] yaooqinn commented on a change in pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #29893:
URL: https://github.com/apache/spark/pull/29893#discussion_r531421331



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -3104,6 +3106,62 @@ class Analyzer(override val catalogManager: CatalogManager)
     }
   }
 
+  object ResolveUserSpecifiedColumns extends Rule[LogicalPlan] {
+    override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperators {
+      case i: InsertIntoStatement if i.table.resolved && i.query.resolved &&
+          i.userSpecifiedCols.nonEmpty =>
+        val resolved = resolveUserSpecifiedColumns(i)
+        val projection = addColumnListOnQuery(i.table.output, resolved, i.query)
+        i.copy(userSpecifiedCols = Nil, query = projection)
+    }
+
+    private def resolveUserSpecifiedColumns(i: InsertIntoStatement): Seq[NamedExpression] = {
+      SchemaUtils.checkColumnNameDuplication(
+        i.userSpecifiedCols, "in the column list", resolver)
+
+      i.userSpecifiedCols.map { col =>
+          i.table.resolve(Seq(col), resolver)
+            .getOrElse(i.table.failAnalysis(s"Cannot resolve column name $col"))
+      }
+    }
+
+    private def addColumnListOnQuery(
+        tableOutput: Seq[Attribute],
+        cols: Seq[NamedExpression],
+        query: LogicalPlan): LogicalPlan = {
+      val errors = new mutable.ArrayBuffer[String]()
+
+      def failAdd(): Unit = {
+        val errMsg = if (errors.nonEmpty) errors.mkString("\n- ", "\n- ", "") else ""
+        query.failAnalysis(
+          s"""Cannot write to table due to mismatched user specified columns and data columns:
+             |Specified columns: ${cols.map(c => s"'${c.name}'").mkString(", ")}
+             |Data columns: ${query.output.map(c => s"'${c.name}'").mkString(", ")}$errMsg"""
+            .stripMargin)
+      }
+
+      if (cols.size != query.output.size) failAdd()
+
+      val nameToQueryExpr = cols.zip(query.output).toMap
+      val resolved = tableOutput.flatMap { tableAttr =>
+        if (nameToQueryExpr.contains(tableAttr)) {
+          TableOutputResolver.checkField(
+            tableAttr, nameToQueryExpr(tableAttr), byName = false, conf, err => errors += err)
+        } else {
+          None

Review comment:
       this is for static partition columns




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-733324380






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700644815


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/33848/
   Test FAILed.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-701338944






----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700515838


   **[Test build #129221 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129221/testReport)** for PR 29893 at commit [`e756a55`](https://github.com/apache/spark/commit/e756a558d89a077108d50103af441dffa4ba4263).


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #29893: [WIP][SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-700399996


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/33818/
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [spark] yaooqinn removed a comment on pull request #29893: [SPARK-32976][SQL]Support column list in INSERT statement

Posted by GitBox <gi...@apache.org>.
yaooqinn removed a comment on pull request #29893:
URL: https://github.com/apache/spark/pull/29893#issuecomment-733501539


   retest this please


----------------------------------------------------------------
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:
users@infra.apache.org



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