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

[GitHub] spark pull request #13549: Added support for sorting after streaming aggrega...

GitHub user tdas opened a pull request:

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

    Added support for sorting after streaming aggregation with complete mode

    ## What changes were proposed in this pull request?
    
    When the output mode is complete, then the output of a streaming aggregation essentially will contain the complete aggregates every time. So this is not different from a batch dataset within an incremental execution. Other non-streaming operations should be supported on this dataset. In this PR, I am just adding support for sorting, as it is a common useful functionality. Support for other operations will come later.
    
    ## How was this patch tested?
    Additional unit tests.

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

    $ git pull https://github.com/tdas/spark SPARK-15812

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

    https://github.com/apache/spark/pull/13549.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 #13549
    
----
commit a287a9a6955b58609722947b3480bc5578d0b37d
Author: Tathagata Das <ta...@gmail.com>
Date:   2016-06-08T01:51:38Z

    Added support for sorting after streaming aggregation with complete mode

----


---
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 #13549: [SPARK-15812][SQ][Streaming] Added support for sorting a...

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

    https://github.com/apache/spark/pull/13549
  
    **[Test build #60242 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/60242/consoleFull)** for PR 13549 at commit [`810b802`](https://github.com/apache/spark/commit/810b802060792da78b61c44ad9b656ce3e02b4a3).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


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

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


[GitHub] spark issue #13549: Added support for sorting after streaming aggregation wi...

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

    https://github.com/apache/spark/pull/13549
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/60148/
    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 #13549: [SPARK-15812][SQ][Streaming] Added support for sorting a...

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

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


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

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


[GitHub] spark pull request #13549: Added support for sorting after streaming aggrega...

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

    https://github.com/apache/spark/pull/13549#discussion_r66182722
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala ---
    @@ -123,27 +159,6 @@ object UnsupportedOperationChecker {
             case _ =>
           }
         }
    -
    -    // Checks related to aggregations
    -    val aggregates = plan.collect { case a @ Aggregate(_, _, _) if a.isStreaming => a }
    --- End diff --
    
    This is moved above to make sure that outputmode related failure occur before other failures, as those failures are more fundamental.


