GitHub user uncleGen opened a pull request:
https://github.com/apache/spark/pull/17025
[SPARK-19690][SS] Join a streaming DataFrame with a batch DataFrame which
has an aggregation may not work
## What changes were proposed in this pull request?
`StatefulAggregationStrategy` should check logicplan is streaming or not
Test code:
```
case class Record(key: Int, value: String)
val df = spark.createDataFrame((1 to 100).map(i => Record(i,
s"value_$i"))).groupBy("value").count
val lines = spark.readStream.format("socket").option("host",
"localhost").option("port", "9999").load
val words = lines.as[String].flatMap(_.split(" "))
val wordCounts = words.join(df, "value")
```
before pr:
```
== Physical Plan ==
*Project [value#13, count#19L]
+- *BroadcastHashJoin [value#13], [value#1], Inner, BuildRight
:- *Filter isnotnull(value#13)
: +- *SerializeFromObject [staticinvoke(class
org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0,
java.lang.String, true], true) AS value#13]
: +- MapPartitions <function1>, obj#12: java.lang.String
: +- DeserializeToObject value#5.toString, obj#11:
java.lang.String
: +- StreamingRelation textSocket, [value#5]
+- BroadcastExchange HashedRelationBroadcastMode(List(input[0, string,
true]))
+- *HashAggregate(keys=[value#1], functions=[count(1)])
+- StateStoreSave [value#1], OperatorStateId(<unknown>,0,0),
Append, 0
+- *HashAggregate(keys=[value#1], functions=[merge_count(1)])
+- StateStoreRestore [value#1],
OperatorStateId(<unknown>,0,0)
+- *HashAggregate(keys=[value#1],
functions=[merge_count(1)])
+- Exchange hashpartitioning(value#1, 200)
+- *HashAggregate(keys=[value#1],
functions=[partial_count(1)])
+- *Project [value#1]
+- *Filter isnotnull(value#1)
+- LocalTableScan [key#0, value#1]
```
after pr:
```
== Physical Plan ==
*Project [value#13, count#19L]
+- *BroadcastHashJoin [value#13], [value#1], Inner, BuildRight
:- *Filter isnotnull(value#13)
: +- *SerializeFromObject [staticinvoke(class
org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0,
java.lang.String, true], true) AS value#13]
: +- MapPartitions <function1>, obj#12: java.lang.String
: +- DeserializeToObject value#5.toString, obj#11:
java.lang.String
: +- StreamingRelation textSocket, [value#5]
+- BroadcastExchange HashedRelationBroadcastMode(List(input[0, string,
true]))
+- *HashAggregate(keys=[value#1], functions=[count(1)])
+- Exchange hashpartitioning(value#1, 200)
+- *HashAggregate(keys=[value#1], functions=[partial_count(1)])
+- *Project [value#1]
+- *Filter isnotnull(value#1)
+- LocalTableScan [key#0, value#1]
```
## How was this patch tested?
add new unit test.
You can merge this pull request into a Git repository by running:
$ git pull https://github.com/uncleGen/spark SPARK-19690
Alternatively you can review and apply these changes as the patch at:
https://github.com/apache/spark/pull/17025.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 #17025
----
commit 0d0c19a2f99b6417f419f72fb28c155a70dda201
Author: uncleGen <[email protected]>
Date: 2017-02-22T10:18:31Z
Join a streaming DataFrame with a batch DataFrame which has an aggregation
may not work
----
---
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 [email protected] or file a JIRA ticket
with INFRA.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]