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

[GitHub] spark pull request #20435: [SPARK-23268][SQL]Reorganize packages in data sou...

GitHub user gengliangwang opened a pull request:

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

    [SPARK-23268][SQL]Reorganize packages in data source V2

    ## What changes were proposed in this pull request?
    1. create a new package for partitioning/distribution related classes.
        As Spark will add new concrete implementations of `Distribution` in new releases, it is good to 
        have a new package for partitioning/distribution related classes.
    
    2. move streaming related class to package `org.apache.spark.sql.sources.v2.reader/writer.streaming`, instead of `org.apache.spark.sql.sources.v2.streaming.reader/writer`.
    So that the there won't be package reader/writer inside package streaming, which is quite confusing.
    ## How was this patch tested?
    Unit test.


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

    $ git pull https://github.com/gengliangwang/spark new_pkg

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

    https://github.com/apache/spark/pull/20435.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 #20435
    
----
commit da29b863970b8ca5039d547ae5240e5c34f9f11a
Author: Wang Gengliang <lt...@...>
Date:   2018-01-30T08:31:10Z

    create a new package for partitioning/distribution related classes

commit 3dc56226ed17637c808df9d8d03f60fcbbae9e55
Author: Wang Gengliang <lt...@...>
Date:   2018-01-30T09:16:01Z

    re-org streaming

----


---

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


[GitHub] spark issue #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    **[Test build #86879 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86879/testReport)** for PR 20435 at commit [`1d90cf1`](https://github.com/apache/spark/commit/1d90cf11710e452c5b2adf819da8003c8f96c6e1).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `case class KafkaSourceOffset(partitionToOffsets: Map[TopicPartition, Long]) extends OffsetV2 `


---

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


[GitHub] spark issue #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    LGTM, also  cc @rdblue 


---

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


[GitHub] spark issue #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    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/397/
    Test PASSed.


---

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


[GitHub] spark issue #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

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


---

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


[GitHub] spark issue #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    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/435/
    Test PASSed.


---

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


[GitHub] spark issue #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    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/430/
    Test PASSed.


---

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


[GitHub] spark issue #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    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 #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

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


---

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


[GitHub] spark issue #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

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


---

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


[GitHub] spark issue #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    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 #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

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


---

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


[GitHub] spark issue #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    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 #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    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/390/
    Test PASSed.


---

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


[GitHub] spark issue #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    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 #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    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 #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    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/398/
    Test PASSed.


---

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


[GitHub] spark issue #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    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/385/
    Test PASSed.


---

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


[GitHub] spark pull request #20435: [SPARK-23268][SQL]Reorganize packages in data sou...

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/20435#discussion_r165048049
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala ---
    @@ -403,7 +403,7 @@ class MicroBatchExecution(
               val current = committedOffsets.get(reader).map(off => reader.deserializeOffset(off.json))
               reader.setOffsetRange(
                 toJava(current),
    -            Optional.of(available.asInstanceOf[OffsetV2]))
    +            Optional.of(available.asInstanceOf[streaming.Offset]))
    --- End diff --
    
    shall we still use `OffsetV2`?


---

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


[GitHub] spark issue #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    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/392/
    Test PASSed.


---

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


