Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-17731][SQL][STREAMING] Metrics for structured streaming #15307

Closed
wants to merge 48 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
48 commits
Select commit Hold shift + click to select a range
deb9a3f
Added metrics
tdas Sep 30, 2016
f3c5dcb
Merge remote-tracking branch 'apache-github/master' into HEAD
tdas Sep 30, 2016
6106a9d
Mima excludes
tdas Sep 30, 2016
3c0c7df
Added queryStatus, and state metric tests in listener
tdas Sep 30, 2016
3b063a2
Fixed unit test
tdas Sep 30, 2016
b31b606
More Mima excludes
tdas Sep 30, 2016
e5a5b7e
Added tests, and addressed some comments.
tdas Oct 3, 2016
348b59e
Merge remote-tracking branch 'apache-github/master' into SPARK-17731
tdas Oct 3, 2016
43e1ab1
Addressed comments, and added more tests
tdas Oct 4, 2016
62f5b0b
Fix polling delay
tdas Oct 4, 2016
4074428
Added synchronization
tdas Oct 4, 2016
2bcbcd2
Renamed *info to *status for consistency, except StreamingQueryInfo
tdas Oct 4, 2016
2d00f26
Added exclude
tdas Oct 4, 2016
d69935c
Added latency.sourceGetBatch
tdas Oct 5, 2016
05f22d7
Made sure status objects are private[sql]
tdas Oct 5, 2016
bbd0d8b
Addressed comments
tdas Oct 5, 2016
02603c7
Minor
tdas Oct 5, 2016
9fd6815
Fixed unnecessary mima change
tdas Oct 5, 2016
f5732a5
Addressed more comments
tdas Oct 5, 2016
e708b3b
Added periodic warning
tdas Oct 5, 2016
a47c73a
Remove unnecessary test
tdas Oct 5, 2016
f883b48
Removed locks
tdas Oct 5, 2016
713f773
Removed output rate, renamed StreamingQueryInfo -> StreamingQueryStat…
tdas Oct 6, 2016
bad5644
Added Mima excludes
tdas Oct 6, 2016
8bb554b
More refactoring, and file num rows test
tdas Oct 6, 2016
dfdca0c
Merge remote-tracking branch 'apache-github/master' into SPARK-17731
tdas Oct 6, 2016
2918525
Added tests for sources
tdas Oct 7, 2016
10d1c24
Fixed mima
tdas Oct 7, 2016
8537783
Pretty strings from statuses
tdas Oct 7, 2016
1655c96
Fixed mima and improved status desc
tdas Oct 7, 2016
3984d54
Added python APIs for StreamingQueryStatus, SourceStatus, SinkStatus
tdas Oct 7, 2016
5e0aa57
Fixed StreamingQuerySuite
tdas Oct 7, 2016
e8ebfc6
Merge remote-tracking branch 'apache-github/master' into SPARK-17731
tdas Oct 10, 2016
b747b23
Fixed python 3.4 tests
tdas Oct 11, 2016
dca9939
Addressed comments
tdas Oct 11, 2016
a7e20cc
Merge remote-tracking branch 'apache-github/master' into SPARK-17731
tdas Oct 11, 2016
af5854c
Fixed style
tdas Oct 11, 2016
3d7c71a
Fixed more issues
tdas Oct 11, 2016
8b4bce8
One more comment
tdas Oct 11, 2016
35bf508
Fixed python 3 tests
tdas Oct 12, 2016
4c08d56
Used conf correctly
tdas Oct 12, 2016
38ac35e
Addressed some comments
tdas Oct 12, 2016
42bc7bf
Addressed comments
tdas Oct 12, 2016
00a7415
Fixed test
tdas Oct 13, 2016
cafbeb7
Addressed more comments
tdas Oct 13, 2016
49da4b1
Fixed trigger details bug
tdas Oct 13, 2016
839d402
Removed unnecessary imports
tdas Oct 13, 2016
5972212
Updated python docs
tdas Oct 13, 2016
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -264,6 +264,33 @@ class KafkaSourceSuite extends KafkaSourceTest {
testUnsupportedConfig("kafka.auto.offset.reset", "latest")
}

test("input row metrics") {
val topic = newTopic()
testUtils.createTopic(topic, partitions = 5)
testUtils.sendMessages(topic, Array("-1"))
require(testUtils.getLatestOffsets(Set(topic)).size === 5)

val kafka = spark
.readStream
.format("kafka")
.option("subscribe", topic)
.option("kafka.bootstrap.servers", testUtils.brokerAddress)
.load()
.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)")
.as[(String, String)]

