You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by cloud-fan <gi...@git.apache.org> on 2017/05/22 19:59:53 UTC

[GitHub] spark pull request #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations...

GitHub user cloud-fan opened a pull request:

    https://github.com/apache/spark/pull/18064

    [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL UI tab

    ## What changes were proposed in this pull request?
    
    Currently the `DataFrameWriter` operations have several problems:
    
    1. non-file-format data source writing action doesn't show up in the SQL tab in Spark UI
    2. file-format data source writing action shows a scan node in the SQL tab, without saying anything about writing.
    3. Spark SQL CLI actions don't show up in the SQL tab.
    
    This PR fixes all of them, by refactoring the `ExecuteCommandExec` to make it have children.
    
     close https://github.com/apache/spark/pull/17540
    
    ## How was this patch tested?
    
    existing tests.
    
    Also test the UI manually. For a simple command: `Seq(1 -> "a").toDF("i", "j").write.parquet("/tmp/qwe")`
    
    before this PR:
    <img width="266" alt="qq20170523-035840 2x" src="https://cloud.githubusercontent.com/assets/3182036/26326050/24e18ba2-3f6c-11e7-8817-6dd275bf6ac5.png">
    after this PR:
    <img width="287" alt="qq20170523-035708 2x" src="https://cloud.githubusercontent.com/assets/3182036/26326054/2ad7f460-3f6c-11e7-8053-d68325beb28f.png">


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/cloud-fan/spark execution

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/18064.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #18064
    
----
commit 5c24f5ac2b1740a0cf08346a85111a2c06920309
Author: Wenchen Fan <we...@databricks.com>
Date:   2017-05-22T19:51:13Z

    Fix DataFrameWriter operations in SQL UI tab

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    **[Test build #77337 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77337/testReport)** for PR 18064 at commit [`eec0946`](https://github.com/apache/spark/commit/eec0946842657539d69deab43641d32d247f67ec).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77198/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18064#discussion_r118391216
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala ---
    @@ -161,50 +161,50 @@ object FileFormatWriter extends Logging {
           }
         }
     
    -    SQLExecution.withNewExecutionId(sparkSession, queryExecution) {
    -      // This call shouldn't be put into the `try` block below because it only initializes and
    -      // prepares the job, any exception thrown from here shouldn't cause abortJob() to be called.
    -      committer.setupJob(job)
    -
    -      try {
    -        val rdd = if (orderingMatched) {
    -          queryExecution.toRdd
    -        } else {
    -          SortExec(
    -            requiredOrdering.map(SortOrder(_, Ascending)),
    -            global = false,
    -            child = queryExecution.executedPlan).execute()
    -        }
    -        val ret = new Array[WriteTaskResult](rdd.partitions.length)
    -        sparkSession.sparkContext.runJob(
    -          rdd,
    -          (taskContext: TaskContext, iter: Iterator[InternalRow]) => {
    -            executeTask(
    -              description = description,
    -              sparkStageId = taskContext.stageId(),
    -              sparkPartitionId = taskContext.partitionId(),
    -              sparkAttemptNumber = taskContext.attemptNumber(),
    -              committer,
    -              iterator = iter)
    -          },
    -          0 until rdd.partitions.length,
    -          (index, res: WriteTaskResult) => {
    -            committer.onTaskCommit(res.commitMsg)
    -            ret(index) = res
    -          })
    -
    -        val commitMsgs = ret.map(_.commitMsg)
    -        val updatedPartitions = ret.flatMap(_.updatedPartitions)
    -          .distinct.map(PartitioningUtils.parsePathFragment)
    -
    -        committer.commitJob(job, commitMsgs)
    -        logInfo(s"Job ${job.getJobID} committed.")
    -        refreshFunction(updatedPartitions)
    -      } catch { case cause: Throwable =>
    -        logError(s"Aborting job ${job.getJobID}.", cause)
    -        committer.abortJob(job)
    -        throw new SparkException("Job aborted.", cause)
    +    // TODO: make sure there is execution id when we reach here.
    --- End diff --
    
    Is there any case that execution id is missing? Otherwise, we can just add an assertion here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77258/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    **[Test build #77404 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77404/testReport)** for PR 18064 at commit [`f479221`](https://github.com/apache/spark/commit/f47922184e65312b47879c6d65e44bf25ea07606).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    Agreed, sorry I haven't updated it. I was out most of last week. I'll get this fixed up as soon as I can. Thanks for all your help!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    @yhuai thanks for the remind! I'll send separated PR for refactor in the future :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77539/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18064#discussion_r118622212
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala ---
    @@ -965,14 +965,20 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton {
       }
     
       test("sanity test for SPARK-6618") {
    -    (1 to 100).par.map { i =>
    --- End diff --
    
    The weird thing is I can't reproduce it locally, and I'd say it's not introduced by this PR, as previously no execution id is set for commands.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77383/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77404/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77276/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    **[Test build #77383 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77383/testReport)** for PR 18064 at commit [`59ec0e9`](https://github.com/apache/spark/commit/59ec0e9607f7d67a87fb1c0abb5330190e2afb43).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    **[Test build #77215 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77215/testReport)** for PR 18064 at commit [`5486950`](https://github.com/apache/spark/commit/5486950edada8ae87d2586f3f6d1e2d82027b015).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    **[Test build #77366 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77366/testReport)** for PR 18064 at commit [`513bb9b`](https://github.com/apache/spark/commit/513bb9b23e823271d10aa5b125eb89f1671cb88b).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    **[Test build #77396 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77396/testReport)** for PR 18064 at commit [`b7a55e7`](https://github.com/apache/spark/commit/b7a55e72dd444696c578447c54012b5b24200e61).
     * This patch **fails PySpark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18064#discussion_r118193454
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala ---
    @@ -390,7 +391,8 @@ case class DataSource(
       /**
        * Writes the given [[DataFrame]] out in this [[FileFormat]].
        */
    -  private def writeInFileFormat(format: FileFormat, mode: SaveMode, data: DataFrame): Unit = {
    +  private def planForWritingFileFormat(
    +      format: FileFormat, mode: SaveMode, data: LogicalPlan): LogicalPlan = {
    --- End diff --
    
    given [[LogicalPlan]].


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18064#discussion_r118474770
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala ---
    @@ -294,32 +294,31 @@ private[hive] class TestHiveSparkSession(
             "CREATE TABLE src1 (key INT, value STRING)".cmd,
             s"LOAD DATA LOCAL INPATH '${quoteHiveFile("data/files/kv3.txt")}' INTO TABLE src1".cmd),
           TestTable("srcpart", () => {
    -        sql(
    -          "CREATE TABLE srcpart (key INT, value STRING) PARTITIONED BY (ds STRING, hr STRING)")
    +        "CREATE TABLE srcpart (key INT, value STRING) PARTITIONED BY (ds STRING, hr STRING)"
    +          .cmd.apply()
    --- End diff --
    
    use `xxx.cmd.apply` instead of `sql("xx")` here, to avoid setting the execution id.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    That works too, if we can attach metrics to these commands.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    Refactor?
    
    I thought that was the problem with the [original PR](https://github.com/apache/spark/pull/17540); that PR was too narrow and didn't unify the physical plans to get all metrics working.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    **[Test build #77374 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77374/testReport)** for PR 18064 at commit [`def0878`](https://github.com/apache/spark/commit/def0878e70c5e1b247b761dee6fdc945ce49a2e3).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18064#discussion_r118473002
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala ---
    @@ -86,12 +86,10 @@ private[kafka010] object KafkaWriter extends Logging {
           topic: Option[String] = None): Unit = {
         val schema = queryExecution.analyzed.output
         validateQuery(queryExecution, kafkaParameters, topic)
    -    SQLExecution.withNewExecutionId(sparkSession, queryExecution) {
    --- End diff --
    
    `KafkaSink.addBath` is also wrapped with `SQLExecution.withNewExecutionId` in `StreamExecution`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    @rxin OK. I will rebase my PR after this gets merged.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    cc @zsxwing for review


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/spark/pull/18064


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    **[Test build #77337 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77337/testReport)** for PR 18064 at commit [`eec0946`](https://github.com/apache/spark/commit/eec0946842657539d69deab43641d32d247f67ec).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77374/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations...

Posted by wzhfy <gi...@git.apache.org>.
Github user wzhfy commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18064#discussion_r117919814
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala ---
    @@ -96,11 +96,12 @@ case class AnalyzeColumnCommand(
           attributesToAnalyze.map(ColumnStat.statExprs(_, ndvMaxErr))
     
         val namedExpressions = expressions.map(e => Alias(e, e.toString)())
    -    val statsRow = Dataset.ofRows(sparkSession, Aggregate(Nil, namedExpressions, relation)).head()
    +    val statsRow = Dataset.ofRows(sparkSession, Aggregate(Nil, namedExpressions, relation))
    +      .queryExecution.executedPlan.executeTake(1).head
    --- End diff --
    
    ```
    new QueryExecution(sparkSession, Aggregate(Nil, namedExpressions, relation)).executedPlan.executeTake(1).head
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18064#discussion_r118723698
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
    @@ -242,9 +242,12 @@ class Dataset[T] private[sql](
        * @param vertical If set to true, prints output rows vertically (one line per column value).
    --- End diff --
    
    Seems to add a `@param` for `isInternal` is better?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    **[Test build #77374 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77374/testReport)** for PR 18064 at commit [`def0878`](https://github.com/apache/spark/commit/def0878e70c5e1b247b761dee6fdc945ce49a2e3).
     * This patch **fails PySpark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77200/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    My suggestion was about getting changes on the interfaces of ExecutedCommandExec and SaveIntoDataSourceCommand to separate prs. It will help code review (both speed and quality).



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18064#discussion_r118607514
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala ---
    @@ -965,14 +965,20 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton {
       }
     
       test("sanity test for SPARK-6618") {
    -    (1 to 100).par.map { i =>
    --- End diff --
    
    Oh, my hunch is we probably leak execution id  to some thread in a thread pool. It's better to fix it rather than changing the test to hide it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18064#discussion_r118723009
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala ---
    @@ -426,36 +418,35 @@ case class DataSource(
         // For partitioned relation r, r.schema's column ordering can be different from the column
         // ordering of data.logicalPlan (partition columns are all moved after data column).  This
         // will be adjusted within InsertIntoHadoopFsRelation.
    -    val plan =
    -      InsertIntoHadoopFsRelationCommand(
    -        outputPath = outputPath,
    -        staticPartitions = Map.empty,
    -        ifPartitionNotExists = false,
    -        partitionColumns = partitionAttributes,
    -        bucketSpec = bucketSpec,
    -        fileFormat = format,
    -        options = options,
    -        query = data.logicalPlan,
    -        mode = mode,
    -        catalogTable = catalogTable,
    -        fileIndex = fileIndex)
    -      sparkSession.sessionState.executePlan(plan).toRdd
    +    InsertIntoHadoopFsRelationCommand(
    +      outputPath = outputPath,
    +      staticPartitions = Map.empty,
    +      ifPartitionNotExists = false,
    +      partitionColumns = partitionColumns.map(UnresolvedAttribute.quoted),
    --- End diff --
    
    Ok. I see.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77540/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    **[Test build #77383 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77383/testReport)** for PR 18064 at commit [`59ec0e9`](https://github.com/apache/spark/commit/59ec0e9607f7d67a87fb1c0abb5330190e2afb43).
     * This patch **fails R style tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77328/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18064#discussion_r118193935
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala ---
    @@ -464,18 +455,18 @@ case class DataSource(
       }
     
       /**
    -   * Writes the given [[DataFrame]] out to this [[DataSource]].
    +   * Returns a logical plan to write the given [[DataFrame]] out to this [[DataSource]].
    --- End diff --
    
    the give [[LogicalPlan]].


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77220/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    **[Test build #77219 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77219/testReport)** for PR 18064 at commit [`9507f19`](https://github.com/apache/spark/commit/9507f1938f894b2884b024c8472084a3a531e20d).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `trait Command extends LogicalPlan `
      * `case class ExecutedCommandExec(cmd: RunnableCommand, children: Seq[SparkPlan]) extends SparkPlan `


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77366/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77396/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    This seems also related to my PR #17998.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77562/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    I just case across this pr. I have one general feedback. It will be great if we can make a pr have a single purpose. This pr contains different kinds of changes in order to fix the UI. If refactoring is needed, I'd recommend to have separate PR for refactoring purposes. Then, use a different PR to do the actual fix.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    **[Test build #77198 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77198/testReport)** for PR 18064 at commit [`5c24f5a`](https://github.com/apache/spark/commit/5c24f5ac2b1740a0cf08346a85111a2c06920309).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    **[Test build #77368 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77368/testReport)** for PR 18064 at commit [`bc7608a`](https://github.com/apache/spark/commit/bc7608ae8a1e2ccd4b013069d41dbcf8229d4542).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    **[Test build #77539 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77539/testReport)** for PR 18064 at commit [`f49a0b3`](https://github.com/apache/spark/commit/f49a0b3fb4bb014a47eed72d538979acf832f7d4).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    **[Test build #77540 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77540/testReport)** for PR 18064 at commit [`6c7d259`](https://github.com/apache/spark/commit/6c7d259c01117bf1dc1fc4d3b2519cac5d44063d).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `  class DecisionTreeClassifierWrapperWriter(instance: DecisionTreeClassifierWrapper)`
      * `  class DecisionTreeClassifierWrapperReader extends MLReader[DecisionTreeClassifierWrapper] `
      * `  class DecisionTreeRegressorWrapperWriter(instance: DecisionTreeRegressorWrapper)`
      * `  class DecisionTreeRegressorWrapperReader extends MLReader[DecisionTreeRegressorWrapper] `
      * `class HasMinSupport(Params):`
      * `class HasNumPartitions(Params):`
      * `class HasMinConfidence(Params):`
      * `case class UnresolvedRelation(`
      * `case class DayOfWeek(child: Expression) extends UnaryExpression with ImplicitCastInputTypes `
      * `case class StringReplace(srcExpr: Expression, searchExpr: Expression, replaceExpr: Expression)`
      * `case class Chr(child: Expression) extends UnaryExpression with ImplicitCastInputTypes `
      * `case class AnalysisBarrier(child: LogicalPlan) extends LeafNode `
      * `case class UnresolvedHint(name: String, parameters: Seq[String], child: LogicalPlan)`
      * `case class ResolvedHint(child: LogicalPlan, hints: HintInfo = HintInfo())`
      * `case class HintInfo(`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77219/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by rdblue <gi...@git.apache.org>.
Github user rdblue commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    @cloud-fan, can you summarize how this differs from the original PR #17540?
    
    I have time to pick this up again, but I thought that the other PR only needed two changes:
    
    * Merge your changes to unify the physical plans when using `ExecuteCommandExec`
    * Move `withNewExecutionId` to stremaing batches, rather than the entire session
    
    This seems to have gone though a lot of updates for testing, though, and I don't see my original commits... is this a reimplementation of those changes?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    @cloud-fan The purpose of the operator is just used to record metrics of writing data. I said it's related because this PR is also going to show the writing op in UI. But looks like this change doesn't do anything about the metrics.
    
    There are many classes are both covered by this PR and #17998, e.g. the commands and how to trigger them. I'd rebase my PR once this is merged. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77215/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    **[Test build #77198 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77198/testReport)** for PR 18064 at commit [`5c24f5a`](https://github.com/apache/spark/commit/5c24f5ac2b1740a0cf08346a85111a2c06920309).
     * This patch **fails to build**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `trait Command extends LogicalPlan `
      * `case class ExecutedCommandExec(cmd: RunnableCommand, children: Seq[SparkPlan]) extends SparkPlan `


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18064#discussion_r118468240
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala ---
    @@ -426,36 +418,35 @@ case class DataSource(
         // For partitioned relation r, r.schema's column ordering can be different from the column
         // ordering of data.logicalPlan (partition columns are all moved after data column).  This
         // will be adjusted within InsertIntoHadoopFsRelation.
    -    val plan =
    -      InsertIntoHadoopFsRelationCommand(
    -        outputPath = outputPath,
    -        staticPartitions = Map.empty,
    -        ifPartitionNotExists = false,
    -        partitionColumns = partitionAttributes,
    -        bucketSpec = bucketSpec,
    -        fileFormat = format,
    -        options = options,
    -        query = data.logicalPlan,
    -        mode = mode,
    -        catalogTable = catalogTable,
    -        fileIndex = fileIndex)
    -      sparkSession.sessionState.executePlan(plan).toRdd
    +    InsertIntoHadoopFsRelationCommand(
    +      outputPath = outputPath,
    +      staticPartitions = Map.empty,
    +      ifPartitionNotExists = false,
    +      partitionColumns = partitionColumns.map(UnresolvedAttribute.quoted),
    --- End diff --
    
    Analyzer will resolve them in `InsertIntoHadoopFsRelationCommand`. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    **[Test build #77219 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77219/testReport)** for PR 18064 at commit [`9507f19`](https://github.com/apache/spark/commit/9507f1938f894b2884b024c8472084a3a531e20d).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    **[Test build #77220 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77220/testReport)** for PR 18064 at commit [`b355c6d`](https://github.com/apache/spark/commit/b355c6d034c6aefcf8f74757353afce870e9bf1d).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `trait Command extends LogicalPlan `
      * `case class ExecutedCommandExec(cmd: RunnableCommand, children: Seq[SparkPlan]) extends SparkPlan `


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    @rdblue the difference:
    1. unify the physical plans when using `ExecuteCommandExec`
    2. Move `withNewExecutionId` to stremaing batches, rather than the entire session
    3. instead of `SQLExecution.nested`, introduce some internal actions that won't set the execution id in nested exeution


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    thanks for the review, merging to master!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77337/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    **[Test build #77328 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77328/testReport)** for PR 18064 at commit [`57f9dde`](https://github.com/apache/spark/commit/57f9dde7d4469bbd7f1e04a04fac2041a2d743e6).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77368/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18064#discussion_r118191647
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala ---
    @@ -86,12 +86,10 @@ private[kafka010] object KafkaWriter extends Logging {
           topic: Option[String] = None): Unit = {
         val schema = queryExecution.analyzed.output
         validateQuery(queryExecution, kafkaParameters, topic)
    -    SQLExecution.withNewExecutionId(sparkSession, queryExecution) {
    --- End diff --
    
    After removing this, can `KafkaSink` still track its streaming writing?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77354/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    **[Test build #77220 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77220/testReport)** for PR 18064 at commit [`b355c6d`](https://github.com/apache/spark/commit/b355c6d034c6aefcf8f74757353afce870e9bf1d).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18064#discussion_r118410964
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala ---
    @@ -86,12 +86,10 @@ private[kafka010] object KafkaWriter extends Logging {
           topic: Option[String] = None): Unit = {
         val schema = queryExecution.analyzed.output
         validateQuery(queryExecution, kafkaParameters, topic)
    -    SQLExecution.withNewExecutionId(sparkSession, queryExecution) {
    --- End diff --
    
    If you mean `KafkaSourceProvider`, is it the same code path as `KafkaSink`? In `KafkaSink.addBath`, `KafkaWriter.write` is also called to write into Kafka. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    **[Test build #77540 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77540/testReport)** for PR 18064 at commit [`6c7d259`](https://github.com/apache/spark/commit/6c7d259c01117bf1dc1fc4d3b2519cac5d44063d).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    **[Test build #77550 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77550/testReport)** for PR 18064 at commit [`9d73408`](https://github.com/apache/spark/commit/9d7340840754f3937f02c24380d99c6c6e7e98be).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    **[Test build #77215 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77215/testReport)** for PR 18064 at commit [`5486950`](https://github.com/apache/spark/commit/5486950edada8ae87d2586f3f6d1e2d82027b015).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `trait Command extends LogicalPlan `
      * `case class ExecutedCommandExec(cmd: RunnableCommand, children: Seq[SparkPlan]) extends SparkPlan `


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    **[Test build #77200 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77200/testReport)** for PR 18064 at commit [`ddba4c5`](https://github.com/apache/spark/commit/ddba4c5134e91c28c75a317b82a941dcdfdea622).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77240/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18064#discussion_r118198979
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala ---
    @@ -86,12 +86,10 @@ private[kafka010] object KafkaWriter extends Logging {
           topic: Option[String] = None): Unit = {
         val schema = queryExecution.analyzed.output
         validateQuery(queryExecution, kafkaParameters, topic)
    -    SQLExecution.withNewExecutionId(sparkSession, queryExecution) {
    --- End diff --
    
    yea, because it's used in `KafkaProvider` which is run with `SaveIntoDataSourceCommand`, and all commands will be wrapped by `SQLExecution.withNewExecutionId`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    **[Test build #77396 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77396/testReport)** for PR 18064 at commit [`b7a55e7`](https://github.com/apache/spark/commit/b7a55e72dd444696c578447c54012b5b24200e61).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18064#discussion_r118391357
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala ---
    @@ -965,14 +965,20 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton {
       }
     
       test("sanity test for SPARK-6618") {
    -    (1 to 100).par.map { i =>
    --- End diff --
    
    Why changed this test? Using 100 threads will be much slower.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    cc @rdblue , I think this issue is important and we should probably fix it in 2.2, so I take over without the streaming changes to make this PR safer to merge.
    
    Also cc @zsxwing 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    @cloud-fan Actually, if we decide to show individual writing commands in UI like this, we may not need "write operator". To have a writing operator on top on those commands might looks confusing. Then we can directly attach the metrics to the commands.
    
    @rxin What do you think?



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    **[Test build #77562 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77562/testReport)** for PR 18064 at commit [`baef6ec`](https://github.com/apache/spark/commit/baef6ec68ed91e493fa67255fd1060f40721e369).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/77550/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    **[Test build #77354 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77354/testReport)** for PR 18064 at commit [`60a064d`](https://github.com/apache/spark/commit/60a064d6d69a790e53fda53d85fd4114a6b39339).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    **[Test build #77258 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77258/testReport)** for PR 18064 at commit [`c76c30c`](https://github.com/apache/spark/commit/c76c30cf28c64ecf665dff7e82c27e48e2323470).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    **[Test build #77200 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77200/testReport)** for PR 18064 at commit [`ddba4c5`](https://github.com/apache/spark/commit/ddba4c5134e91c28c75a317b82a941dcdfdea622).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `trait Command extends LogicalPlan `
      * `case class ExecutedCommandExec(cmd: RunnableCommand, children: Seq[SparkPlan]) extends SparkPlan `


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18064#discussion_r118443224
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala ---
    @@ -426,36 +418,35 @@ case class DataSource(
         // For partitioned relation r, r.schema's column ordering can be different from the column
         // ordering of data.logicalPlan (partition columns are all moved after data column).  This
         // will be adjusted within InsertIntoHadoopFsRelation.
    -    val plan =
    -      InsertIntoHadoopFsRelationCommand(
    -        outputPath = outputPath,
    -        staticPartitions = Map.empty,
    -        ifPartitionNotExists = false,
    -        partitionColumns = partitionAttributes,
    -        bucketSpec = bucketSpec,
    -        fileFormat = format,
    -        options = options,
    -        query = data.logicalPlan,
    -        mode = mode,
    -        catalogTable = catalogTable,
    -        fileIndex = fileIndex)
    -      sparkSession.sessionState.executePlan(plan).toRdd
    +    InsertIntoHadoopFsRelationCommand(
    +      outputPath = outputPath,
    +      staticPartitions = Map.empty,
    +      ifPartitionNotExists = false,
    +      partitionColumns = partitionColumns.map(UnresolvedAttribute.quoted),
    --- End diff --
    
    `InsertIntoHadoopFsRelationCommand` will pass `partitionColumns` to `FileFormatWriter`. Seems `FileFormatWriter` won't resolve those `partitionColumns`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    **[Test build #77276 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77276/testReport)** for PR 18064 at commit [`984bab7`](https://github.com/apache/spark/commit/984bab7bb2297e39088f909c41344a8fc8e06936).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    **[Test build #77240 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/77240/testReport)** for PR 18064 at commit [`6504cc3`](https://github.com/apache/spark/commit/6504cc342731d94cafe3b20cbb2017d013603a77).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18064#discussion_r118440922
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala ---
    @@ -426,36 +418,35 @@ case class DataSource(
         // For partitioned relation r, r.schema's column ordering can be different from the column
         // ordering of data.logicalPlan (partition columns are all moved after data column).  This
         // will be adjusted within InsertIntoHadoopFsRelation.
    -    val plan =
    -      InsertIntoHadoopFsRelationCommand(
    -        outputPath = outputPath,
    -        staticPartitions = Map.empty,
    -        ifPartitionNotExists = false,
    -        partitionColumns = partitionAttributes,
    -        bucketSpec = bucketSpec,
    -        fileFormat = format,
    -        options = options,
    -        query = data.logicalPlan,
    -        mode = mode,
    -        catalogTable = catalogTable,
    -        fileIndex = fileIndex)
    -      sparkSession.sessionState.executePlan(plan).toRdd
    +    InsertIntoHadoopFsRelationCommand(
    +      outputPath = outputPath,
    +      staticPartitions = Map.empty,
    +      ifPartitionNotExists = false,
    +      partitionColumns = partitionColumns.map(UnresolvedAttribute.quoted),
    --- End diff --
    
    Is it safe to use unresolved attributes here?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18064#discussion_r118193391
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala ---
    @@ -426,36 +418,35 @@ case class DataSource(
         // For partitioned relation r, r.schema's column ordering can be different from the column
         // ordering of data.logicalPlan (partition columns are all moved after data column).  This
         // will be adjusted within InsertIntoHadoopFsRelation.
    -    val plan =
    -      InsertIntoHadoopFsRelationCommand(
    -        outputPath = outputPath,
    -        staticPartitions = Map.empty,
    -        ifPartitionNotExists = false,
    -        partitionColumns = partitionAttributes,
    -        bucketSpec = bucketSpec,
    -        fileFormat = format,
    -        options = options,
    -        query = data.logicalPlan,
    -        mode = mode,
    -        catalogTable = catalogTable,
    -        fileIndex = fileIndex)
    -      sparkSession.sessionState.executePlan(plan).toRdd
    +    InsertIntoHadoopFsRelationCommand(
    +      outputPath = outputPath,
    +      staticPartitions = Map.empty,
    +      ifPartitionNotExists = false,
    +      partitionColumns = partitionColumns.map(UnresolvedAttribute.quoted),
    +      bucketSpec = bucketSpec,
    +      fileFormat = format,
    +      options = options,
    +      query = data,
    +      mode = mode,
    +      catalogTable = catalogTable,
    +      fileIndex = fileIndex)
       }
     
       /**
        * Writes the given [[DataFrame]] out to this [[DataSource]] and returns a [[BaseRelation]] for
    --- End diff --
    
    Comment needs to change: given [[LogicalPlan]].


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18064#discussion_r118403738
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala ---
    @@ -965,14 +965,20 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton {
       }
     
       test("sanity test for SPARK-6618") {
    -    (1 to 100).par.map { i =>
    --- End diff --
    
    this test was failing and complained that the execution id is already set. I don't know the detail about how we schedule the 100 tasks on threads with `par`, so I changed it to use `Thread` directly to be more clear. I'll reduce the number of threads.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18064: [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/18064
  
    @viirya I'm not very clear about the purpose of the "write operator", does my PR cover it?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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