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

[GitHub] spark pull request #17371: [SPARK-19903][PYSPARK][SS] window operator miss t...

GitHub user uncleGen opened a pull request:

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

    [SPARK-19903][PYSPARK][SS] window operator miss the `watermark` metadata of time column

    ## What changes were proposed in this pull request?
    
    reproduce code:
    
    ```
    import sys
    from pyspark.sql import SparkSession
    from pyspark.sql.functions import explode, split, window
    bootstrapServers = sys.argv[1]
    subscribeType = sys.argv[2]
    topics = sys.argv[3]
    spark = SparkSession\
      .builder\
      .appName("StructuredKafkaWordCount")\
      .getOrCreate()
    
    lines = spark\
      .readStream\
      .format("kafka")\
      .option("kafka.bootstrap.servers", bootstrapServers)\
      .option(subscribeType, topics)\
      .load()\
      .selectExpr("CAST(value AS STRING)", "CAST(timestamp AS TIMESTAMP)")
    
    words = lines.select(explode(split(lines.value, ' ')).alias('word'),lines.timestamp)
    
    windowedCounts = words.withWatermark("timestamp", "30 seconds").groupBy(
    window(words.timestamp, "30 seconds", "30 seconds"), words.word
    ).count()
    
    query = windowedCounts\
      .writeStream\
      .outputMode('append')\
      .format('console')\ 
      .option("truncate", "false")\
      .start()
    query.awaitTermination()
    ```
    
    An exception was thrown:
    
    ```
    pyspark.sql.utils.AnalysisException: Append output mode not supported when there are streaming aggregations on streaming DataFrames/DataSets without watermark;;
    Aggregate [window#32, word#21], [window#32 AS window#26, word#21, count(1) AS count#31L]
    +- Filter ((timestamp#16 >= window#32.start) && (timestamp#16 < window#32.end))
       +- Expand [ArrayBuffer(named_struct(start, ...]
          +- EventTimeWatermark timestamp#16: timestamp, interval 10 seconds
             +- Project [word#21, timestamp#16]
                +- Generate explode(split(value#15,  )), true, false, [word#21]
                   +- Project [cast(value#1 as string) AS value#15, cast(timestamp#5 as timestamp) AS timestamp#16]
                      +- StreamingRelation DataSource(org.apache.spark.sql.SparkSession ...]
    ```
    
    IIUC, the root cause is:  `words.withWatermark("timestamp", "30 seconds")` add the watermark metadata into time column, but in `groupBy(
    window(words.timestamp, "30 seconds", "30 seconds"), words.word
    )`, the `words.timestamp` miss the metadata. At last, it failed to pass the check:
    
    ```
    if (watermarkAttributes.isEmpty) {
          throwError(
                s"$outputMode output mode not supported when there are streaming aggregations on " +
                    s"streaming DataFrames/DataSets without watermark")(plan)
    }
    ```
    
    after pr, run successfully.
    
    ## How was this patch tested?
    
    Jenkins
    
    Please review http://spark.apache.org/contributing.html before opening a pull request.


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

    $ git pull https://github.com/uncleGen/spark python-window

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

    https://github.com/apache/spark/pull/17371.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 #17371
    
----
commit 654c5121fd26a85036787882d3d2c3b56360b686
Author: uncleGen <hu...@gmail.com>
Date:   2017-03-21T07:49:11Z

    bug fix: window operator miss the `watermark` metadata of time column

----