---
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 #13549: [SPARK-15812][SQ][Streaming] Added support for sorting a...

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

    https://github.com/apache/spark/pull/13549
  
    **[Test build #60242 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/60242/consoleFull)** for PR 13549 at commit [`810b802`](https://github.com/apache/spark/commit/810b802060792da78b61c44ad9b656ce3e02b4a3).


---
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 #13549: Added support for sorting after streaming aggregation wi...

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

    https://github.com/apache/spark/pull/13549
  
    Any reason why there isn't a jira ticket?



---
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 #13549: [SPARK-15812][SQ][Streaming] Added support for sorting a...

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

    https://github.com/apache/spark/pull/13549
  
    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 #13549: [SPARK-15812][SQ][Streaming] Added support for sorting a...

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

    https://github.com/apache/spark/pull/13549
  
    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 #13549: Added support for sorting after streaming aggregation wi...

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

    https://github.com/apache/spark/pull/13549
  
    **[Test build #60148 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/60148/consoleFull)** for PR 13549 at commit [`a287a9a`](https://github.com/apache/spark/commit/a287a9a6955b58609722947b3480bc5578d0b37d).


---
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 #13549: [SPARK-15812][SQ][Streaming] Added support for so...

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

    https://github.com/apache/spark/pull/13549#discussion_r66491058
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala ---
    @@ -43,6 +43,41 @@ object UnsupportedOperationChecker {
             "Queries without streaming sources cannot be executed with write.startStream()")(plan)
         }
     
    +    // Disallow multiple streaming aggregations
    +    val aggregates = plan.collect { case a@Aggregate(_, _, _) if a.isStreaming => a }
    +
    +    if (aggregates.size > 1) {
    +      throwError(
    +        "Multiple streaming aggregations are not supported with " +
    +          "streaming DataFrames/Datasets")(plan)
    +    }
    +
    +    // Disallow some output mode
    +    outputMode match {
    +      case InternalOutputModes.Append if aggregates.nonEmpty =>
    +        throwError(
    +          s"$outputMode output mode not supported when there are streaming aggregations on " +
    +            s"streaming DataFrames/DataSets")(plan)
    +
    +      case InternalOutputModes.Complete | InternalOutputModes.Update if aggregates.isEmpty =>
    +        throwError(
    +          s"$outputMode output mode not supported when there are no streaming aggregations on " +
    +            s"streaming DataFrames/Datasets")(plan)
    +
    +      case _ =>
    +    }
    +
    +    /**
    +     * Whether the subplan will contain complete data or incremental data in every incremental
    +     * execution. Some operations may be allowed only when the child logical plan gives complete
    +     * data.
    +     */
    +    def containsCompleteData(subplan: LogicalPlan): Boolean = {
    +      val aggs = plan.collect { case a@Aggregate(_, _, _) if a.isStreaming => a }
    --- End diff --
    
    I completely agree. This whole file needs to be restructured in future. This is really a 2.0 interim fix.


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

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


[GitHub] spark issue #13549: Added support for sorting after streaming aggregation wi...

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

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


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

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


[GitHub] spark pull request #13549: [SPARK-15812][SQ][Streaming] Added support for so...

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

    https://github.com/apache/spark/pull/13549#discussion_r66361125
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala ---
    @@ -43,6 +43,41 @@ object UnsupportedOperationChecker {
             "Queries without streaming sources cannot be executed with write.startStream()")(plan)
         }
     
    +    // Disallow multiple streaming aggregations
    +    val aggregates = plan.collect { case a@Aggregate(_, _, _) if a.isStreaming => a }
    +
    +    if (aggregates.size > 1) {
    +      throwError(
    +        "Multiple streaming aggregations are not supported with " +
    +          "streaming DataFrames/Datasets")(plan)
    +    }
    +
    +    // Disallow some output mode
    +    outputMode match {
    +      case InternalOutputModes.Append if aggregates.nonEmpty =>
    +        throwError(
    +          s"$outputMode output mode not supported when there are streaming aggregations on " +
    +            s"streaming DataFrames/DataSets")(plan)
    +
    +      case InternalOutputModes.Complete | InternalOutputModes.Update if aggregates.isEmpty =>
    +        throwError(
    +          s"$outputMode output mode not supported when there are no streaming aggregations on " +
    +            s"streaming DataFrames/Datasets")(plan)
    +
    +      case _ =>
    +    }
    +
    +    /**
    +     * Whether the subplan will contain complete data or incremental data in every incremental
    +     * execution. Some operations may be allowed only when the child logical plan gives complete
    +     * data.
    +     */
    +    def containsCompleteData(subplan: LogicalPlan): Boolean = {
    +      val aggs = plan.collect { case a@Aggregate(_, _, _) if a.isStreaming => a }
    --- End diff --
    
    There is a lot of implicit coupling here.  I think the correctness of this function depends on the fact that:
     - we only allow aggregation
     - currently the aggregate operator is responsible for outputting complete data
     - aggregates always output complete data if the mode is complete


---
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 #13549: Added support for sorting after streaming aggregation wi...

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

    https://github.com/apache/spark/pull/13549
  
    **[Test build #60148 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/60148/consoleFull)** for PR 13549 at commit [`a287a9a`](https://github.com/apache/spark/commit/a287a9a6955b58609722947b3480bc5578d0b37d).
     * 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 #13549: [SPARK-15812][SQ][Streaming] Added support for so...

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

    https://github.com/apache/spark/pull/13549#discussion_r66361373
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala ---
    @@ -104,6 +104,31 @@ class StreamingAggregationSuite extends StreamTest with BeforeAndAfterAll {
         }
       }
     
    +  test("operation after aggregate in complete mode") {
    --- End diff --
    
    what operation


---
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 #13549: [SPARK-15812][SQ][Streaming] Added support for sorting a...

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

    https://github.com/apache/spark/pull/13549
  
    Added. My bad. 


---
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 #13549: [SPARK-15812][SQ][Streaming] Added support for sorting a...

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

    https://github.com/apache/spark/pull/13549
  
    @marmbrus could you take a look once again. 


---
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 #13549: [SPARK-15812][SQ][Streaming] Added support for sorting a...

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

    https://github.com/apache/spark/pull/13549
  
    This is okay for 2.0, but we'll need to rethink the way we are doing query planning to handle incremental input.


---
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 #13549: [SPARK-15812][SQ][Streaming] Added support for sorting a...

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

    https://github.com/apache/spark/pull/13549
  
    **[Test build #60243 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/60243/consoleFull)** for PR 13549 at commit [`810b802`](https://github.com/apache/spark/commit/810b802060792da78b61c44ad9b656ce3e02b4a3).


---
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 #13549: [SPARK-15812][SQ][Streaming] Added support for sorting a...

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

    https://github.com/apache/spark/pull/13549
  
    **[Test build #60243 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/60243/consoleFull)** for PR 13549 at commit [`810b802`](https://github.com/apache/spark/commit/810b802060792da78b61c44ad9b656ce3e02b4a3).
     * 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 #13549: [SPARK-15812][SQ][Streaming] Added support for sorting a...

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

    https://github.com/apache/spark/pull/13549
  
    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 pull request #13549: [SPARK-15812][SQ][Streaming] Added support for so...

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

    https://github.com/apache/spark/pull/13549#discussion_r66361438
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala ---
    @@ -189,9 +189,20 @@ class UnsupportedOperationsSuite extends SparkFunSuite {
         _.intersect(_),
         streamStreamSupported = false)
     
    -  // Unary operations
    +  // Sort: supported only on batch subplans and on aggregation + complete output mode
       testUnaryOperatorInStreamingPlan("sort", Sort(Nil, true, _))
       testUnaryOperatorInStreamingPlan("sort partitions", SortPartitions(Nil, _), expectedMsg = "sort")
    +  assertSupportedInStreamingPlan(
    +    "sort - sort over aggregated data in Complete output mode",
    +    Sort(Nil, true, Aggregate(Nil, aggExprs("c"), streamRelation)),
    --- End diff --
    
    I think these would be easier to read written with the DSL.


---
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 #13549: [SPARK-15812][SQ][Streaming] Added support for sorting a...

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

    https://github.com/apache/spark/pull/13549
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/60243/
    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 #13549: Added support for sorting after streaming aggregation wi...

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

    https://github.com/apache/spark/pull/13549
  
    @marmbrus 


---
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 #13549: [SPARK-15812][SQ][Streaming] Added support for so...

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

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


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