You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by windpiger <gi...@git.apache.org> on 2017/01/19 08:19:32 UTC

[GitHub] spark pull request #16642: [SPARK-19284][SQL]append to datasource partitione...

GitHub user windpiger opened a pull request:

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

    [SPARK-19284][SQL]append to datasource partitioned table without custom partition location

    ## What changes were proposed in this pull request?
    
    when we append data to a existed partitioned datasource table, the InsertIntoHadoopFsRelationCommand.getCustomPartitionLocations currently
    return the same location with Hive default, it should return None.
    
    ## How was this patch tested?

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

    $ git pull https://github.com/windpiger/spark appendSchema

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

    https://github.com/apache/spark/pull/16642.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 #16642
    
----
commit 7d016d3a277548c7f914aa09be1190bf9a305416
Author: windpiger <so...@outlook.com>
Date:   2017-01-19T08:16:30Z

    [SPARK-19284][SQL]append to datasource partitioned table without custompartionlocation

----


---
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 #16642: [SPARK-19284][SQL]append to partitioned datasourc...

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/16642#discussion_r97262909
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/sources/PartitionedWriteSuite.scala ---
    @@ -92,6 +111,16 @@ class PartitionedWriteSuite extends QueryTest with SharedSQLContext {
         }
       }
     
    +  test("append data to an existed partitioned table without custom partition path") {
    +    withTable("t") {
    +      withSQLConf("spark.sql.sources.commitProtocolClass" ->
    --- End diff --
    
    nit: SQLConf.FILE_COMMIT_PROTOCOL_CLASS.key -> classOf[OnlyDetectCustomPathFileCommitProtocol].getName


---
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 #16642: [SPARK-19284][SQL]append to partitioned datasource table...

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

    https://github.com/apache/spark/pull/16642
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/71829/
    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 #16642: [SPARK-19284][SQL]append to partitioned datasourc...

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/16642#discussion_r97222948
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/sources/PartitionedWriteSuite.scala ---
    @@ -92,6 +96,47 @@ class PartitionedWriteSuite extends QueryTest with SharedSQLContext {
         }
       }
     
    +  test("append data an existed partition in a datasource table," +
    +    "custom location sent to Task should be None ") {
    +    withTable("t") {
    +      Seq((1, 2)).toDF("a", "b").write.partitionBy("b").saveAsTable("t")
    +      val writer = Seq((3, 2)).toDF("a", "b").write.mode("append").partitionBy("b")
    +
    +      spark.sessionState.executePlan(writer.createTableCommand(TableIdentifier("t")))
    --- End diff --
    
    how about we make a custom `FileCommitProtocol` to test this behavior?


---
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 #16642: [SPARK-19284][SQL]append to partitioned datasource table...

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

    https://github.com/apache/spark/pull/16642
  
    **[Test build #71645 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/71645/testReport)** for PR 16642 at commit [`7d016d3`](https://github.com/apache/spark/commit/7d016d3a277548c7f914aa09be1190bf9a305416).
     * This patch **fails from timeout after a configured wait of \`250m\`**.
     * 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 #16642: [SPARK-19284][SQL]append to partitioned datasource table...

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

    https://github.com/apache/spark/pull/16642
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/71703/
    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 #16642: [SPARK-19284][SQL]append to partitioned datasource table...

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

    https://github.com/apache/spark/pull/16642
  
    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 #16642: [SPARK-19284][SQL]append to partitioned datasource table...

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

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


---
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 #16642: [SPARK-19284][SQL]append to partitioned datasource table...

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

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


---
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 #16642: [SPARK-19284][SQL]append to partitioned datasource table...

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

    https://github.com/apache/spark/pull/16642
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/71827/
    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 #16642: [SPARK-19284][SQL]append to partitioned datasource table...

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

    https://github.com/apache/spark/pull/16642
  
    LGTM, pending test


---
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 #16642: [SPARK-19284][SQL]append to partitioned datasource table...

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

    https://github.com/apache/spark/pull/16642
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/71645/
    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 #16642: [SPARK-19284][SQL]append to partitioned datasource table...

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

    https://github.com/apache/spark/pull/16642
  
    **[Test build #71793 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/71793/testReport)** for PR 16642 at commit [`f76f75b`](https://github.com/apache/spark/commit/f76f75b8e8ec804307c2b80ab4a7ceb02dcae716).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `case class SparkListenerExecutorBlacklisted(`
      * `case class SparkListenerExecutorUnblacklisted(time: Long, executorId: String)`
      * `case class SparkListenerNodeBlacklisted(`
      * `case class SparkListenerNodeUnblacklisted(time: Long, hostId: String)`
      * `case class QualifiedTableName(database: String, name: String)`
      * `  class MaintenanceTask(periodMs: Long, task: => Unit, onError: => Unit) `
      * `class FindHiveSerdeTable(session: SparkSession) extends Rule[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 #16642: [SPARK-19284][SQL]append to partitioned datasource table...

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

    https://github.com/apache/spark/pull/16642
  
    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 #16642: [SPARK-19284][SQL]append to partitioned datasource table...

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

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


---
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 #16642: [SPARK-19284][SQL]append to partitioned datasource table...

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

    https://github.com/apache/spark/pull/16642
  
    thanks, 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 #16642: [SPARK-19284][SQL]append to partitioned datasource table...

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

    https://github.com/apache/spark/pull/16642
  
    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 #16642: [SPARK-19284][SQL]append to partitioned datasource table...

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

    https://github.com/apache/spark/pull/16642
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/71704/
    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 #16642: [SPARK-19284][SQL]append to partitioned datasource table...

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

    https://github.com/apache/spark/pull/16642
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/71830/
    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 #16642: [SPARK-19284][SQL]append to partitioned datasourc...

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

    https://github.com/apache/spark/pull/16642#discussion_r97262179
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/sources/PartitionedWriteSuite.scala ---
    @@ -92,6 +96,47 @@ class PartitionedWriteSuite extends QueryTest with SharedSQLContext {
         }
       }
     
    +  test("append data an existed partition in a datasource table," +
    +    "custom location sent to Task should be None ") {
    +    withTable("t") {
    +      Seq((1, 2)).toDF("a", "b").write.partitionBy("b").saveAsTable("t")
    +      val writer = Seq((3, 2)).toDF("a", "b").write.mode("append").partitionBy("b")
    +
    +      spark.sessionState.executePlan(writer.createTableCommand(TableIdentifier("t")))
    --- End diff --
    
    good idea, thanks!


---
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 #16642: [SPARK-19284][SQL]append to partitioned datasource table...

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

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


---
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 #16642: [SPARK-19284][SQL]append to partitioned datasource table...

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

    https://github.com/apache/spark/pull/16642
  
    **[Test build #71713 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/71713/testReport)** for PR 16642 at commit [`3a5ebd7`](https://github.com/apache/spark/commit/3a5ebd7ee5ead531bc9a778703faebc4807b8611).


---
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 #16642: [SPARK-19284][SQL]append to partitioned datasourc...

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

    https://github.com/apache/spark/pull/16642#discussion_r97262157
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/sources/PartitionedWriteSuite.scala ---
    @@ -92,6 +96,47 @@ class PartitionedWriteSuite extends QueryTest with SharedSQLContext {
         }
       }
     
    +  test("append data an existed partition in a datasource table," +
    --- End diff --
    
    thanks~


---
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 #16642: [SPARK-19284][SQL]append to partitioned datasource table...

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

    https://github.com/apache/spark/pull/16642
  
    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 #16642: [SPARK-19284][SQL]append to partitioned datasource table...

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

    https://github.com/apache/spark/pull/16642
  
    **[Test build #71703 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/71703/testReport)** for PR 16642 at commit [`095d421`](https://github.com/apache/spark/commit/095d421a05f985785964c2fae0e7c4f84fc1752a).
     * 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 #16642: [SPARK-19284][SQL]append to partitioned datasource table...

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

    https://github.com/apache/spark/pull/16642
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/71713/
    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 #16642: [SPARK-19284][SQL]append to partitioned datasourc...

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/16642#discussion_r97032136
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala ---
    @@ -122,7 +122,6 @@ case class CreateDataSourceTableAsSelectCommand(
       override def run(sparkSession: SparkSession): Seq[Row] = {
         assert(table.tableType != CatalogTableType.VIEW)
         assert(table.provider.isDefined)
    -    assert(table.schema.isEmpty)
    --- End diff --
    
    we should move this assert to the branch for when table not exist.


---
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 #16642: [SPARK-19284][SQL]append to partitioned datasource table...

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

    https://github.com/apache/spark/pull/16642
  
    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 #16642: [SPARK-19284][SQL]append to partitioned datasourc...

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/16642#discussion_r97222768
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/sources/PartitionedWriteSuite.scala ---
    @@ -92,6 +96,47 @@ class PartitionedWriteSuite extends QueryTest with SharedSQLContext {
         }
       }
     
    +  test("append data an existed partition in a datasource table," +
    --- End diff --
    
    `append data to an existing partitioned table without custom partition path`


---
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 #16642: [SPARK-19284][SQL]append to partitioned datasource table...

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

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


---
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 #16642: [SPARK-19284][SQL]append to partitioned datasource table...

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

    https://github.com/apache/spark/pull/16642
  
    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 #16642: [SPARK-19284][SQL]append to partitioned datasource table...

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

    https://github.com/apache/spark/pull/16642
  
    **[Test build #71713 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/71713/testReport)** for PR 16642 at commit [`3a5ebd7`](https://github.com/apache/spark/commit/3a5ebd7ee5ead531bc9a778703faebc4807b8611).
     * 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 pull request #16642: [SPARK-19284][SQL]append to partitioned datasourc...

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/16642#discussion_r97222549
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala ---
    @@ -412,6 +412,10 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) {
       }
     
       private def createTable(tableIdent: TableIdentifier): Unit = {
    +    df.sparkSession.sessionState.executePlan(createTableCommand(tableIdent)).toRdd
    +  }
    +
    +  private[sql] def createTableCommand(tableIdent: TableIdentifier): CreateTable = {
    --- End diff --
    
    why this change?


---
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 #16642: [SPARK-19284][SQL]append to partitioned datasource table...

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

    https://github.com/apache/spark/pull/16642
  
    **[Test build #71704 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/71704/testReport)** for PR 16642 at commit [`3a5ebd7`](https://github.com/apache/spark/commit/3a5ebd7ee5ead531bc9a778703faebc4807b8611).


---
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 #16642: [SPARK-19284][SQL]append to partitioned datasource table...

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

    https://github.com/apache/spark/pull/16642
  
    **[Test build #71662 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/71662/testReport)** for PR 16642 at commit [`7d016d3`](https://github.com/apache/spark/commit/7d016d3a277548c7f914aa09be1190bf9a305416).
     * 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 #16642: [SPARK-19284][SQL]append to partitioned datasource table...

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

    https://github.com/apache/spark/pull/16642
  
    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 #16642: [SPARK-19284][SQL]append to partitioned datasource table...

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

    https://github.com/apache/spark/pull/16642
  
    **[Test build #71645 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/71645/testReport)** for PR 16642 at commit [`7d016d3`](https://github.com/apache/spark/commit/7d016d3a277548c7f914aa09be1190bf9a305416).


---
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 #16642: [SPARK-19284][SQL]append to partitioned datasource table...

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

    https://github.com/apache/spark/pull/16642
  
    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 #16642: [SPARK-19284][SQL]append to partitioned datasource table...

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

    https://github.com/apache/spark/pull/16642
  
    **[Test build #71662 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/71662/testReport)** for PR 16642 at commit [`7d016d3`](https://github.com/apache/spark/commit/7d016d3a277548c7f914aa09be1190bf9a305416).


---
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 #16642: [SPARK-19284][SQL]append to partitioned datasource table...

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

    https://github.com/apache/spark/pull/16642
  
    **[Test build #71830 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/71830/testReport)** for PR 16642 at commit [`c200b98`](https://github.com/apache/spark/commit/c200b986fed37015a30f99ba2f870dda84cc2ef6).
     * 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 #16642: [SPARK-19284][SQL]append to partitioned datasource table...

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

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


---
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 #16642: [SPARK-19284][SQL]append to partitioned datasourc...

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

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


---
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 #16642: [SPARK-19284][SQL]append to partitioned datasource table...

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

    https://github.com/apache/spark/pull/16642
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/71662/
    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 #16642: [SPARK-19284][SQL]append to partitioned datasource table...

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

    https://github.com/apache/spark/pull/16642
  
    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 #16642: [SPARK-19284][SQL]append to partitioned datasource table...

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

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


---
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 #16642: [SPARK-19284][SQL]append to partitioned datasource table...

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

    https://github.com/apache/spark/pull/16642
  
    **[Test build #71703 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/71703/testReport)** for PR 16642 at commit [`095d421`](https://github.com/apache/spark/commit/095d421a05f985785964c2fae0e7c4f84fc1752a).


---
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 #16642: [SPARK-19284][SQL]append to partitioned datasource table...

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

    https://github.com/apache/spark/pull/16642
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/71793/
    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