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

[GitHub] spark pull request #18954: [SPARK-17654] [SQL] Enable creating hive bucketed...

GitHub user tejasapatil opened a pull request:

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

    [SPARK-17654] [SQL] Enable creating hive bucketed tables

    ## What changes were proposed in this pull request?
    
    ### Semantics:
    - If the Hive table is bucketed, then INSERT node expect the child distribution to be based on the hash of the bucket columns. Else it would be empty. (Just to compare with Spark native bucketing : the required distribution is not enforced even if the table is bucketed or not... this saves the shuffle in comparison with hive).
    - Sort ordering for INSERT node over Hive bucketed table is determined as follows:
    
    | Table type   | Normal table | Bucketed table |
    | ------------- | ------------- | ------------- |
    | non-partitioned insert  | Nil | sort columns |
    | static partition   | Nil | sort columns |
    | dynamic partitions   | partition columns | (partition columns + bucketId + sort columns) |
    
    Just to compare how sort ordering is expressed for Spark native bucketing:
    
    | Table type   | Normal table | Bucketed table |
    | ------------- | ------------- | ------------- |
    |  sort ordering | partition columns | (partition columns + bucketId + sort columns) |
    
    Why is there a difference ? With hive, since there bucketed insertions would need a shuffle, sort ordering can be relaxed for both non-partitioned and static partition cases. Every RDD partition would get rows corresponding to a single bucket so those can be written to corresponding output file after sort. In case of dynamic partitions, the rows need to be routed to appropriate partition which makes it similar to Spark's constraints.
    
    - Only `Overwrite` mode is allowed for hive bucketed tables as any other mode will break the bucketing guarantees of the table. This is a difference wrt how Spark bucketing works.
    - With the PR, if there are no files created for empty buckets, the query will fail. Will support creation of empty files in coming iteration. This is a difference wrt how Spark bucketing works as it does NOT need files for empty buckets.
    
    ### Summary of changes done:
    - `ClusteredDistribution` and `HashPartitioning` are modified to store the hashing function used.
    - `RunnableCommand`'s' can now express the required distribution and ordering. This is used by `ExecutedCommandExec` which run these commands
      - The good thing about this is that I could remove the logic for enforcing sort ordering inside `FileFormatWriter` which felt out of place. Ideally, this kinda adding of physical nodes should be done within the planner which is what happens with this PR.
    - `InsertIntoHiveTable` enforces both distribution and sort ordering
    - `InsertIntoHadoopFsRelationCommand` enforces sort ordering ONLY (and not the distribution)
    - Fixed a bug due to which any alter commands to bucketed table (eg. updating stats) would wipe out the bucketing spec from metastore. This made insertions to bucketed table non-idempotent operation.
    
    ## How was this patch tested?
    
    - Added new unit tests

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

    $ git pull https://github.com/tejasapatil/spark bucket_write

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

    https://github.com/apache/spark/pull/18954.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 #18954
    
----
commit 43fae74ff017959edbffa1cbd1405f58c5abe279
Author: Tejas Patil <te...@fb.com>
Date:   2017-08-03T22:57:54Z

    bucketed writer implementation

commit 4b009a909768f2d8066fb58a45d1c54378fa8ff9
Author: Tejas Patil <te...@fb.com>
Date:   2017-08-15T23:27:06Z

    Move `requiredOrdering` into RunnableCommand instead of `FileFormatWriter`

----


