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 <husty...@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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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

Reply via email to