[GitHub] spark issue #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    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 #20435: [SPARK-23268][SQL]Reorganize packages in data sou...

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/20435#discussion_r165047744
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala ---
    @@ -20,14 +20,15 @@ package org.apache.spark.sql.kafka010
     import org.apache.kafka.common.TopicPartition
     
     import org.apache.spark.sql.execution.streaming.{Offset, SerializedOffset}
    -import org.apache.spark.sql.sources.v2.streaming.reader.{Offset => OffsetV2, PartitionOffset}
    +import org.apache.spark.sql.sources.v2.reader.streaming.{Offset => OffsetV2, PartitionOffset}
     
     /**
      * An [[Offset]] for the [[KafkaSource]]. This one tracks all partitions of subscribed topics and
      * their offsets.
      */
     private[kafka010]
    -case class KafkaSourceOffset(partitionToOffsets: Map[TopicPartition, Long]) extends OffsetV2 {
    +case class KafkaSourceOffset(partitionToOffsets: Map[TopicPartition, Long])
    +  extends OffsetV2 {
    --- End diff --
    
    unnecessary change?


---

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


[GitHub] spark issue #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

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


---

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


[GitHub] spark issue #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

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


---

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


[GitHub] spark issue #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    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 #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    cc @marmbrus 


---

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


[GitHub] spark issue #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    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 #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    **[Test build #86830 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86830/testReport)** for PR 20435 at commit [`f451194`](https://github.com/apache/spark/commit/f4511943418ed4b863991104dd22d65ae3a1b9c9).
     * This patch **fails to build**.
     * 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 #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    **[Test build #86838 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86838/testReport)** for PR 20435 at commit [`0b6b59e`](https://github.com/apache/spark/commit/0b6b59ea86d00e8128af98891fa5d10934cb65cd).
     * 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 #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

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


---

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


[GitHub] spark issue #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    **[Test build #86814 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86814/testReport)** for PR 20435 at commit [`3dc5622`](https://github.com/apache/spark/commit/3dc56226ed17637c808df9d8d03f60fcbbae9e55).
     * This patch **fails Scala style 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 #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    cc @zsxwing @marmbrus too


---

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


[GitHub] spark issue #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    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 #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    **[Test build #86838 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86838/testReport)** for PR 20435 at commit [`0b6b59e`](https://github.com/apache/spark/commit/0b6b59ea86d00e8128af98891fa5d10934cb65cd).


---

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


[GitHub] spark issue #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

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


---

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


[GitHub] spark issue #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    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 #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

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


---

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


[GitHub] spark issue #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    **[Test build #86879 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86879/testReport)** for PR 20435 at commit [`1d90cf1`](https://github.com/apache/spark/commit/1d90cf11710e452c5b2adf819da8003c8f96c6e1).


---

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


[GitHub] spark pull request #20435: [SPARK-23268][SQL]Reorganize packages in data sou...

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/20435#discussion_r164943009
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala ---
    @@ -20,14 +20,16 @@ package org.apache.spark.sql.kafka010
     import org.apache.kafka.common.TopicPartition
     
     import org.apache.spark.sql.execution.streaming.{Offset, SerializedOffset}
    -import org.apache.spark.sql.sources.v2.streaming.reader.{Offset => OffsetV2, PartitionOffset}
    +import org.apache.spark.sql.sources.v2.reader.streaming
    +import org.apache.spark.sql.sources.v2.reader.streaming.PartitionOffset
    --- End diff --
    
    can we keep it same as before?
    ```
    import org.apache.spark.sql.sources.v2.reader.streaming.{Offset => OffsetV2, PartitionOffset}
    ```


---

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


[GitHub] spark issue #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    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 #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    **[Test build #86840 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86840/testReport)** for PR 20435 at commit [`0b6b59e`](https://github.com/apache/spark/commit/0b6b59ea86d00e8128af98891fa5d10934cb65cd).


---

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


[GitHub] spark issue #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    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/382/
    Test PASSed.


---

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


[GitHub] spark issue #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    **[Test build #86820 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86820/testReport)** for PR 20435 at commit [`17f8a5e`](https://github.com/apache/spark/commit/17f8a5e78e538928c9069e945907e1e92eedc972).
     * This patch **fails to build**.
     * This patch **does not merge 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 #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    **[Test build #86840 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86840/testReport)** for PR 20435 at commit [`0b6b59e`](https://github.com/apache/spark/commit/0b6b59ea86d00e8128af98891fa5d10934cb65cd).
     * 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 #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

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


---

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


[GitHub] spark pull request #20435: [SPARK-23268][SQL]Reorganize packages in data sou...

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

    https://github.com/apache/spark/pull/20435#discussion_r164953225
  
    --- Diff: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala ---
    @@ -20,14 +20,16 @@ package org.apache.spark.sql.kafka010
     import org.apache.kafka.common.TopicPartition
     
     import org.apache.spark.sql.execution.streaming.{Offset, SerializedOffset}
    -import org.apache.spark.sql.sources.v2.streaming.reader.{Offset => OffsetV2, PartitionOffset}
    +import org.apache.spark.sql.sources.v2.reader.streaming
    +import org.apache.spark.sql.sources.v2.reader.streaming.PartitionOffset
     
     /**
      * An [[Offset]] for the [[KafkaSource]]. This one tracks all partitions of subscribed topics and
    --- End diff --
    
    Should this `Offset` be `streaming.Offset`?


---

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


[GitHub] spark issue #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    **[Test build #86824 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86824/testReport)** for PR 20435 at commit [`c7c0a1d`](https://github.com/apache/spark/commit/c7c0a1d9f0d8a78f4fe7af825f41150d33bd676b).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `case class KafkaSourceOffset(partitionToOffsets: Map[TopicPartition, Long])`


---

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


[GitHub] spark issue #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    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 #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    **[Test build #86820 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86820/testReport)** for PR 20435 at commit [`17f8a5e`](https://github.com/apache/spark/commit/17f8a5e78e538928c9069e945907e1e92eedc972).


---

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


[GitHub] spark pull request #20435: [SPARK-23268][SQL]Reorganize packages in data sou...

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

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


---

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


[GitHub] spark issue #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    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 #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    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 #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    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 #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

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


---

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


[GitHub] spark issue #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    **[Test build #86839 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86839/testReport)** for PR 20435 at commit [`0b6b59e`](https://github.com/apache/spark/commit/0b6b59ea86d00e8128af98891fa5d10934cb65cd).


---

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


[GitHub] spark issue #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    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 #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    **[Test build #86817 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86817/testReport)** for PR 20435 at commit [`719cae2`](https://github.com/apache/spark/commit/719cae295d4c08fcc04fcab030c70bdd8798a2c1).


---

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


[GitHub] spark issue #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    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 #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    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 #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    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 #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    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 #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

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


---

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


[GitHub] spark issue #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    **[Test build #86832 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86832/testReport)** for PR 20435 at commit [`0b6b59e`](https://github.com/apache/spark/commit/0b6b59ea86d00e8128af98891fa5d10934cb65cd).
     * This patch **fails PySpark 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 #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    **[Test build #86839 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86839/testReport)** for PR 20435 at commit [`0b6b59e`](https://github.com/apache/spark/commit/0b6b59ea86d00e8128af98891fa5d10934cb65cd).
     * 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 #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    **[Test build #86832 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86832/testReport)** for PR 20435 at commit [`0b6b59e`](https://github.com/apache/spark/commit/0b6b59ea86d00e8128af98891fa5d10934cb65cd).


---

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


[GitHub] spark pull request #20435: [SPARK-23268][SQL]Reorganize packages in data sou...

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

    https://github.com/apache/spark/pull/20435#discussion_r164953380
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/memoryV2.scala ---
    @@ -30,9 +30,8 @@ import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
     import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.{Append, Complete, Update}
     import org.apache.spark.sql.execution.streaming.Sink
     import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2}
    -import org.apache.spark.sql.sources.v2.streaming.StreamWriteSupport
    -import org.apache.spark.sql.sources.v2.streaming.writer.StreamWriter
    -import org.apache.spark.sql.sources.v2.writer._
    +import org.apache.spark.sql.sources.v2.writer.{StreamWriteSupport, _}
    --- End diff --
    
    `import org.apache.spark.sql.sources.v2.writer._`?


---

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


[GitHub] spark issue #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    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/379/
    Test PASSed.


---

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


[GitHub] spark issue #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    cc @jose-torres 


---

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


[GitHub] spark issue #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    Streaming part LGTM; I have no particular opinion or context on the distribution stuff.


---

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


[GitHub] spark issue #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    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 #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    **[Test build #86817 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86817/testReport)** for PR 20435 at commit [`719cae2`](https://github.com/apache/spark/commit/719cae295d4c08fcc04fcab030c70bdd8798a2c1).
     * This patch **fails to build**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `case class KafkaSourceOffset(partitionToOffsets: Map[TopicPartition, Long])`


---

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


[GitHub] spark issue #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    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 #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    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 #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    LGTM to adding the new package of partitioning/distribution. 


---

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


[GitHub] spark issue #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    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/378/
    Test PASSed.


---

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


[GitHub] spark issue #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

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


---

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


[GitHub] spark issue #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    **[Test build #86873 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86873/testReport)** for PR 20435 at commit [`e609060`](https://github.com/apache/spark/commit/e609060abb47fb73b48bf2aff46ed33e769364ea).
     * 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 #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    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 #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    **[Test build #86827 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86827/testReport)** for PR 20435 at commit [`d272952`](https://github.com/apache/spark/commit/d27295251e2b4e0e1c2340cd361f9e7c7d7f4f71).
     * This patch **fails to build**.
     * 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 #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    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/388/
    Test PASSed.


---

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


[GitHub] spark issue #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    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 #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

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


---

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


[GitHub] spark issue #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

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


---

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


[GitHub] spark issue #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

    https://github.com/apache/spark/pull/20435
  
    Thanks! Merged to master/2.3


---

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


[GitHub] spark issue #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

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


---

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


[GitHub] spark issue #20435: [SPARK-23268][SQL]Reorganize packages in data source V2

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

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


---

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