---
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 #18954: [SPARK-17654] [SQL] Enable populating hive bucket...

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

    https://github.com/apache/spark/pull/18954#discussion_r133874032
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala ---
    @@ -50,7 +50,9 @@ case class EnsureRequirements(conf: SQLConf) extends Rule[SparkPlan] {
           numPartitions: Int): Partitioning = {
         requiredDistribution match {
           case AllTuples => SinglePartition
    -      case ClusteredDistribution(clustering) => HashPartitioning(clustering, numPartitions)
    +      case ClusteredDistribution(clustering, numClusters, hashingFunctionClass) =>
    +        HashPartitioning(clustering, numClusters.getOrElse(numPartitions),
    --- End diff --
    
    This is going to create a partitioning that satisfies that distribution. According to modified  `HashPartitioning`, if `numPartitions` isn't equal to `numClusters`, `satisfies` returns `false`. It seems a conflict if we ask to create a partitioning of `numPartitions` with a `ClusteredDistribution` of `numClusters`?


---
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 #18954: [SPARK-17654] [SQL] Enable populating hive bucketed tabl...

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

    https://github.com/apache/spark/pull/18954
  
    **[Test build #80809 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/80809/testReport)** for PR 18954 at commit [`4b2f1eb`](https://github.com/apache/spark/commit/4b2f1eb956eb1cbb3acfdaee99775fd1c948352b).


---
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 #18954: [SPARK-17654] [SQL] Enable populating hive bucketed tabl...

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

    https://github.com/apache/spark/pull/18954
  
    **[Test build #80814 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/80814/testReport)** for PR 18954 at commit [`4b2f1eb`](https://github.com/apache/spark/commit/4b2f1eb956eb1cbb3acfdaee99775fd1c948352b).
     * 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 #18954: [SPARK-17654] [SQL] Enable creating hive bucketed tables

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

    https://github.com/apache/spark/pull/18954
  
    cc @cloud-fan @gatorsmile @sameeragarwal @rxin 


---
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 #18954: [SPARK-17654] [SQL] Enable populating hive bucketed tabl...

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

    https://github.com/apache/spark/pull/18954
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/80809/
    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 #18954: [SPARK-17654] [SQL] Enable creating hive bucketed tables

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

    https://github.com/apache/spark/pull/18954
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/80711/
    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 #18954: [SPARK-17654] [SQL] Enable populating hive bucketed tabl...

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

    https://github.com/apache/spark/pull/18954
  
    I have a new PR (https://github.com/apache/spark/pull/19001) which supersedes this one. It has everything this PR does (ie. writer side changes) plus reader side 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 #18954: [SPARK-17654] [SQL] Enable creating hive bucketed tables

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

    https://github.com/apache/spark/pull/18954
  
    **[Test build #80711 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/80711/testReport)** for PR 18954 at commit [`4b009a9`](https://github.com/apache/spark/commit/4b009a909768f2d8066fb58a45d1c54378fa8ff9).


---
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 #18954: [SPARK-17654] [SQL] Enable populating hive bucket...

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

    https://github.com/apache/spark/pull/18954#discussion_r134103430
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala ---
    @@ -50,7 +50,9 @@ case class EnsureRequirements(conf: SQLConf) extends Rule[SparkPlan] {
           numPartitions: Int): Partitioning = {
         requiredDistribution match {
           case AllTuples => SinglePartition
    -      case ClusteredDistribution(clustering) => HashPartitioning(clustering, numPartitions)
    +      case ClusteredDistribution(clustering, numClusters, hashingFunctionClass) =>
    +        HashPartitioning(clustering, numClusters.getOrElse(numPartitions),
    --- End diff --
    
    Good point. I gave this more thought and have made 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 #18954: [SPARK-17654] [SQL] Enable populating hive bucketed tabl...

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

    https://github.com/apache/spark/pull/18954
  
    **[Test build #80878 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/80878/testReport)** for PR 18954 at commit [`9b8f084`](https://github.com/apache/spark/commit/9b8f0842eb5b61e6ae1a9fc76aebe9ff88c2a39b).
     * This patch **fails SparkR unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `sealed trait Distribution `


---
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 #18954: [SPARK-17654] [SQL] Enable populating hive bucketed tabl...

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

    https://github.com/apache/spark/pull/18954
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/80878/
    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 #18954: [SPARK-17654] [SQL] Enable populating hive bucketed tabl...

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

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


---
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 #18954: [SPARK-17654] [SQL] Enable creating hive bucketed tables

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

    https://github.com/apache/spark/pull/18954
  
    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 #18954: [SPARK-17654] [SQL] Enable populating hive bucketed tabl...

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

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


---
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 #18954: [SPARK-17654] [SQL] Enable creating hive bucketed tables

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

    https://github.com/apache/spark/pull/18954
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/80733/
    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 #18954: [SPARK-17654] [SQL] Enable populating hive bucketed tabl...

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

    https://github.com/apache/spark/pull/18954
  
    **[Test build #80814 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/80814/testReport)** for PR 18954 at commit [`4b2f1eb`](https://github.com/apache/spark/commit/4b2f1eb956eb1cbb3acfdaee99775fd1c948352b).


---
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 #18954: [SPARK-17654] [SQL] Enable populating hive bucketed tabl...

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

    https://github.com/apache/spark/pull/18954
  
    **[Test build #81006 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/81006/testReport)** for PR 18954 at commit [`d5cf3c9`](https://github.com/apache/spark/commit/d5cf3c91eb9918e570e42fdbba64be09aab60e3c).
     * 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 #18954: [SPARK-17654] [SQL] Enable populating hive bucketed tabl...

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

    https://github.com/apache/spark/pull/18954
  
    @tejasapatil Can you close this for now because it's not active for a long time.


---

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


[GitHub] spark issue #18954: [SPARK-17654] [SQL] Enable populating hive bucketed tabl...

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

    https://github.com/apache/spark/pull/18954
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/80814/
    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 #18954: [SPARK-17654] [SQL] Enable creating hive bucketed tables

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

    https://github.com/apache/spark/pull/18954
  
    Jenkins 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 #18954: [SPARK-17654] [SQL] Enable populating hive bucketed tabl...

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

    https://github.com/apache/spark/pull/18954
  
    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 #18954: [SPARK-17654] [SQL] Enable populating hive bucketed tabl...

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

    https://github.com/apache/spark/pull/18954
  
    Jenkins 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 #18954: [SPARK-17654] [SQL] Enable creating hive bucketed tables

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

    https://github.com/apache/spark/pull/18954
  
    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 #18954: [SPARK-17654] [SQL] Enable populating hive bucketed tabl...

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

    https://github.com/apache/spark/pull/18954
  
    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 #18954: [SPARK-17654] [SQL] Enable creating hive bucketed tables

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

    https://github.com/apache/spark/pull/18954
  
    **[Test build #80711 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/80711/testReport)** for PR 18954 at commit [`4b009a9`](https://github.com/apache/spark/commit/4b009a909768f2d8066fb58a45d1c54378fa8ff9).
     * This patch **fails due to an unknown error code, -9**.
     * 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 #18954: [SPARK-17654] [SQL] Enable creating hive bucketed tables

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

    https://github.com/apache/spark/pull/18954
  
    **[Test build #80733 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/80733/testReport)** for PR 18954 at commit [`4b009a9`](https://github.com/apache/spark/commit/4b009a909768f2d8066fb58a45d1c54378fa8ff9).
     * 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 #18954: [SPARK-17654] [SQL] Enable populating hive bucket...

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

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


---

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


[GitHub] spark issue #18954: [SPARK-17654] [SQL] Enable populating hive bucketed tabl...

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

    https://github.com/apache/spark/pull/18954
  
    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 #18954: [SPARK-17654] [SQL] Enable populating hive bucketed tabl...

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

    https://github.com/apache/spark/pull/18954
  
    **[Test build #80809 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/80809/testReport)** for PR 18954 at commit [`4b2f1eb`](https://github.com/apache/spark/commit/4b2f1eb956eb1cbb3acfdaee99775fd1c948352b).
     * This patch **fails SparkR 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 #18954: [SPARK-17654] [SQL] Enable populating hive bucketed tabl...

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

    https://github.com/apache/spark/pull/18954
  
    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 #18954: [SPARK-17654] [SQL] Enable creating hive bucketed tables

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

    https://github.com/apache/spark/pull/18954
  
    Jenkins test 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 #18954: [SPARK-17654] [SQL] Enable populating hive bucketed tabl...

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

    https://github.com/apache/spark/pull/18954
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/81006/
    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 #18954: [SPARK-17654] [SQL] Enable creating hive bucketed tables

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

    https://github.com/apache/spark/pull/18954
  
    **[Test build #80733 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/80733/testReport)** for PR 18954 at commit [`4b009a9`](https://github.com/apache/spark/commit/4b009a909768f2d8066fb58a45d1c54378fa8ff9).


---
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