val mapped = kafka.map(kv => kv._2.toInt + 1)
testStream(mapped)(
makeSureGetOffsetCalled,
AddKafkaData(Set(topic), 1, 2, 3),
CheckAnswer(2, 3, 4),
AssertOnLastQueryStatus { status =>
assert(status.triggerDetails.get("numRows.input.total").toInt > 0)
assert(status.sourceStatuses(0).processingRate > 0.0)
}
)
}

private def newTopic(): String = s"topic-${topicId.getAndIncrement()}"

private def testFromLatestOffsets(topic: String, options: (String, String)*): Unit = {
Expand Down
13 changes: 13 additions & 0 deletions project/MimaExcludes.scala
Original file line number Diff line number Diff line change
Expand Up @@ -56,6 +56,19 @@ object MimaExcludes {
ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.databaseExists"),
ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.tableExists"),
ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.functionExists"),

// [SPARK-17731][SQL][Streaming] Metrics for structured streaming
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.SourceStatus.this"),
ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.streaming.SourceStatus.offsetDesc"),
ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQuery.status"),
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.SinkStatus.this"),
ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.streaming.StreamingQueryInfo"),
ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryStarted.this"),
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryStarted.queryInfo"),
ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryProgress.this"),
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryProgress.queryInfo"),
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryTerminated.queryInfo"),

// [SPARK-17338][SQL] add global temp view
ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.dropGlobalTempView"),
ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.catalog.Catalog.dropTempView"),
Expand Down
301 changes: 301 additions & 0 deletions python/pyspark/sql/streaming.py
Original file line number Diff line number Diff line change
Expand Up @@ -189,6 +189,304 @@ def resetTerminated(self):
self._jsqm.resetTerminated()


class StreamingQueryStatus(object):
"""A class used to report information about the progress of a StreamingQuery.
.. note:: Experimental
.. versionadded:: 2.1
"""

def __init__(self, jsqs):
self._jsqs = jsqs

def __str__(self):
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

thanks for adding this!

"""
Pretty string of this query status.
>>> print(sqs)
StreamingQueryStatus:
Query name: query
Query id: 1
Status timestamp: 123
Input rate: 15.5 rows/sec
Processing rate 23.5 rows/sec
Latency: 345.0 ms
Trigger details:
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
Input rate: 15.5 rows/sec
Processing rate: 23.5 rows/sec
Trigger details:
numRows.input.source: 100
latency.getOffset.source: 10
latency.getBatch.source: 20
Sink status: MySink
Committed offsets: [#1, -]
"""
return self._jsqs.toString()

@property
@ignore_unicode_prefix
@since(2.1)
def name(self):
"""
Name of the query. This name is unique across all active queries.
>>> sqs.name
u'query'
"""
return self._jsqs.name()

@property
@since(2.1)
def id(self):
"""
Id of the query. This id is unique across all queries that have been started in
the current process.
>>> int(sqs.id)
1
"""
return self._jsqs.id()

@property
@since(2.1)
def timestamp(self):
"""
Timestamp (ms) of when this query was generated.
>>> int(sqs.timestamp)
123
"""
return self._jsqs.timestamp()

@property
@since(2.1)
def inputRate(self):
"""
Current total rate (rows/sec) at which data is being generated by all the sources.
>>> sqs.inputRate
15.5
"""
return self._jsqs.inputRate()

@property
@since(2.1)
def processingRate(self):
"""
Current rate (rows/sec) at which the query is processing data from all the sources.
>>> sqs.processingRate
23.5
"""
return self._jsqs.processingRate()

@property
@since(2.1)
def latency(self):
"""
Current average latency between the data being available in source and the sink
writing the corresponding output.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

When will this return None? Can you please document?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good point.

>>> sqs.latency
345.0
"""
if (self._jsqs.latency().nonEmpty()):
return self._jsqs.latency().get()
else:
return None

@property
@ignore_unicode_prefix
@since(2.1)
def sourceStatuses(self):
"""
Current statuses of the sources as a list.
>>> len(sqs.sourceStatuses)
1
>>> sqs.sourceStatuses[0].description
u'MySource1'
"""
return [SourceStatus(ss) for ss in self._jsqs.sourceStatuses()]

@property
@ignore_unicode_prefix
@since(2.1)
def sinkStatus(self):
"""
Current status of the sink.
>>> sqs.sinkStatus.description
u'MySink'
"""
return SinkStatus(self._jsqs.sinkStatus())

@property
@ignore_unicode_prefix
@since(2.1)
def triggerDetails(self):
"""
Low-level details of the currently active trigger (e.g. number of rows processed
in trigger, latency of intermediate steps, etc.).
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'isDataPresentInTrigger': u'true'}
"""
return self._jsqs.triggerDetails()


class SourceStatus(object):
"""
Status and metrics of a streaming Source.
.. note:: Experimental
.. versionadded:: 2.1
"""

def __init__(self, jss):
self._jss = jss

def __str__(self):
"""
Pretty string of this source status.
>>> print(sqs.sourceStatuses[0])
SourceStatus: MySource1
Available offset: #0
Input rate: 15.5 rows/sec
Processing rate: 23.5 rows/sec
Trigger details:
numRows.input.source: 100
latency.getOffset.source: 10
latency.getBatch.source: 20
"""
return self._jss.toString()

@property
@ignore_unicode_prefix
@since(2.1)
def description(self):
"""
Description of the source corresponding to this status.
>>> sqs.sourceStatuses[0].description
u'MySource1'
"""
return self._jss.description()

@property
@ignore_unicode_prefix
@since(2.1)
def offsetDesc(self):
"""
Description of the current offset if known.
>>> sqs.sourceStatuses[0].offsetDesc
u'#0'
"""
return self._jss.offsetDesc()

@property
@since(2.1)
def inputRate(self):
"""
Current rate (rows/sec) at which data is being generated by the source.
>>> sqs.sourceStatuses[0].inputRate
15.5
"""
return self._jss.inputRate()

@property
@since(2.1)
def processingRate(self):
"""
Current rate (rows/sec) at which the query is processing data from the source.
>>> sqs.sourceStatuses[0].processingRate
23.5
"""
return self._jss.processingRate()

@property
@ignore_unicode_prefix
@since(2.1)
def triggerDetails(self):
"""
Low-level details of the currently active trigger (e.g. number of rows processed
in trigger, latency of intermediate steps, etc.).
If no trigger is currently active, then it will have details of the last completed trigger.
>>> sqs.sourceStatuses[0].triggerDetails
{u'numRows.input.source': u'100', u'latency.getOffset.source': u'10',
u'latency.getBatch.source': u'20'}
"""
return self._jss.triggerDetails()


class SinkStatus(object):
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

No OutputRate? (Though you did mention in the comments that it doesn't work with whole-stage codegen.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yeah. no output rate.

"""
Status and metrics of a streaming Sink.
.. note:: Experimental
.. versionadded:: 2.1
"""

def __init__(self, jss):
self._jss = jss

def __str__(self):
"""
Pretty string of this source status.
>>> print(sqs.sinkStatus)
SinkStatus: MySink
Committed offsets: [#1, -]
"""
return self._jss.toString()

@property
@ignore_unicode_prefix
@since(2.1)
def description(self):
"""
Description of the source corresponding to this status.
>>> sqs.sinkStatus.description
u'MySink'
"""
return self._jss.description()

@property
@ignore_unicode_prefix
@since(2.1)
def offsetDesc(self):
"""
Description of the current offsets up to which data has been written by the sink.
>>> sqs.sinkStatus.offsetDesc
u'[#1, -]'
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

what does this offset descriptor supposed to mean?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

what does the - stand for?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Isnt it "up to which data has been written by the sink" clear enough?

"""
return self._jss.offsetDesc()


class Trigger(object):
"""Used to indicate how often results should be produced by a :class:`StreamingQuery`.
Expand Down Expand Up @@ -753,11 +1051,14 @@ def _test():
globs['sdf_schema'] = StructType([StructField("data", StringType(), False)])
globs['df'] = \
globs['spark'].readStream.format('text').load('python/test_support/sql/streaming')
globs['sqs'] = StreamingQueryStatus(
spark.sparkContext._jvm.org.apache.spark.sql.streaming.StreamingQueryStatus.testStatus())

(failure_count, test_count) = doctest.testmod(
pyspark.sql.streaming, globs=globs,
optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE | doctest.REPORT_NDIFF)
globs['spark'].stop()

if failure_count:
exit(-1)

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -164,6 +164,13 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product {
ret
}

/**
* Returns a Seq containing the leaves in this tree.
*/
def collectLeaves(): Seq[BaseType] = {
this.collect { case p if p.children.isEmpty => p }
}

/**
* Finds and returns the first [[TreeNode]] of the tree for which the given partial function
* is defined (pre-order), and applies the partial function to it.
Expand Down
Loading