---
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 #17371: [SPARK-19903][PYSPARK][SS] window operator miss the `wat...

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

    https://github.com/apache/spark/pull/17371
  
    **[Test build #74959 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74959/testReport)** for PR 17371 at commit [`654c512`](https://github.com/apache/spark/commit/654c5121fd26a85036787882d3d2c3b56360b686).


---
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 #17371: [SPARK-19903][PYSPARK][SS] window operator miss t...

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

    https://github.com/apache/spark/pull/17371#discussion_r107101883
  
    --- Diff: python/pyspark/sql/functions.py ---
    @@ -1163,7 +1163,10 @@ def check_string_field(field, fieldName):
                 raise TypeError("%s should be provided as a string" % fieldName)
     
         sc = SparkContext._active_spark_context
    -    time_col = _to_java_column(timeColumn)
    +    if isinstance(timeColumn, Column):
    --- End diff --
    
    @viirya Sounds  reasonable, I pushed an update, take a review 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 #17371: [SPARK-19903][PYSPARK][SS] window operator miss t...

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

    https://github.com/apache/spark/pull/17371#discussion_r107095629
  
    --- Diff: python/pyspark/sql/functions.py ---
    @@ -1163,7 +1163,10 @@ def check_string_field(field, fieldName):
                 raise TypeError("%s should be provided as a string" % fieldName)
     
         sc = SparkContext._active_spark_context
    -    time_col = _to_java_column(timeColumn)
    +    if isinstance(timeColumn, Column):
    --- End diff --
    
    IIUC, it is OK for current codebase. Am I missing something?


---
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 #17371: [SPARK-19903][PYSPARK][SS] window operator miss the `wat...

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

    https://github.com/apache/spark/pull/17371
  
    Can we add an analysis rule that just pulls up missing metadata from attributes in the child?  It could run once after other rules.


---
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 #17371: [SPARK-19903][PYSPARK][SS] window operator miss the `wat...

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

    https://github.com/apache/spark/pull/17371
  
    I really think the core problem here is that we allow you to use resolved attributes at all in the user API.  Unfortunately we are somewhat stuck with that bad decision.  Personally, I never use `df['col']` and only ever use `col("col")` since that avoids the problem.
    
    However, I don't think that piecemeal switching to unresolved attributes is a good idea.


---
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 #17371: [SPARK-19903][PYSPARK][SS] window operator miss the `wat...

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

    https://github.com/apache/spark/pull/17371
  
    I don't think that will solve the problem though.  You will just get a different error message.


---
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 #17371: [SPARK-19903][PYSPARK][SS] window operator miss the `wat...

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

    https://github.com/apache/spark/pull/17371
  
    **[Test build #74967 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74967/testReport)** for PR 17371 at commit [`890c6e6`](https://github.com/apache/spark/commit/890c6e6416417febf8da9960633bb66cda5201c7).


---
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 #17371: [SPARK-19903][PYSPARK][SS] window operator miss the `wat...

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

    https://github.com/apache/spark/pull/17371
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74959/
    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 #17371: [SPARK-19903][PYSPARK][SS] window operator miss the `wat...

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

    https://github.com/apache/spark/pull/17371
  
    gentle ping @uncleGen, is this PR still active?


---
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 #17371: [SPARK-19903][PYSPARK][SS] window operator miss the `wat...

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

    https://github.com/apache/spark/pull/17371
  
    yeah, I just tried it. `IncrementalExecution` will re-new the attribute for each batch. Although we can replace the attribute...


---
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 #17371: [SPARK-19903][PYSPARK][SS] window operator miss the `wat...

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

    https://github.com/apache/spark/pull/17371
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74967/
    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 #17371: [SPARK-19903][PYSPARK][SS] window operator miss the `wat...

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

    https://github.com/apache/spark/pull/17371
  
    IMHO, the output after `withWatermark` should be new attribute and have new expression id. Maybe @zsxwing @marmbrus have more insights on this?
    
    Btw, does this issue also happen in Scala code?


---
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 #17371: [SPARK-19903][PYSPARK][SS] window operator miss the `wat...

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

    https://github.com/apache/spark/pull/17371
  
    For now, after `withWatermark`, we only update the metadata for the column of event time. The expression id is the same. So once we use the column before adding watermark `words.timestamp` as grouping expression, it binds to the old attribute before watermarking.
    
    I am thinking, should we create new expression id for the watermarking column with `withWatermark`? So we must write the query like:
    
        wordsWithWatermark = words.withWatermark("timestamp", "30 seconds")
        windowedCounts = wordsWithWatermark.groupBy(window(wordsWithWatermark.timestamp, "30 seconds", "30 seconds"), wordsWithWatermark.word).count()



---
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 #17371: [SPARK-19903][PYSPARK][SS] window operator miss the `wat...

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

    https://github.com/apache/spark/pull/17371
  
    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 #17371: [SPARK-19903][PYSPARK][SS] window operator miss t...

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

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


---
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 #17371: [SPARK-19903][PYSPARK][SS] window operator miss t...

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

    https://github.com/apache/spark/pull/17371#discussion_r107094080
  
    --- Diff: python/pyspark/sql/functions.py ---
    @@ -1163,7 +1163,10 @@ def check_string_field(field, fieldName):
                 raise TypeError("%s should be provided as a string" % fieldName)
     
         sc = SparkContext._active_spark_context
    -    time_col = _to_java_column(timeColumn)
    +    if isinstance(timeColumn, Column):
    --- End diff --
    
    hmm, doesn't this break the current API? 


---
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 #17371: [SPARK-19903][PYSPARK][SS] window operator miss the `wat...

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

    https://github.com/apache/spark/pull/17371
  
    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 #17371: [SPARK-19903][PYSPARK][SS] window operator miss the `wat...

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

    https://github.com/apache/spark/pull/17371
  
    **[Test build #74959 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74959/testReport)** for PR 17371 at commit [`654c512`](https://github.com/apache/spark/commit/654c5121fd26a85036787882d3d2c3b56360b686).
     * 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 #17371: [SPARK-19903][PYSPARK][SS] window operator miss the `wat...

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

    https://github.com/apache/spark/pull/17371
  
    Unfortunately, yes, allowing resolved attributes in user API will have this kind of trouble.
    
    > However, I don't think that piecemeal switching to unresolved attributes is a good idea.
    
    Agreed. Should we create new attributes after `withWatermark` to avoid the problem? It might be cumbersome from the user side, however.



---
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 #17371: [SPARK-19903][PYSPARK][SS] window operator miss t...

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

    https://github.com/apache/spark/pull/17371#discussion_r107096074
  
    --- Diff: python/pyspark/sql/functions.py ---
    @@ -1163,7 +1163,10 @@ def check_string_field(field, fieldName):
                 raise TypeError("%s should be provided as a string" % fieldName)
     
         sc = SparkContext._active_spark_context
    -    time_col = _to_java_column(timeColumn)
    +    if isinstance(timeColumn, Column):
    --- End diff --
    
    After this change, you can't pass in a `Column`. But it is supported for now.


---
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 #17371: [SPARK-19903][PYSPARK][SS] window operator miss the `wat...

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

    https://github.com/apache/spark/pull/17371
  
    @viirya Great, you give a more clear explanation. 
    
    > I am thinking, should we create new expression id for the watermarking column with withWatermark? So we must write the query like:
    
    It really can fix this problem, but not very user-friendly. 


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