You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by wangyum <gi...@git.apache.org> on 2018/11/02 18:41:30 UTC

[GitHub] spark pull request #22930: [SPARK-24869][SQL] Fix SaveIntoDataSourceCommand'...

GitHub user wangyum opened a pull request:

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

    [SPARK-24869][SQL] Fix SaveIntoDataSourceCommand's input Dataset does not use cached data

    ## What changes were proposed in this pull request?
    
    This PR fix 2 things:
    1. Fix `SaveIntoDataSourceCommand`'s input Dataset does not use Cached Data.
    2. Fix `SaveIntoDataSourceCommand`s web ui when writing data.
    
    ## How was this patch tested?
    unit tests and  manual tests
    
    Manual test web UI:
    ```scala
    val udf1 = udf({ (x: Int, y: Int) => x + y })
    val df = spark.range(0, 3).toDF("a").withColumn("b", udf1(col("a"), lit(10)))
    df.write.mode(SaveMode.Overwrite).jdbc(jdbcUrl, jdbcTable, new Properties)
    ```
    **Before this PR**:
    ![image](https://user-images.githubusercontent.com/5399861/45927682-4b43b580-bf69-11e8-9161-64399542cf77.png)
    
    
    **After this PR:**
    
    ![image](https://user-images.githubusercontent.com/5399861/45927685-539bf080-bf69-11e8-8700-f32430d9aeae.png)
    
    


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

    $ git pull https://github.com/wangyum/spark SPARK-24869

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

    https://github.com/apache/spark/pull/22930.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 #22930
    
----
commit 9125b31fd2b1b6df0a5aaeba743f9d287cf2e897
Author: Yuming Wang <yu...@...>
Date:   2018-11-02T18:39:44Z

    Fix SPARK-24869

----


---

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


[GitHub] spark issue #22930: [SPARK-24869][SQL] Fix SaveIntoDataSourceCommand's input...

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

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


---

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


[GitHub] spark issue #22930: [SPARK-24869][SQL] Fix SaveIntoDataSourceCommand's input...

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

    https://github.com/apache/spark/pull/22930
  
    **[Test build #98419 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/98419/testReport)** for PR 22930 at commit [`a799e3f`](https://github.com/apache/spark/commit/a799e3f0f39459b2eb14978fa622af4a3e0b3294).
     * This patch **fails due to an unknown error code, -9**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22930: [SPARK-24869][SQL] Fix SaveIntoDataSourceCommand's input...

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

    https://github.com/apache/spark/pull/22930
  
    cc @gatorsmile @gengliangwang @maropu


---

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


[GitHub] spark issue #22930: [SPARK-24869][SQL] Fix SaveIntoDataSourceCommand's input...

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

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


---

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


[GitHub] spark issue #22930: [SPARK-24869][SQL] Fix SaveIntoDataSourceCommand's input...

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

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


---

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


[GitHub] spark issue #22930: [SPARK-24869][SQL] Fix SaveIntoDataSourceCommand's input...

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

    https://github.com/apache/spark/pull/22930
  
    retest this please


---

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


[GitHub] spark pull request #22930: [SPARK-24869][SQL] Fix SaveIntoDataSourceCommand'...

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

    https://github.com/apache/spark/pull/22930#discussion_r230609078
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommand.scala ---
    @@ -37,13 +37,12 @@ case class SaveIntoDataSourceCommand(
         query: LogicalPlan,
         dataSource: CreatableRelationProvider,
         options: Map[String, String],
    -    mode: SaveMode) extends RunnableCommand {
    +    mode: SaveMode,
    +    outputColumnNames: Seq[String]) extends DataWritingCommand {
     
    -  override protected def innerChildren: Seq[QueryPlan[_]] = Seq(query)
    -
    -  override def run(sparkSession: SparkSession): Seq[Row] = {
    -    dataSource.createRelation(
    -      sparkSession.sqlContext, mode, options, Dataset.ofRows(sparkSession, query))
    --- End diff --
    
    See what I commented in https://github.com/apache/spark/pull/22941


---

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


[GitHub] spark pull request #22930: [SPARK-24869][SQL] Fix SaveIntoDataSourceCommand'...

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

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


---

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


[GitHub] spark issue #22930: [SPARK-24869][SQL] Fix SaveIntoDataSourceCommand's input...

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

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


---

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


[GitHub] spark issue #22930: [SPARK-24869][SQL] Fix SaveIntoDataSourceCommand's input...

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

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


---

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


[GitHub] spark issue #22930: [SPARK-24869][SQL] Fix SaveIntoDataSourceCommand's input...

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

    https://github.com/apache/spark/pull/22930
  
    retest this please


---

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


[GitHub] spark issue #22930: [SPARK-24869][SQL] Fix SaveIntoDataSourceCommand's input...

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

    https://github.com/apache/spark/pull/22930
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/4736/
    Test PASSed.


---

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


[GitHub] spark issue #22930: [SPARK-24869][SQL] Fix SaveIntoDataSourceCommand's input...

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

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


---

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


[GitHub] spark issue #22930: [SPARK-24869][SQL] Fix SaveIntoDataSourceCommand's input...

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

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


---

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


[GitHub] spark issue #22930: [SPARK-24869][SQL] Fix SaveIntoDataSourceCommand's input...

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

    https://github.com/apache/spark/pull/22930
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/4732/
    Test PASSed.


---

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


[GitHub] spark issue #22930: [SPARK-24869][SQL] Fix SaveIntoDataSourceCommand's input...

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

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


---

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


[GitHub] spark issue #22930: [SPARK-24869][SQL] Fix SaveIntoDataSourceCommand's input...

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

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


---

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


[GitHub] spark pull request #22930: [SPARK-24869][SQL] Fix SaveIntoDataSourceCommand'...

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

    https://github.com/apache/spark/pull/22930#discussion_r230471897
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala ---
    @@ -56,10 +62,6 @@ class DefaultSource
       with SchemaRelationProvider
       with CreatableRelationProvider {
     
    -  case class FakeRelation(sqlContext: SQLContext) extends BaseRelation {
    --- End diff --
    
    Move `FakeRelation` to `DataSourceTest`. then `FakeDataSourceStrategy` can use it.


---

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


[GitHub] spark issue #22930: [SPARK-24869][SQL] Fix SaveIntoDataSourceCommand's input...

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

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


---

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


[GitHub] spark issue #22930: [SPARK-24869][SQL] Fix SaveIntoDataSourceCommand's input...

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

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


---

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


[GitHub] spark issue #22930: [SPARK-24869][SQL] Fix SaveIntoDataSourceCommand's input...

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

    https://github.com/apache/spark/pull/22930
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/4730/
    Test PASSed.


---

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


[GitHub] spark issue #22930: [SPARK-24869][SQL] Fix SaveIntoDataSourceCommand's input...

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

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


---

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


[GitHub] spark issue #22930: [SPARK-24869][SQL] Fix SaveIntoDataSourceCommand's input...

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

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


---

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


[GitHub] spark issue #22930: [SPARK-24869][SQL] Fix SaveIntoDataSourceCommand's input...

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

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


---

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


[GitHub] spark issue #22930: [SPARK-24869][SQL] Fix SaveIntoDataSourceCommand's input...

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

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


---

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


[GitHub] spark pull request #22930: [SPARK-24869][SQL] Fix SaveIntoDataSourceCommand'...

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

    https://github.com/apache/spark/pull/22930#discussion_r230471768
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/test/TestSQLContext.scala ---
    @@ -39,7 +39,10 @@ private[spark] class TestSparkSession(sc: SparkContext) extends SparkSession(sc)
     
       @transient
       override lazy val sessionState: SessionState = {
    -    new TestSQLSessionStateBuilder(this, None).build()
    +    val ss = new TestSQLSessionStateBuilder(this, None).build()
    --- End diff --
    
    `FakeDataSourceStrategy` is needed. otherwise::
    ```java
    java.lang.AssertionError: assertion failed: No plan for Relation[a#23] FakeRelation(org.apache.spark.sql.SQLContext@6838be66)
    ```


---

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


[GitHub] spark issue #22930: [SPARK-24869][SQL] Fix SaveIntoDataSourceCommand's input...

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

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


---

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


[GitHub] spark issue #22930: [SPARK-24869][SQL] Fix SaveIntoDataSourceCommand's input...

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

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


---

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


[GitHub] spark issue #22930: [SPARK-24869][SQL] Fix SaveIntoDataSourceCommand's input...

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

    https://github.com/apache/spark/pull/22930
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/4724/
    Test PASSed.


---

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


[GitHub] spark issue #22930: [SPARK-24869][SQL] Fix SaveIntoDataSourceCommand's input...

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

    https://github.com/apache/spark/pull/22930
  
    **[Test build #98408 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/98408/testReport)** for PR 22930 at commit [`9125b31`](https://github.com/apache/spark/commit/9125b31fd2b1b6df0a5aaeba743f9d287cf2e897).


---

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


[GitHub] spark issue #22930: [SPARK-24869][SQL] Fix SaveIntoDataSourceCommand's input...

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

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


---

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


[GitHub] spark issue #22930: [SPARK-24869][SQL] Fix SaveIntoDataSourceCommand's input...

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

    https://github.com/apache/spark/pull/22930
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/4737/
    Test PASSed.


---

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


[GitHub] spark issue #22930: [SPARK-24869][SQL] Fix SaveIntoDataSourceCommand's input...

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

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


---

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


[GitHub] spark issue #22930: [SPARK-24869][SQL] Fix SaveIntoDataSourceCommand's input...

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

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


---

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


[GitHub] spark issue #22930: [SPARK-24869][SQL] Fix SaveIntoDataSourceCommand's input...

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

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


---

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


[GitHub] spark issue #22930: [SPARK-24869][SQL] Fix SaveIntoDataSourceCommand's input...

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

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


---

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


[GitHub] spark issue #22930: [SPARK-24869][SQL] Fix SaveIntoDataSourceCommand's input...

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

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


---

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