diff options
author | Tathagata Das <tathagata.das1565@gmail.com> | 2016-11-16 10:00:59 -0800 |
---|---|---|
committer | Shixiong Zhu <shixiong@databricks.com> | 2016-11-16 10:00:59 -0800 |
commit | 0048ce7ce64b02cbb6a1c4a2963a0b1b9541047e (patch) | |
tree | e267944042ecd6cc22a9078532baee2051c4a660 /python/pyspark/sql/streaming.py | |
parent | 608ecc512b759514c75a1b475582f237ed569f10 (diff) | |
download | spark-0048ce7ce64b02cbb6a1c4a2963a0b1b9541047e.tar.gz spark-0048ce7ce64b02cbb6a1c4a2963a0b1b9541047e.tar.bz2 spark-0048ce7ce64b02cbb6a1c4a2963a0b1b9541047e.zip |
[SPARK-18459][SPARK-18460][STRUCTUREDSTREAMING] Rename triggerId to batchId and add triggerDetails to json in StreamingQueryStatus
## What changes were proposed in this pull request?
SPARK-18459: triggerId seems like a number that should be increasing with each trigger, whether or not there is data in it. However, actually, triggerId increases only where there is a batch of data in a trigger. So its better to rename it to batchId.
SPARK-18460: triggerDetails was missing from json representation. Fixed it.
## How was this patch tested?
Updated existing unit tests.
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes #15895 from tdas/SPARK-18459.
Diffstat (limited to 'python/pyspark/sql/streaming.py')
-rw-r--r-- | python/pyspark/sql/streaming.py | 6 |
1 files changed, 3 insertions, 3 deletions
diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index f326f16232..0e4589be97 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -212,12 +212,12 @@ class StreamingQueryStatus(object): Processing rate 23.5 rows/sec Latency: 345.0 ms Trigger details: + batchId: 5 isDataPresentInTrigger: true isTriggerActive: true latency.getBatch.total: 20 latency.getOffset.total: 10 numRows.input.total: 100 - triggerId: 5 Source statuses [1 source]: Source 1 - MySource1 Available offset: 0 @@ -341,8 +341,8 @@ class StreamingQueryStatus(object): If no trigger is currently active, then it will have details of the last completed trigger. >>> sqs.triggerDetails - {u'triggerId': u'5', u'latency.getBatch.total': u'20', u'numRows.input.total': u'100', - u'isTriggerActive': u'true', u'latency.getOffset.total': u'10', + {u'latency.getBatch.total': u'20', u'numRows.input.total': u'100', + u'isTriggerActive': u'true', u'batchId': u'5', u'latency.getOffset.total': u'10', u'isDataPresentInTrigger': u'true'} """ return self._jsqs.triggerDetails() |