[spark] branch master updated: [SPARK-41040][SS] Fix self-union streaming query failure when using readStream.table

2022-11-08 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 7074e4fee7e [SPARK-41040][SS] Fix self-union streaming query failure 
when using readStream.table
7074e4fee7e is described below

commit 7074e4fee7e6944013cfaa3c0c2a1458cce8a72d
Author: Shixiong Zhu 
AuthorDate: Tue Nov 8 08:31:24 2022 -0800

[SPARK-41040][SS] Fix self-union streaming query failure when using 
readStream.table

### What changes were proposed in this pull request?

https://github.com/apache/spark/pull/36963 added a check to disallow any 
source setting `CatalogTable` in the batch plan. However, this check is not 
safe to enforce:

- In a self-union query, the batch plan created by the source will be 
shared by multiple nodes in the plan. When we transform the plan, the batch 
plan will be visited multiple times. Hence, the first visit will set the 
`CatalogTable` and the second visit will try to set it again and fail the query.
- A source built by arbitrary developers can set `CatalogTable` in the 
batch plan. We should not fail as it would break an existing source.

This PR fixes the issue by removing the check and set `CatalogTable` only 
if the batch plan doesn't have one.

### Why are the changes needed?

Fix a bug in master.

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

The new added unit test

Closes #38553 from zsxwing/SPARK-41040.

Authored-by: Shixiong Zhu 
Signed-off-by: Shixiong Zhu 
---
 .../sql/execution/streaming/MicroBatchExecution.scala  | 18 +++---
 .../sql/streaming/test/DataStreamTableAPISuite.scala   | 13 +
 2 files changed, 28 insertions(+), 3 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala
index 7ed19b35114..051e45c71e6 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala
@@ -621,10 +621,22 @@ class MicroBatchExecution(
   if (hasFileMetadata) {
 newRelation = newRelation.withMetadataColumns()
   }
-  catalogTable.foreach { table =>
-assert(newRelation.catalogTable.isEmpty,
+  // If the catalog table is not set in the batch plan generated 
by the source, we will
+  // pick up the one from `StreamingExecutionRelation`. Otherwise, 
we will skip this
+  // step. The skipping can happen in the following cases:
+  // - We re-visit the same `StreamingExecutionRelation`. For 
example, self-union will
+  //   share the same `StreamingExecutionRelation` and `transform` 
will visit it twice.
+  //   This is safe to skip.
+  // - A source that sets the catalog table explicitly. We will 
pick up the one provided
+  //   by the source directly to maintain the same behavior.
+  if (newRelation.catalogTable.isEmpty) {
+catalogTable.foreach { table =>
+  newRelation = newRelation.copy(catalogTable = Some(table))
+}
+  } else if (catalogTable.exists(_ ne 
newRelation.catalogTable.get)) {
+// Output a warning if `catalogTable` is provided by the 
source rather than engine
+logWarning(
   s"Source $source should not produce the information of 
catalog table by its own.")
-newRelation = newRelation.copy(catalogTable = Some(table))
   }
   newRelation
   }
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala
index 0d1242fbb19..6bbf2239dbf 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala
@@ -484,6 +484,19 @@ class DataStreamTableAPISuite extends StreamTest with 
BeforeAndAfter {
 }
   }
 
+  test("SPARK-41040: self-union using readStream.table should not fail") {
+withTable("self_union_table") {
+  spark.range(10).write.format("parquet").saveAsTable("self_union_table")
+  val df = spark.readStream.format("parquet").table("self_union_table")
+  val q = df.union(df).writeStream.format("noop").start()
+  try {
+q.processAll

[spark] branch master updated: [SPARK-41045][SQL] Pre-compute to eliminate ScalaReflection calls after deserializer is created

2022-11-08 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new ef402edff91 [SPARK-41045][SQL] Pre-compute to eliminate 
ScalaReflection calls after deserializer is created
ef402edff91 is described below

commit ef402edff91377d37c0c1b8d40921ed7bd9f7160
Author: Shixiong Zhu 
AuthorDate: Tue Nov 8 08:18:50 2022 -0800

[SPARK-41045][SQL] Pre-compute to eliminate ScalaReflection calls after 
deserializer is created

### What changes were proposed in this pull request?

Currently when `ScalaReflection` returns a deserializer, for a few complex 
types, such as array, map, udt, etc, it creates functions that may still touch 
`ScalaReflection` after the deserializer is created.

`ScalaReflection` is a performance bottleneck for multiple threads as it 
holds multiple global locks. We can refactor `ScalaReflection.deserializerFor` 
to pre-compute everything that needs to touch `ScalaReflection` before creating 
the deserializer. After this, once the deserializer is created, it can be 
reused by multiple threads without touching `ScalaReflection.deserializerFor` 
any more and it will be much faster.

### Why are the changes needed?

Optimize `ScalaReflection.deserializerFor` to make deserializers faster 
under multiple threads.

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

This is refactoring `deserializerFor` to optimize the code. Existing tests 
should already cover the correctness.

Closes #38556 from zsxwing/scala-ref.

Authored-by: Shixiong Zhu 
Signed-off-by: Shixiong Zhu 
---
 .../sql/catalyst/DeserializerBuildHelper.scala |   5 +-
 .../spark/sql/catalyst/JavaTypeInference.scala |   8 +-
 .../spark/sql/catalyst/ScalaReflection.scala   | 157 +++--
 3 files changed, 85 insertions(+), 85 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/DeserializerBuildHelper.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/DeserializerBuildHelper.scala
index 0d3b9977e4f..7051c2d2264 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/DeserializerBuildHelper.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/DeserializerBuildHelper.scala
@@ -49,10 +49,9 @@ object DeserializerBuildHelper {
   dataType: DataType,
   nullable: Boolean,
   walkedTypePath: WalkedTypePath,
-  funcForCreatingDeserializer: (Expression, WalkedTypePath) => 
Expression): Expression = {
+  funcForCreatingDeserializer: Expression => Expression): Expression = {
 val casted = upCastToExpectedType(expr, dataType, walkedTypePath)
-expressionWithNullSafety(funcForCreatingDeserializer(casted, 
walkedTypePath),
-  nullable, walkedTypePath)
+expressionWithNullSafety(funcForCreatingDeserializer(casted), nullable, 
walkedTypePath)
   }
 
   def expressionWithNullSafety(
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala
index dccaf1c4835..827807055ce 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala
@@ -218,9 +218,7 @@ object JavaTypeInference {
 
 // Assumes we are deserializing the first column of a row.
 deserializerForWithNullSafetyAndUpcast(GetColumnByOrdinal(0, dataType), 
dataType,
-  nullable = nullable, walkedTypePath, (casted, walkedTypePath) => {
-deserializerFor(typeToken, casted, walkedTypePath)
-  })
+  nullable = nullable, walkedTypePath, deserializerFor(typeToken, _, 
walkedTypePath))
   }
 
   private def deserializerFor(
@@ -280,7 +278,7 @@ object JavaTypeInference {
 dataType,
 nullable = elementNullable,
 newTypePath,
-(casted, typePath) => deserializerFor(typeToken.getComponentType, 
casted, typePath))
+deserializerFor(typeToken.getComponentType, _, newTypePath))
 }
 
 val arrayData = UnresolvedMapObjects(mapFunction, path)
@@ -309,7 +307,7 @@ object JavaTypeInference {
 dataType,
 nullable = elementNullable,
 newTypePath,
-(casted, typePath) => deserializerFor(et, casted, typePath))
+deserializerFor(et, _, newTypePath))
 }
 
 UnresolvedMapObjects(mapFunction, path, customCollectionCls = Some(c))
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala
index 12093b9f4b2..d895a0fbe19 100644
--- 
a/s

[spark] branch master updated: [SPARK-32896][SS][FOLLOW-UP] Rename the API to `toTable`

2020-12-02 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 878cc0e  [SPARK-32896][SS][FOLLOW-UP] Rename the API to `toTable`
878cc0e is described below

commit 878cc0e6e95f300a0a58c742654f53a28b30b174
Author: Yuanjian Li 
AuthorDate: Wed Dec 2 17:36:25 2020 -0800

[SPARK-32896][SS][FOLLOW-UP] Rename the API to `toTable`

### What changes were proposed in this pull request?
As the discussion in 
https://github.com/apache/spark/pull/30521#discussion_r531463427, rename the 
API to `toTable`.

### Why are the changes needed?
Rename the API for further extension and accuracy.

### Does this PR introduce _any_ user-facing change?
Yes, it's an API change but the new API is not released yet.

### How was this patch tested?
Existing UT.

Closes #30571 from xuanyuanking/SPARK-32896-follow.

Authored-by: Yuanjian Li 
Signed-off-by: Shixiong Zhu 
---
 .../main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala| 2 +-
 .../org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala   | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
index d67e175..9e35997 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
@@ -304,7 +304,7 @@ final class DataStreamWriter[T] private[sql](ds: 
Dataset[T]) {
* @since 3.1.0
*/
   @throws[TimeoutException]
-  def saveAsTable(tableName: String): StreamingQuery = {
+  def toTable(tableName: String): StreamingQuery = {
 this.source = SOURCE_NAME_TABLE
 this.tableName = tableName
 startInternal(None)
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala
index 062b106..bf85043 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala
@@ -291,7 +291,7 @@ class DataStreamTableAPISuite extends StreamTest with 
BeforeAndAfter {
 val query = inputDF
   .writeStream
   .option("checkpointLocation", checkpointDir.getAbsolutePath)
-  .saveAsTable(tableIdentifier)
+  .toTable(tableIdentifier)
 
 inputData.addData(newInputs: _*)
 


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



[spark] branch master updated: [SPARK-32896][SS][FOLLOW-UP] Rename the API to `toTable`

2020-12-02 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 6fa797e  [SPARK-32896][SS][FOLLOW-UP] Rename the API to `toTable`
6fa797e is described below

commit 6fa797e977412d071dd4dc079053ec64a21b3041
Author: Yuanjian Li 
AuthorDate: Wed Dec 2 17:31:10 2020 -0800

[SPARK-32896][SS][FOLLOW-UP] Rename the API to `toTable`

### What changes were proposed in this pull request?
As the discussion in 
https://github.com/apache/spark/pull/30521#discussion_r531463427, rename the 
API to `toTable`.

### Why are the changes needed?
Rename the API for further extension and accuracy.

### Does this PR introduce _any_ user-facing change?
Yes, it's an API change but the new API is not released yet.

### How was this patch tested?
Existing UT.

Closes #30571 from xuanyuanking/SPARK-32896-follow.

Authored-by: Yuanjian Li 
Signed-off-by: Shixiong Zhu 
---
 .../main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala| 2 +-
 .../org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala   | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
index d67e175..9e35997 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
@@ -304,7 +304,7 @@ final class DataStreamWriter[T] private[sql](ds: 
Dataset[T]) {
* @since 3.1.0
*/
   @throws[TimeoutException]
-  def saveAsTable(tableName: String): StreamingQuery = {
+  def toTable(tableName: String): StreamingQuery = {
 this.source = SOURCE_NAME_TABLE
 this.tableName = tableName
 startInternal(None)
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala
index 062b106..bf85043 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala
@@ -291,7 +291,7 @@ class DataStreamTableAPISuite extends StreamTest with 
BeforeAndAfter {
 val query = inputDF
   .writeStream
   .option("checkpointLocation", checkpointDir.getAbsolutePath)
-  .saveAsTable(tableIdentifier)
+  .toTable(tableIdentifier)
 
 inputData.addData(newInputs: _*)
 


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



[spark] branch master updated: [SPARK-31953][SS] Add Spark Structured Streaming History Server Support

2020-12-02 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 4f96670  [SPARK-31953][SS] Add Spark Structured Streaming History 
Server Support
4f96670 is described below

commit 4f9667035886a67e6c9a4e8fad2efa390e87ca68
Author: uncleGen 
AuthorDate: Wed Dec 2 17:11:51 2020 -0800

[SPARK-31953][SS] Add Spark Structured Streaming History Server Support

### What changes were proposed in this pull request?

Add Spark Structured Streaming History Server Support.

### Why are the changes needed?

Add a streaming query history server plugin.


![image](https://user-images.githubusercontent.com/7402327/84248291-d26cfe80-ab3b-11ea-86d2-98205fa2bcc4.png)

![image](https://user-images.githubusercontent.com/7402327/84248347-e44ea180-ab3b-11ea-81de-eefe207656f2.png)

![image](https://user-images.githubusercontent.com/7402327/84248396-f0d2fa00-ab3b-11ea-9b0d-e410115471b0.png)

- Follow-ups
  - Query duration should not update in history UI.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Update UT.

Closes #28781 from uncleGen/SPARK-31953.

Lead-authored-by: uncleGen 
Co-authored-by: Genmao Yu 
Co-authored-by: Yuanjian Li 
Signed-off-by: Shixiong Zhu 
---
 dev/.rat-excludes  |   1 +
 .../org.apache.spark.status.AppHistoryServerPlugin |   1 +
 .../streaming/StreamingQueryListenerBus.scala  |  26 +++-
 .../ui/StreamingQueryHistoryServerPlugin.scala |  43 ++
 .../execution/ui/StreamingQueryStatusStore.scala   |  53 +++
 .../apache/spark/sql/internal/SharedState.scala|   8 +-
 .../sql/streaming/StreamingQueryManager.scala  |   3 +-
 .../sql/streaming/ui/StreamingQueryPage.scala  |  44 +++---
 .../ui/StreamingQueryStatisticsPage.scala  |  27 ++--
 .../ui/StreamingQueryStatusListener.scala  | 166 +
 .../spark/sql/streaming/ui/StreamingQueryTab.scala |   3 +-
 .../apache/spark/sql/streaming/ui/UIUtils.scala|  12 +-
 .../resources/spark-events/local-1596020211915 | 160 
 .../org/apache/spark/deploy/history/Utils.scala}   |  39 ++---
 .../streaming/ui/StreamingQueryHistorySuite.scala  |  63 
 .../sql/streaming/ui/StreamingQueryPageSuite.scala |  42 +++---
 .../ui/StreamingQueryStatusListenerSuite.scala | 159 
 17 files changed, 673 insertions(+), 177 deletions(-)

diff --git a/dev/.rat-excludes b/dev/.rat-excludes
index 7da330d..167cf22 100644
--- a/dev/.rat-excludes
+++ b/dev/.rat-excludes
@@ -123,6 +123,7 @@ SessionHandler.java
 GangliaReporter.java
 application_1578436911597_0052
 config.properties
+local-1596020211915
 app-20200706201101-0003
 py.typed
 _metadata
diff --git 
a/sql/core/src/main/resources/META-INF/services/org.apache.spark.status.AppHistoryServerPlugin
 
b/sql/core/src/main/resources/META-INF/services/org.apache.spark.status.AppHistoryServerPlugin
index 0bba2f8..6771eef 100644
--- 
a/sql/core/src/main/resources/META-INF/services/org.apache.spark.status.AppHistoryServerPlugin
+++ 
b/sql/core/src/main/resources/META-INF/services/org.apache.spark.status.AppHistoryServerPlugin
@@ -1 +1,2 @@
 org.apache.spark.sql.execution.ui.SQLHistoryServerPlugin
+org.apache.spark.sql.execution.ui.StreamingQueryHistoryServerPlugin
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingQueryListenerBus.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingQueryListenerBus.scala
index 1b8d69f..4b98acd 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingQueryListenerBus.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingQueryListenerBus.scala
@@ -31,16 +31,21 @@ import org.apache.spark.util.ListenerBus
  * Spark listener bus, so that it can receive 
[[StreamingQueryListener.Event]]s and dispatch them
  * to StreamingQueryListeners.
  *
- * Note that each bus and its registered listeners are associated with a 
single SparkSession
+ * Note 1: Each bus and its registered listeners are associated with a single 
SparkSession
  * and StreamingQueryManager. So this bus will dispatch events to registered 
listeners for only
  * those queries that were started in the associated SparkSession.
+ *
+ * Note 2: To rebuild Structured Streaming UI in SHS, this bus will be 
registered into
+ * [[org.apache.spark.scheduler.ReplayListenerBus]]. We check 
`sparkListenerBus` defined or not to
+ * determine how to process [[StreamingQueryListener.Event]]. If false, it 
means this bus is used to
+ * replay all streaming query event from eventLog.
  */
-class StreamingQueryListenerBus(sparkListenerBus: LiveListenerBus)
+class

[spark] branch branch-2.4 updated: [SPARK-31923][CORE] Ignore internal accumulators that use unrecognized types rather than crashing (branch-2.4)

2020-06-08 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch branch-2.4
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-2.4 by this push:
 new 48017cc  [SPARK-31923][CORE] Ignore internal accumulators that use 
unrecognized types rather than crashing (branch-2.4)
48017cc is described below

commit 48017cc36bdf7d84506daeed589e4cbebff269f8
Author: Shixiong Zhu 
AuthorDate: Mon Jun 8 16:52:34 2020 -0700

[SPARK-31923][CORE] Ignore internal accumulators that use unrecognized 
types rather than crashing (branch-2.4)

### What changes were proposed in this pull request?

Backport #28744 to branch-2.4.

### Why are the changes needed?

Low risky fix for branch-2.4.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

New unit tests.

Closes #28758 from zsxwing/SPARK-31923-2.4.

Authored-by: Shixiong Zhu 
Signed-off-by: Shixiong Zhu 
---
 .../scala/org/apache/spark/util/JsonProtocol.scala | 20 ++---
 .../org/apache/spark/util/JsonProtocolSuite.scala  | 47 ++
 2 files changed, 62 insertions(+), 5 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala 
b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala
index 50c6461..0e613ce 100644
--- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala
+++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala
@@ -326,12 +326,22 @@ private[spark] object JsonProtocol {
 case v: Long => JInt(v)
 // We only have 3 kind of internal accumulator types, so if it's not 
int or long, it must be
 // the blocks accumulator, whose type is `java.util.List[(BlockId, 
BlockStatus)]`
-case v =>
-  JArray(v.asInstanceOf[java.util.List[(BlockId, 
BlockStatus)]].asScala.toList.map {
-case (id, status) =>
-  ("Block ID" -> id.toString) ~
-  ("Status" -> blockStatusToJson(status))
+case v: java.util.List[_] =>
+  JArray(v.asScala.toList.flatMap {
+case (id: BlockId, status: BlockStatus) =>
+  Some(
+("Block ID" -> id.toString) ~
+("Status" -> blockStatusToJson(status))
+  )
+case _ =>
+  // Ignore unsupported types. A user may put `METRICS_PREFIX` in 
the name. We should
+  // not crash.
+  None
   })
+case _ =>
+  // Ignore unsupported types. A user may put `METRICS_PREFIX` in the 
name. We should not
+  // crash.
+  JNothing
   }
 } else {
   // For all external accumulators, just use strings
diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala 
b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala
index 74b72d9..40fb2e3 100644
--- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala
@@ -436,6 +436,53 @@ class JsonProtocolSuite extends SparkFunSuite {
 testAccumValue(Some("anything"), 123, JString("123"))
   }
 
+  /** Create an AccumulableInfo and verify we can serialize and deserialize 
it. */
+  private def testAccumulableInfo(
+  name: String,
+  value: Option[Any],
+  expectedValue: Option[Any]): Unit = {
+val isInternal = name.startsWith(InternalAccumulator.METRICS_PREFIX)
+val accum = AccumulableInfo(
+  123L,
+  Some(name),
+  update = value,
+  value = value,
+  internal = isInternal,
+  countFailedValues = false)
+val json = JsonProtocol.accumulableInfoToJson(accum)
+val newAccum = JsonProtocol.accumulableInfoFromJson(json)
+assert(newAccum == accum.copy(update = expectedValue, value = 
expectedValue))
+  }
+
+  test("SPARK-31923: unexpected value type of internal accumulator") {
+// Because a user may use `METRICS_PREFIX` in an accumulator name, we 
should test unexpected
+// types to make sure we don't crash.
+import InternalAccumulator.METRICS_PREFIX
+testAccumulableInfo(
+  METRICS_PREFIX + "fooString",
+  value = Some("foo"),
+  expectedValue = None)
+testAccumulableInfo(
+  METRICS_PREFIX + "fooList",
+  value = Some(java.util.Arrays.asList("string")),
+  expectedValue = Some(java.util.Collections.emptyList())
+)
+val blocks = Seq(
+  (TestBlockId("block1"), BlockStatus(StorageLevel.MEMORY_ONLY, 1L, 2L)),
+  (TestBlockId("block2"), BlockStatus(StorageLevel.DISK_ONLY, 3L, 4L)))
+testAccumulableInfo(
+  METRICS_PREFIX + "fooList",
+  value = Some(java.util.Arrays.asList(
+&

[spark] branch branch-3.0 updated: [SPARK-31923][CORE] Ignore internal accumulators that use unrecognized types rather than crashing

2020-06-08 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch branch-3.0
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.0 by this push:
 new b00ac30  [SPARK-31923][CORE] Ignore internal accumulators that use 
unrecognized types rather than crashing
b00ac30 is described below

commit b00ac30dfb621962e5b39c52a3bb09440936a0ff
Author: Shixiong Zhu 
AuthorDate: Mon Jun 8 12:06:17 2020 -0700

[SPARK-31923][CORE] Ignore internal accumulators that use unrecognized 
types rather than crashing

### What changes were proposed in this pull request?

Ignore internal accumulators that use unrecognized types rather than 
crashing so that an event log containing such accumulators can still be 
converted to JSON and logged.

### Why are the changes needed?

A user may use internal accumulators by adding the `internal.metrics.` 
prefix to the accumulator name to hide sensitive information from UI 
(Accumulators except internal ones will be shown in Spark UI).

However, `org.apache.spark.util.JsonProtocol.accumValueToJson` assumes an 
internal accumulator has only 3 possible types: `int`, `long`, and 
`java.util.List[(BlockId, BlockStatus)]`. When an internal accumulator uses an 
unexpected type, it will crash.

An event log that contains such accumulator will be dropped because it 
cannot be converted to JSON, and it will cause weird UI issue when rendering in 
Spark History Server. For example, if `SparkListenerTaskEnd` is dropped because 
of this issue, the user will see the task is still running even if it was 
finished.

It's better to make `accumValueToJson` more robust because it's up to the 
user to pick up the accumulator name.

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

The new unit tests.

Closes #28744 from zsxwing/fix-internal-accum.

Authored-by: Shixiong Zhu 
Signed-off-by: Shixiong Zhu 
(cherry picked from commit b333ed0c4a5733a9c36ad79de1d4c13c6cf3c5d4)
Signed-off-by: Shixiong Zhu 
---
 .../scala/org/apache/spark/util/JsonProtocol.scala | 20 ++---
 .../org/apache/spark/util/JsonProtocolSuite.scala  | 48 ++
 2 files changed, 63 insertions(+), 5 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala 
b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala
index f445fd4..d53ca0f 100644
--- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala
+++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala
@@ -351,12 +351,22 @@ private[spark] object JsonProtocol {
 case v: Long => JInt(v)
 // We only have 3 kind of internal accumulator types, so if it's not 
int or long, it must be
 // the blocks accumulator, whose type is `java.util.List[(BlockId, 
BlockStatus)]`
-case v =>
-  JArray(v.asInstanceOf[java.util.List[(BlockId, 
BlockStatus)]].asScala.toList.map {
-case (id, status) =>
-  ("Block ID" -> id.toString) ~
-  ("Status" -> blockStatusToJson(status))
+case v: java.util.List[_] =>
+  JArray(v.asScala.toList.flatMap {
+case (id: BlockId, status: BlockStatus) =>
+  Some(
+("Block ID" -> id.toString) ~
+("Status" -> blockStatusToJson(status))
+  )
+case _ =>
+  // Ignore unsupported types. A user may put `METRICS_PREFIX` in 
the name. We should
+  // not crash.
+  None
   })
+case _ =>
+  // Ignore unsupported types. A user may put `METRICS_PREFIX` in the 
name. We should not
+  // crash.
+  JNothing
   }
 } else {
   // For all external accumulators, just use strings
diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala 
b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala
index d1f09d8..5f1c753 100644
--- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala
@@ -482,6 +482,54 @@ class JsonProtocolSuite extends SparkFunSuite {
 testAccumValue(Some("anything"), 123, JString("123"))
   }
 
+  /** Create an AccumulableInfo and verify we can serialize and deserialize 
it. */
+  private def testAccumulableInfo(
+  name: String,
+  value: Option[Any],
+  expectedValue: Option[Any]): Unit = {
+val isInternal = name.startsWith(InternalAccumulator.METRICS_PREFIX)
+val accum = AccumulableInfo(
+  123L,
+  Some(name),
+  update = value,
+  value = value,
+  internal = isInternal,
+  countFailedValues = false)
+val json = JsonProtocol.accumulabl

[spark] branch master updated: [SPARK-31923][CORE] Ignore internal accumulators that use unrecognized types rather than crashing

2020-06-08 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new b333ed0  [SPARK-31923][CORE] Ignore internal accumulators that use 
unrecognized types rather than crashing
b333ed0 is described below

commit b333ed0c4a5733a9c36ad79de1d4c13c6cf3c5d4
Author: Shixiong Zhu 
AuthorDate: Mon Jun 8 12:06:17 2020 -0700

[SPARK-31923][CORE] Ignore internal accumulators that use unrecognized 
types rather than crashing

### What changes were proposed in this pull request?

Ignore internal accumulators that use unrecognized types rather than 
crashing so that an event log containing such accumulators can still be 
converted to JSON and logged.

### Why are the changes needed?

A user may use internal accumulators by adding the `internal.metrics.` 
prefix to the accumulator name to hide sensitive information from UI 
(Accumulators except internal ones will be shown in Spark UI).

However, `org.apache.spark.util.JsonProtocol.accumValueToJson` assumes an 
internal accumulator has only 3 possible types: `int`, `long`, and 
`java.util.List[(BlockId, BlockStatus)]`. When an internal accumulator uses an 
unexpected type, it will crash.

An event log that contains such accumulator will be dropped because it 
cannot be converted to JSON, and it will cause weird UI issue when rendering in 
Spark History Server. For example, if `SparkListenerTaskEnd` is dropped because 
of this issue, the user will see the task is still running even if it was 
finished.

It's better to make `accumValueToJson` more robust because it's up to the 
user to pick up the accumulator name.

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

The new unit tests.

Closes #28744 from zsxwing/fix-internal-accum.

Authored-by: Shixiong Zhu 
Signed-off-by: Shixiong Zhu 
---
 .../scala/org/apache/spark/util/JsonProtocol.scala | 20 ++---
 .../org/apache/spark/util/JsonProtocolSuite.scala  | 48 ++
 2 files changed, 63 insertions(+), 5 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala 
b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala
index 844d9b7..1c788a3 100644
--- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala
+++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala
@@ -363,12 +363,22 @@ private[spark] object JsonProtocol {
 case v: Long => JInt(v)
 // We only have 3 kind of internal accumulator types, so if it's not 
int or long, it must be
 // the blocks accumulator, whose type is `java.util.List[(BlockId, 
BlockStatus)]`
-case v =>
-  JArray(v.asInstanceOf[java.util.List[(BlockId, 
BlockStatus)]].asScala.toList.map {
-case (id, status) =>
-  ("Block ID" -> id.toString) ~
-  ("Status" -> blockStatusToJson(status))
+case v: java.util.List[_] =>
+  JArray(v.asScala.toList.flatMap {
+case (id: BlockId, status: BlockStatus) =>
+  Some(
+("Block ID" -> id.toString) ~
+("Status" -> blockStatusToJson(status))
+  )
+case _ =>
+  // Ignore unsupported types. A user may put `METRICS_PREFIX` in 
the name. We should
+  // not crash.
+  None
   })
+case _ =>
+  // Ignore unsupported types. A user may put `METRICS_PREFIX` in the 
name. We should not
+  // crash.
+  JNothing
   }
 } else {
   // For all external accumulators, just use strings
diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala 
b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala
index 248142a..5a4073b 100644
--- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala
@@ -507,6 +507,54 @@ class JsonProtocolSuite extends SparkFunSuite {
 testAccumValue(Some("anything"), 123, JString("123"))
   }
 
+  /** Create an AccumulableInfo and verify we can serialize and deserialize 
it. */
+  private def testAccumulableInfo(
+  name: String,
+  value: Option[Any],
+  expectedValue: Option[Any]): Unit = {
+val isInternal = name.startsWith(InternalAccumulator.METRICS_PREFIX)
+val accum = AccumulableInfo(
+  123L,
+  Some(name),
+  update = value,
+  value = value,
+  internal = isInternal,
+  countFailedValues = false)
+val json = JsonProtocol.accumulableInfoToJson(accum)
+val newAccum = JsonProtocol.accumulableInfoFromJson(json)
+assert(newAccum == accum.

[spark] branch master updated: [SPARK-30915][SS] CompactibleFileStreamLog: Avoid reading the metadata log file when finding the latest batch ID

2020-05-22 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 5a258b0  [SPARK-30915][SS] CompactibleFileStreamLog: Avoid reading the 
metadata log file when finding the latest batch ID
5a258b0 is described below

commit 5a258b0b67ee7c97a90d8b719c7a2171707c9244
Author: Jungtaek Lim (HeartSaVioR) 
AuthorDate: Fri May 22 16:46:17 2020 -0700

[SPARK-30915][SS] CompactibleFileStreamLog: Avoid reading the metadata log 
file when finding the latest batch ID

### What changes were proposed in this pull request?

This patch adds the new method `getLatestBatchId()` in 
CompactibleFileStreamLog in complement of getLatest() which doesn't read the 
content of the latest batch metadata log file, and apply to both 
FileStreamSource and FileStreamSink to avoid unnecessary latency on reading log 
file.

### Why are the changes needed?

Once compacted metadata log file becomes huge, writing outputs for the 
compact + 1 batch is also affected due to unnecessarily reading the compacted 
metadata log file. This unnecessary latency can be simply avoided.

### Does this PR introduce any user-facing change?

No.

### How was this patch tested?

New UT. Also manually tested under query which has huge metadata log on 
file stream sink:

> before applying the patch

![Screen Shot 2020-02-21 at 4 20 19 
PM](https://user-images.githubusercontent.com/1317309/75016223-d3ffb180-54cd-11ea-9063-49405943049d.png)

> after applying the patch

![Screen Shot 2020-02-21 at 4 06 18 
PM](https://user-images.githubusercontent.com/1317309/75016220-d235ee00-54cd-11ea-81a7-7c03a43c4db4.png)

Peaks are compact batches - please compare the next batch after compact 
batches, especially the area of "light brown".

Closes #27664 from HeartSaVioR/SPARK-30915.

Authored-by: Jungtaek Lim (HeartSaVioR) 
Signed-off-by: Shixiong Zhu 
---
 .../streaming/CompactibleFileStreamLog.scala   |  2 +-
 .../sql/execution/streaming/FileStreamSink.scala   |  2 +-
 .../execution/streaming/FileStreamSourceLog.scala  |  2 +-
 .../sql/execution/streaming/HDFSMetadataLog.scala  | 23 --
 .../streaming/FileStreamSinkLogSuite.scala | 83 ++
 5 files changed, 102 insertions(+), 10 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLog.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLog.scala
index 10bcfe6..e8ae0ea 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLog.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLog.scala
@@ -213,7 +213,7 @@ abstract class CompactibleFileStreamLog[T <: AnyRef : 
ClassTag](
* Returns all files except the deleted ones.
*/
   def allFiles(): Array[T] = {
-var latestId = getLatest().map(_._1).getOrElse(-1L)
+var latestId = getLatestBatchId().getOrElse(-1L)
 // There is a race condition when `FileStreamSink` is deleting old files 
and `StreamFileIndex`
 // is calling this method. This loop will retry the reading to deal with 
the
 // race condition.
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala
index b679f16..3224547 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala
@@ -142,7 +142,7 @@ class FileStreamSink(
   }
 
   override def addBatch(batchId: Long, data: DataFrame): Unit = {
-if (batchId <= fileLog.getLatest().map(_._1).getOrElse(-1L)) {
+if (batchId <= fileLog.getLatestBatchId().getOrElse(-1L)) {
   logInfo(s"Skipping already committed batch $batchId")
 } else {
   val committer = FileCommitProtocol.instantiate(
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceLog.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceLog.scala
index 7b2ea96..c438877 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceLog.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceLog.scala
@@ -96,7 +96,7 @@ class FileStreamSourceLog(
 val searchKeys = removedBatches.map(_._1)
 val retrievedBatches = if (searchKeys.nonEmpty) {
   logWarning(s"Get batches from removed files, this is unexpected in the 
current code path!!!")
-  val latestBatchId = getLatest().map(

[spark] branch branch-3.0 updated: [SPARK-30943][SS] Show "batch ID" in tool tip string for Structured Streaming UI graphs

2020-02-25 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch branch-3.0
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.0 by this push:
 new 5343059  [SPARK-30943][SS] Show "batch ID" in tool tip string for 
Structured Streaming UI graphs
5343059 is described below

commit 53430594587ad0134eff5cd2b5e06a7a3eec1b99
Author: Jungtaek Lim (HeartSaVioR) 
AuthorDate: Tue Feb 25 15:29:36 2020 -0800

[SPARK-30943][SS] Show "batch ID" in tool tip string for Structured 
Streaming UI graphs

### What changes were proposed in this pull request?

This patch changes the tool tip string in Structured Streaming UI graphs to 
show batch ID (and timestamp as well) instead of only showing timestamp, which 
was a key for DStream but no longer a key for Structured Streaming.

This patch does some refactoring as there're some spots on confusion 
between js file for streaming and structured streaming.

Note that this patch doesn't actually change the x axis, as once we change 
it we should decouple the logic for graphs between streaming and structured 
streaming. It won't change UX meaningfully as in x axis we only show min and 
max which we still would like to know about "time" as well as batch ID.

### Why are the changes needed?

In Structured Streaming, everything is aligned for "batch ID" where the UI 
is only showing timestamp - end users have to manually find and correlate batch 
ID and the timestamp which is clearly a huge pain.

### Does this PR introduce any user-facing change?

No

### How was this patch tested?

Manually tested. Screenshots:

![Screen Shot 2020-02-25 at 7 22 38 
AM](https://user-images.githubusercontent.com/1317309/75197701-40b2ce80-57a2-11ea-9578-c2eb2d1091de.png)
![Screen Shot 2020-02-25 at 7 22 44 
AM](https://user-images.githubusercontent.com/1317309/75197704-427c9200-57a2-11ea-9439-e0a8303d0860.png)
![Screen Shot 2020-02-25 at 7 22 58 
AM](https://user-images.githubusercontent.com/1317309/75197706-43152880-57a2-11ea-9617-1276c3ba181e.png)
![Screen Shot 2020-02-25 at 7 23 04 
AM](https://user-images.githubusercontent.com/1317309/75197708-43152880-57a2-11ea-9de2-7d37eaf88102.png)
![Screen Shot 2020-02-25 at 7 23 31 
AM](https://user-images.githubusercontent.com/1317309/75197710-43adbf00-57a2-11ea-9ae4-4e292de39c36.png)

Closes #27687 from HeartSaVioR/SPARK-30943.

Authored-by: Jungtaek Lim (HeartSaVioR) 
Signed-off-by: Shixiong Zhu 
(cherry picked from commit 9ea6c0a8975a1277abba799b51aca4e2818c23e7)
Signed-off-by: Shixiong Zhu 
---
 .../org/apache/spark/ui/static/streaming-page.js   |  2 +-
 .../spark/ui/static/structured-streaming-page.js   |  4 +--
 .../ui/StreamingQueryStatisticsPage.scala  | 36 ++
 .../apache/spark/streaming/ui/StreamingPage.scala  | 13 +++-
 4 files changed, 45 insertions(+), 10 deletions(-)

diff --git 
a/core/src/main/resources/org/apache/spark/ui/static/streaming-page.js 
b/core/src/main/resources/org/apache/spark/ui/static/streaming-page.js
index 5b75bc3..ed3e65c3 100644
--- a/core/src/main/resources/org/apache/spark/ui/static/streaming-page.js
+++ b/core/src/main/resources/org/apache/spark/ui/static/streaming-page.js
@@ -171,7 +171,7 @@ function drawTimeline(id, data, minX, maxX, minY, maxY, 
unitY, batchInterval) {
 .attr("cy", function(d) { return y(d.y); })
 .attr("r", function(d) { return isFailedBatch(d.x) ? "2" : "3";})
 .on('mouseover', function(d) {
-var tip = formatYValue(d.y) + " " + unitY + " at " + 
timeFormat[d.x];
+var tip = formatYValue(d.y) + " " + unitY + " at " + 
timeTipStrings[d.x];
 showBootstrapTooltip(d3.select(this).node(), tip);
 // show the point
 d3.select(this)
diff --git 
a/core/src/main/resources/org/apache/spark/ui/static/structured-streaming-page.js
 
b/core/src/main/resources/org/apache/spark/ui/static/structured-streaming-page.js
index 70250fd..c92226b 100644
--- 
a/core/src/main/resources/org/apache/spark/ui/static/structured-streaming-page.js
+++ 
b/core/src/main/resources/org/apache/spark/ui/static/structured-streaming-page.js
@@ -106,12 +106,12 @@ function drawAreaStack(id, labels, values, minX, maxX, 
minY, maxY) {
 .on('mouseover', function(d) {
 var tip = '';
 var idx = 0;
-var _values = timeToValues[d._x]
+var _values = formattedTimeToValues[d._x];
 _values.forEach(function (k) {
 tip += labels[idx] + ': ' + k + '   ';
 idx += 1;
 });
-tip += " at " + d._x
+tip += " a

[spark] branch master updated: [SPARK-30943][SS] Show "batch ID" in tool tip string for Structured Streaming UI graphs

2020-02-25 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 9ea6c0a  [SPARK-30943][SS] Show "batch ID" in tool tip string for 
Structured Streaming UI graphs
9ea6c0a is described below

commit 9ea6c0a8975a1277abba799b51aca4e2818c23e7
Author: Jungtaek Lim (HeartSaVioR) 
AuthorDate: Tue Feb 25 15:29:36 2020 -0800

[SPARK-30943][SS] Show "batch ID" in tool tip string for Structured 
Streaming UI graphs

### What changes were proposed in this pull request?

This patch changes the tool tip string in Structured Streaming UI graphs to 
show batch ID (and timestamp as well) instead of only showing timestamp, which 
was a key for DStream but no longer a key for Structured Streaming.

This patch does some refactoring as there're some spots on confusion 
between js file for streaming and structured streaming.

Note that this patch doesn't actually change the x axis, as once we change 
it we should decouple the logic for graphs between streaming and structured 
streaming. It won't change UX meaningfully as in x axis we only show min and 
max which we still would like to know about "time" as well as batch ID.

### Why are the changes needed?

In Structured Streaming, everything is aligned for "batch ID" where the UI 
is only showing timestamp - end users have to manually find and correlate batch 
ID and the timestamp which is clearly a huge pain.

### Does this PR introduce any user-facing change?

No

### How was this patch tested?

Manually tested. Screenshots:

![Screen Shot 2020-02-25 at 7 22 38 
AM](https://user-images.githubusercontent.com/1317309/75197701-40b2ce80-57a2-11ea-9578-c2eb2d1091de.png)
![Screen Shot 2020-02-25 at 7 22 44 
AM](https://user-images.githubusercontent.com/1317309/75197704-427c9200-57a2-11ea-9439-e0a8303d0860.png)
![Screen Shot 2020-02-25 at 7 22 58 
AM](https://user-images.githubusercontent.com/1317309/75197706-43152880-57a2-11ea-9617-1276c3ba181e.png)
![Screen Shot 2020-02-25 at 7 23 04 
AM](https://user-images.githubusercontent.com/1317309/75197708-43152880-57a2-11ea-9de2-7d37eaf88102.png)
![Screen Shot 2020-02-25 at 7 23 31 
AM](https://user-images.githubusercontent.com/1317309/75197710-43adbf00-57a2-11ea-9ae4-4e292de39c36.png)

Closes #27687 from HeartSaVioR/SPARK-30943.

Authored-by: Jungtaek Lim (HeartSaVioR) 
Signed-off-by: Shixiong Zhu 
---
 .../org/apache/spark/ui/static/streaming-page.js   |  2 +-
 .../spark/ui/static/structured-streaming-page.js   |  4 +--
 .../ui/StreamingQueryStatisticsPage.scala  | 36 ++
 .../apache/spark/streaming/ui/StreamingPage.scala  | 13 +++-
 4 files changed, 45 insertions(+), 10 deletions(-)

diff --git 
a/core/src/main/resources/org/apache/spark/ui/static/streaming-page.js 
b/core/src/main/resources/org/apache/spark/ui/static/streaming-page.js
index 5b75bc3..ed3e65c3 100644
--- a/core/src/main/resources/org/apache/spark/ui/static/streaming-page.js
+++ b/core/src/main/resources/org/apache/spark/ui/static/streaming-page.js
@@ -171,7 +171,7 @@ function drawTimeline(id, data, minX, maxX, minY, maxY, 
unitY, batchInterval) {
 .attr("cy", function(d) { return y(d.y); })
 .attr("r", function(d) { return isFailedBatch(d.x) ? "2" : "3";})
 .on('mouseover', function(d) {
-var tip = formatYValue(d.y) + " " + unitY + " at " + 
timeFormat[d.x];
+var tip = formatYValue(d.y) + " " + unitY + " at " + 
timeTipStrings[d.x];
 showBootstrapTooltip(d3.select(this).node(), tip);
 // show the point
 d3.select(this)
diff --git 
a/core/src/main/resources/org/apache/spark/ui/static/structured-streaming-page.js
 
b/core/src/main/resources/org/apache/spark/ui/static/structured-streaming-page.js
index 70250fd..c92226b 100644
--- 
a/core/src/main/resources/org/apache/spark/ui/static/structured-streaming-page.js
+++ 
b/core/src/main/resources/org/apache/spark/ui/static/structured-streaming-page.js
@@ -106,12 +106,12 @@ function drawAreaStack(id, labels, values, minX, maxX, 
minY, maxY) {
 .on('mouseover', function(d) {
 var tip = '';
 var idx = 0;
-var _values = timeToValues[d._x]
+var _values = formattedTimeToValues[d._x];
 _values.forEach(function (k) {
 tip += labels[idx] + ': ' + k + '   ';
 idx += 1;
 });
-tip += " at " + d._x
+tip += " at " + formattedTimeTipStrings[d._x];
 showBootstrapTooltip(d3.select(th

[spark] branch branch-3.0 updated: [SPARK-29748][DOCS][FOLLOW-UP] Add a note that the legacy environment variable to set in both executor and driver

2020-02-14 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch branch-3.0
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.0 by this push:
 new 1385fc0  [SPARK-29748][DOCS][FOLLOW-UP] Add a note that the legacy 
environment variable to set in both executor and driver
1385fc0 is described below

commit 1385fc02ce7d28e6570971e1687e74d245a5533f
Author: HyukjinKwon 
AuthorDate: Fri Feb 14 10:18:08 2020 -0800

[SPARK-29748][DOCS][FOLLOW-UP] Add a note that the legacy environment 
variable to set in both executor and driver

### What changes were proposed in this pull request?

This PR address the comment at 
https://github.com/apache/spark/pull/26496#discussion_r379194091 and improves 
the migration guide to explicitly note that the legacy environment variable to 
set in both executor and driver.

### Why are the changes needed?

To clarify this env should be set both in driver and executors.

### Does this PR introduce any user-facing change?

Nope.

### How was this patch tested?

I checked it via md editor.

Closes #27573 from HyukjinKwon/SPARK-29748.

Authored-by: HyukjinKwon 
Signed-off-by: Shixiong Zhu 
(cherry picked from commit b343757b1bd5d0344b82f36aa4d65ed34f840606)
Signed-off-by: Shixiong Zhu 
---
 docs/pyspark-migration-guide.md | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/docs/pyspark-migration-guide.md b/docs/pyspark-migration-guide.md
index 8ea4fec..f7f2038 100644
--- a/docs/pyspark-migration-guide.md
+++ b/docs/pyspark-migration-guide.md
@@ -87,7 +87,7 @@ Please refer [Migration Guide: SQL, Datasets and 
DataFrame](sql-migration-guide.
   - Since Spark 3.0, `Column.getItem` is fixed such that it does not call 
`Column.apply`. Consequently, if `Column` is used as an argument to `getItem`, 
the indexing operator should be used.
 For example, `map_col.getItem(col('id'))` should be replaced with 
`map_col[col('id')]`.
 
-  - As of Spark 3.0 `Row` field names are no longer sorted alphabetically when 
constructing with named arguments for Python versions 3.6 and above, and the 
order of fields will match that as entered. To enable sorted fields by default, 
as in Spark 2.4, set the environment variable 
`PYSPARK_ROW_FIELD_SORTING_ENABLED` to "true". For Python versions less than 
3.6, the field names will be sorted alphabetically as the only option.
+  - As of Spark 3.0 `Row` field names are no longer sorted alphabetically when 
constructing with named arguments for Python versions 3.6 and above, and the 
order of fields will match that as entered. To enable sorted fields by default, 
as in Spark 2.4, set the environment variable 
`PYSPARK_ROW_FIELD_SORTING_ENABLED` to "true" for both executors and driver - 
this environment variable must be consistent on all executors and driver; 
otherwise, it may cause failures or incorrect answers. For [...]
 
 ## Upgrading from PySpark 2.3 to 2.4
 


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



[spark] branch master updated: [SPARK-29748][DOCS][FOLLOW-UP] Add a note that the legacy environment variable to set in both executor and driver

2020-02-14 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new b343757  [SPARK-29748][DOCS][FOLLOW-UP] Add a note that the legacy 
environment variable to set in both executor and driver
b343757 is described below

commit b343757b1bd5d0344b82f36aa4d65ed34f840606
Author: HyukjinKwon 
AuthorDate: Fri Feb 14 10:18:08 2020 -0800

[SPARK-29748][DOCS][FOLLOW-UP] Add a note that the legacy environment 
variable to set in both executor and driver

### What changes were proposed in this pull request?

This PR address the comment at 
https://github.com/apache/spark/pull/26496#discussion_r379194091 and improves 
the migration guide to explicitly note that the legacy environment variable to 
set in both executor and driver.

### Why are the changes needed?

To clarify this env should be set both in driver and executors.

### Does this PR introduce any user-facing change?

Nope.

### How was this patch tested?

I checked it via md editor.

Closes #27573 from HyukjinKwon/SPARK-29748.

Authored-by: HyukjinKwon 
Signed-off-by: Shixiong Zhu 
---
 docs/pyspark-migration-guide.md | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/docs/pyspark-migration-guide.md b/docs/pyspark-migration-guide.md
index 8ea4fec..f7f2038 100644
--- a/docs/pyspark-migration-guide.md
+++ b/docs/pyspark-migration-guide.md
@@ -87,7 +87,7 @@ Please refer [Migration Guide: SQL, Datasets and 
DataFrame](sql-migration-guide.
   - Since Spark 3.0, `Column.getItem` is fixed such that it does not call 
`Column.apply`. Consequently, if `Column` is used as an argument to `getItem`, 
the indexing operator should be used.
 For example, `map_col.getItem(col('id'))` should be replaced with 
`map_col[col('id')]`.
 
-  - As of Spark 3.0 `Row` field names are no longer sorted alphabetically when 
constructing with named arguments for Python versions 3.6 and above, and the 
order of fields will match that as entered. To enable sorted fields by default, 
as in Spark 2.4, set the environment variable 
`PYSPARK_ROW_FIELD_SORTING_ENABLED` to "true". For Python versions less than 
3.6, the field names will be sorted alphabetically as the only option.
+  - As of Spark 3.0 `Row` field names are no longer sorted alphabetically when 
constructing with named arguments for Python versions 3.6 and above, and the 
order of fields will match that as entered. To enable sorted fields by default, 
as in Spark 2.4, set the environment variable 
`PYSPARK_ROW_FIELD_SORTING_ENABLED` to "true" for both executors and driver - 
this environment variable must be consistent on all executors and driver; 
otherwise, it may cause failures or incorrect answers. For [...]
 
 ## Upgrading from PySpark 2.3 to 2.4
 


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



[spark] branch branch-3.0 updated: [SPARK-29543][SS][FOLLOWUP] Move `spark.sql.streaming.ui.*` configs to StaticSQLConf

2020-02-02 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch branch-3.0
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.0 by this push:
 new f9b8637  [SPARK-29543][SS][FOLLOWUP] Move `spark.sql.streaming.ui.*` 
configs to StaticSQLConf
f9b8637 is described below

commit f9b86370cb04b72a4f00cbd4d60873960aa2792c
Author: Yuanjian Li 
AuthorDate: Sun Feb 2 23:37:13 2020 -0800

[SPARK-29543][SS][FOLLOWUP] Move `spark.sql.streaming.ui.*` configs to 
StaticSQLConf

### What changes were proposed in this pull request?
Put the configs below needed by Structured Streaming UI into StaticSQLConf:

- spark.sql.streaming.ui.enabled
- spark.sql.streaming.ui.retainedProgressUpdates
- spark.sql.streaming.ui.retainedQueries

### Why are the changes needed?
Make all SS UI configs consistent with other similar configs in usage and 
naming.

### Does this PR introduce any user-facing change?
Yes, add new static config `spark.sql.streaming.ui.retainedProgressUpdates`.

### How was this patch tested?
Existing UT.

Closes #27425 from xuanyuanking/SPARK-29543-follow.

Authored-by: Yuanjian Li 
Signed-off-by: Shixiong Zhu 
(cherry picked from commit a4912cee615314e9578e6ab4eae25f147feacbd5)
Signed-off-by: Shixiong Zhu 
---
 .../org/apache/spark/sql/internal/SQLConf.scala  | 16 
 .../apache/spark/sql/internal/StaticSQLConf.scala| 20 
 .../org/apache/spark/sql/internal/SharedState.scala  | 15 ---
 .../streaming/ui/StreamingQueryStatusListener.scala  | 10 ++
 .../spark/sql/streaming/ui/StreamingQueryTab.scala   |  2 +-
 .../ui/StreamingQueryStatusListenerSuite.scala   |  4 ++--
 6 files changed, 37 insertions(+), 30 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
index 04572c3..3ad3416 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
@@ -1150,18 +1150,6 @@ object SQLConf {
   .booleanConf
   .createWithDefault(true)
 
-  val STREAMING_UI_ENABLED =
-buildConf("spark.sql.streaming.ui.enabled")
-  .doc("Whether to run the structured streaming UI for the Spark 
application.")
-  .booleanConf
-  .createWithDefault(true)
-
-  val STREAMING_UI_INACTIVE_QUERY_RETENTION =
-buildConf("spark.sql.streaming.ui.numInactiveQueries")
-  .doc("The number of inactive queries to retain for structured streaming 
ui.")
-  .intConf
-  .createWithDefault(100)
-
   val VARIABLE_SUBSTITUTE_ENABLED =
 buildConf("spark.sql.variable.substitute")
   .doc("This enables substitution using syntax like ${var} ${system:var} 
and ${env:var}.")
@@ -2284,10 +2272,6 @@ class SQLConf extends Serializable with Logging {
 
   def isUnsupportedOperationCheckEnabled: Boolean = 
getConf(UNSUPPORTED_OPERATION_CHECK_ENABLED)
 
-  def isStreamingUIEnabled: Boolean = getConf(STREAMING_UI_ENABLED)
-
-  def streamingUIInactiveQueryRetention: Int = 
getConf(STREAMING_UI_INACTIVE_QUERY_RETENTION)
-
   def streamingFileCommitProtocolClass: String = 
getConf(STREAMING_FILE_COMMIT_PROTOCOL_CLASS)
 
   def fileSinkLogDeletion: Boolean = getConf(FILE_SINK_LOG_DELETION)
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala
index 66ac9ddb..6bc7522 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala
@@ -176,4 +176,24 @@ object StaticSQLConf {
   .internal()
   .booleanConf
   .createWithDefault(true)
+
+  val STREAMING_UI_ENABLED =
+buildStaticConf("spark.sql.streaming.ui.enabled")
+  .doc("Whether to run the Structured Streaming Web UI for the Spark 
application when the " +
+"Spark Web UI is enabled.")
+  .booleanConf
+  .createWithDefault(true)
+
+  val STREAMING_UI_RETAINED_PROGRESS_UPDATES =
+buildStaticConf("spark.sql.streaming.ui.retainedProgressUpdates")
+  .doc("The number of progress updates to retain for a streaming query for 
Structured " +
+"Streaming UI.")
+  .intConf
+  .createWithDefault(100)
+
+  val STREAMING_UI_RETAINED_QUERIES =
+buildStaticConf("spark.sql.streaming.ui.retainedQueries")
+  .doc("The number of inactive queries to retain for Structured Streaming 
UI.")
+  .intConf
+  .createWithDefault(100)
 }
diff --git 
a/sql/core/src/main/scala/org/apache/s

[spark] branch master updated: [SPARK-29543][SS][FOLLOWUP] Move `spark.sql.streaming.ui.*` configs to StaticSQLConf

2020-02-02 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new a4912ce  [SPARK-29543][SS][FOLLOWUP] Move `spark.sql.streaming.ui.*` 
configs to StaticSQLConf
a4912ce is described below

commit a4912cee615314e9578e6ab4eae25f147feacbd5
Author: Yuanjian Li 
AuthorDate: Sun Feb 2 23:37:13 2020 -0800

[SPARK-29543][SS][FOLLOWUP] Move `spark.sql.streaming.ui.*` configs to 
StaticSQLConf

### What changes were proposed in this pull request?
Put the configs below needed by Structured Streaming UI into StaticSQLConf:

- spark.sql.streaming.ui.enabled
- spark.sql.streaming.ui.retainedProgressUpdates
- spark.sql.streaming.ui.retainedQueries

### Why are the changes needed?
Make all SS UI configs consistent with other similar configs in usage and 
naming.

### Does this PR introduce any user-facing change?
Yes, add new static config `spark.sql.streaming.ui.retainedProgressUpdates`.

### How was this patch tested?
Existing UT.

Closes #27425 from xuanyuanking/SPARK-29543-follow.

Authored-by: Yuanjian Li 
Signed-off-by: Shixiong Zhu 
---
 .../org/apache/spark/sql/internal/SQLConf.scala  | 16 
 .../apache/spark/sql/internal/StaticSQLConf.scala| 20 
 .../org/apache/spark/sql/internal/SharedState.scala  | 15 ---
 .../streaming/ui/StreamingQueryStatusListener.scala  | 10 ++
 .../spark/sql/streaming/ui/StreamingQueryTab.scala   |  2 +-
 .../ui/StreamingQueryStatusListenerSuite.scala   |  4 ++--
 6 files changed, 37 insertions(+), 30 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
index 04572c3..3ad3416 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
@@ -1150,18 +1150,6 @@ object SQLConf {
   .booleanConf
   .createWithDefault(true)
 
-  val STREAMING_UI_ENABLED =
-buildConf("spark.sql.streaming.ui.enabled")
-  .doc("Whether to run the structured streaming UI for the Spark 
application.")
-  .booleanConf
-  .createWithDefault(true)
-
-  val STREAMING_UI_INACTIVE_QUERY_RETENTION =
-buildConf("spark.sql.streaming.ui.numInactiveQueries")
-  .doc("The number of inactive queries to retain for structured streaming 
ui.")
-  .intConf
-  .createWithDefault(100)
-
   val VARIABLE_SUBSTITUTE_ENABLED =
 buildConf("spark.sql.variable.substitute")
   .doc("This enables substitution using syntax like ${var} ${system:var} 
and ${env:var}.")
@@ -2284,10 +2272,6 @@ class SQLConf extends Serializable with Logging {
 
   def isUnsupportedOperationCheckEnabled: Boolean = 
getConf(UNSUPPORTED_OPERATION_CHECK_ENABLED)
 
-  def isStreamingUIEnabled: Boolean = getConf(STREAMING_UI_ENABLED)
-
-  def streamingUIInactiveQueryRetention: Int = 
getConf(STREAMING_UI_INACTIVE_QUERY_RETENTION)
-
   def streamingFileCommitProtocolClass: String = 
getConf(STREAMING_FILE_COMMIT_PROTOCOL_CLASS)
 
   def fileSinkLogDeletion: Boolean = getConf(FILE_SINK_LOG_DELETION)
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala
index 66ac9ddb..6bc7522 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala
@@ -176,4 +176,24 @@ object StaticSQLConf {
   .internal()
   .booleanConf
   .createWithDefault(true)
+
+  val STREAMING_UI_ENABLED =
+buildStaticConf("spark.sql.streaming.ui.enabled")
+  .doc("Whether to run the Structured Streaming Web UI for the Spark 
application when the " +
+"Spark Web UI is enabled.")
+  .booleanConf
+  .createWithDefault(true)
+
+  val STREAMING_UI_RETAINED_PROGRESS_UPDATES =
+buildStaticConf("spark.sql.streaming.ui.retainedProgressUpdates")
+  .doc("The number of progress updates to retain for a streaming query for 
Structured " +
+"Streaming UI.")
+  .intConf
+  .createWithDefault(100)
+
+  val STREAMING_UI_RETAINED_QUERIES =
+buildStaticConf("spark.sql.streaming.ui.retainedQueries")
+  .doc("The number of inactive queries to retain for Structured Streaming 
UI.")
+  .intConf
+  .createWithDefault(100)
 }
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala
i

[spark] branch master updated: [SPARK-30657][SPARK-30658][SS] Fixed two bugs in streaming limits

2020-01-31 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 481e521  [SPARK-30657][SPARK-30658][SS] Fixed two bugs in streaming 
limits
481e521 is described below

commit 481e5211d237173ea0fb7c0b292eb7abd2b8a3fe
Author: Tathagata Das 
AuthorDate: Fri Jan 31 09:26:03 2020 -0800

[SPARK-30657][SPARK-30658][SS] Fixed two bugs in streaming limits

This PR solves two bugs related to streaming limits

**Bug 1 (SPARK-30658)**: Limit before a streaming aggregate (i.e. 
`df.limit(5).groupBy().count()`) in complete mode was not being planned as a 
stateful streaming limit. The planner rule planned a logical limit with a 
stateful streaming limit plan only if the query is in append mode. As a result, 
instead of allowing max 5 rows across batches, the planned streaming query was 
allowing 5 rows in every batch thus producing incorrect results.

**Solution**: Change the planner rule to plan the logical limit with a 
streaming limit plan even when the query is in complete mode if the logical 
limit has no stateful operator before it.

**Bug 2 (SPARK-30657)**: `LocalLimitExec` does not consume the iterator of 
the child plan. So if there is a limit after a stateful operator like streaming 
dedup in append mode (e.g. `df.dropDuplicates().limit(5)`), the state changes 
of streaming duplicate may not be committed (most stateful ops commit state 
changes only after the generated iterator is fully consumed).

**Solution**: Change the planner rule to always use a new 
`StreamingLocalLimitExec` which always fully consumes the iterator. This is the 
safest thing to do. However, this will introduce a performance regression as 
consuming the iterator is extra work. To minimize this performance impact, add 
an additional post-planner optimization rule to replace 
`StreamingLocalLimitExec` with `LocalLimitExec` when there is no stateful 
operator before the limit that could be affected by it.

No

Updated incorrect unit tests and added new ones

Closes #27373 from tdas/SPARK-30657.

Authored-by: Tathagata Das 
Signed-off-by: Shixiong Zhu 
---
 .../spark/sql/execution/SparkStrategies.scala  |  38 ---
 .../execution/streaming/IncrementalExecution.scala |  34 ++-
 ...GlobalLimitExec.scala => streamingLimits.scala} |  55 --
 .../apache/spark/sql/streaming/StreamSuite.scala   | 112 -
 4 files changed, 211 insertions(+), 28 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
index 00ad4e0..bd2684d 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
@@ -451,21 +451,35 @@ abstract class SparkStrategies extends 
QueryPlanner[SparkPlan] {
* Used to plan the streaming global limit operator for streams in append 
mode.
* We need to check for either a direct Limit or a Limit wrapped in a 
ReturnAnswer operator,
* following the example of the SpecialLimits Strategy above.
-   * Streams with limit in Append mode use the stateful 
StreamingGlobalLimitExec.
-   * Streams with limit in Complete mode use the stateless CollectLimitExec 
operator.
-   * Limit is unsupported for streams in Update mode.
*/
   case class StreamingGlobalLimitStrategy(outputMode: OutputMode) extends 
Strategy {
-override def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
-  case ReturnAnswer(rootPlan) => rootPlan match {
-case Limit(IntegerLiteral(limit), child)
-if plan.isStreaming && outputMode == InternalOutputModes.Append =>
-  StreamingGlobalLimitExec(limit, LocalLimitExec(limit, 
planLater(child))) :: Nil
-case _ => Nil
+
+private def generatesStreamingAppends(plan: LogicalPlan): Boolean = {
+
+  /** Ensures that this plan does not have a streaming aggregate in it. */
+  def hasNoStreamingAgg: Boolean = {
+plan.collectFirst { case a: Aggregate if a.isStreaming => a }.isEmpty
   }
-  case Limit(IntegerLiteral(limit), child)
-  if plan.isStreaming && outputMode == InternalOutputModes.Append =>
-StreamingGlobalLimitExec(limit, LocalLimitExec(limit, 
planLater(child))) :: Nil
+
+  // The following cases of limits on a streaming plan has to be executed 
with a stateful
+  // streaming plan.
+  // 1. When the query is in append mode (that is, all logical plan 
operate on appended data).
+  // 2. When the plan does not contain any streaming aggregate (that is, 
plan has only
+  //operators that operate on appended data). This must be executed 
with

[spark] branch master updated: [SPARK-30656][SS] Support the "minPartitions" option in Kafka batch source and streaming source v1

2020-01-30 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new f56ba37  [SPARK-30656][SS] Support the "minPartitions" option in Kafka 
batch source and streaming source v1
f56ba37 is described below

commit f56ba37d8bf618f2bef23d808e0fc5704261b139
Author: Shixiong Zhu 
AuthorDate: Thu Jan 30 18:14:50 2020 -0800

[SPARK-30656][SS] Support the "minPartitions" option in Kafka batch source 
and streaming source v1

### What changes were proposed in this pull request?

- Add `minPartitions` support for Kafka Streaming V1 source.
- Add `minPartitions` support for Kafka batch V1  and V2 source.
- There is lots of refactoring (moving codes to KafkaOffsetReader) to reuse 
codes.

### Why are the changes needed?

Right now, the "minPartitions" option only works in Kafka streaming source 
v2. It would be great that we can support it in batch and streaming source v1 
(v1 is the fallback mode when a user hits a regression in v2) as well.

### Does this PR introduce any user-facing change?

Yep. The `minPartitions` options is supported in Kafka batch and streaming 
queries for both data source V1 and V2.

### How was this patch tested?

New unit tests are added to test "minPartitions".

Closes #27388 from zsxwing/kafka-min-partitions.

Authored-by: Shixiong Zhu 
Signed-off-by: Shixiong Zhu 
---
 docs/structured-streaming-kafka-integration.md |   2 +-
 .../org/apache/spark/sql/kafka010/KafkaBatch.scala |  32 +
 .../spark/sql/kafka010/KafkaMicroBatchStream.scala |  75 +-
 .../sql/kafka010/KafkaOffsetRangeCalculator.scala  |  20 +--
 .../spark/sql/kafka010/KafkaOffsetReader.scala | 156 +
 .../apache/spark/sql/kafka010/KafkaRelation.scala  |  32 +
 .../apache/spark/sql/kafka010/KafkaSource.scala|  64 +
 .../apache/spark/sql/kafka010/KafkaSourceRDD.scala |  21 +--
 .../sql/kafka010/KafkaMicroBatchSourceSuite.scala  |  29 
 .../kafka010/KafkaOffsetRangeCalculatorSuite.scala | 107 +-
 .../sql/kafka010/KafkaOffsetReaderSuite.scala  | 139 ++
 .../spark/sql/kafka010/KafkaRelationSuite.scala|  22 +++
 12 files changed, 448 insertions(+), 251 deletions(-)

diff --git a/docs/structured-streaming-kafka-integration.md 
b/docs/structured-streaming-kafka-integration.md
index 0820b38..a15 100644
--- a/docs/structured-streaming-kafka-integration.md
+++ b/docs/structured-streaming-kafka-integration.md
@@ -469,7 +469,7 @@ The following configurations are optional:
   minPartitions
   int
   none
-  streaming
+  streaming and batch
   Desired minimum number of partitions to read from Kafka.
   By default, Spark has a 1-1 mapping of topicPartitions to Spark partitions 
consuming from Kafka.
   If you set this option to a value greater than your topicPartitions, Spark 
will divvy up large
diff --git 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatch.scala
 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatch.scala
index 3006770..9ad083f 100644
--- 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatch.scala
+++ 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatch.scala
@@ -57,36 +57,12 @@ private[kafka010] class KafkaBatch(
   driverGroupIdPrefix = s"$uniqueGroupId-driver")
 
 // Leverage the KafkaReader to obtain the relevant partition offsets
-val (fromPartitionOffsets, untilPartitionOffsets) = {
-  try {
-(kafkaOffsetReader.fetchPartitionOffsets(startingOffsets, 
isStartingOffsets = true),
-  kafkaOffsetReader.fetchPartitionOffsets(endingOffsets, 
isStartingOffsets = false))
-  } finally {
-kafkaOffsetReader.close()
-  }
+val offsetRanges: Seq[KafkaOffsetRange] = try {
+  kafkaOffsetReader.getOffsetRangesFromUnresolvedOffsets(startingOffsets, 
endingOffsets)
+} finally {
+  kafkaOffsetReader.close()
 }
 
-// Obtain topicPartitions in both from and until partition offset, ignoring
-// topic partitions that were added and/or deleted between the two above 
calls.
-if (fromPartitionOffsets.keySet != untilPartitionOffsets.keySet) {
-  implicit val topicOrdering: Ordering[TopicPartition] = Ordering.by(t => 
t.topic())
-  val fromTopics = fromPartitionOffsets.keySet.toList.sorted.mkString(",")
-  val untilTopics = 
untilPartitionOffsets.keySet.toList.sorted.mkString(",")
-  throw new IllegalStateException("different topic partitions " +
-s"for starting offsets topics[${fromTopics}] and " +
-s"ending offsets topics[${untilTopics}]")

[spark] branch master updated: [SPARK-29543][SS][UI] Structured Streaming Web UI

2020-01-29 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 7173786  [SPARK-29543][SS][UI] Structured Streaming Web UI
7173786 is described below

commit 71737861531180bbda9aec8d241b1428fe91cab2
Author: uncleGen 
AuthorDate: Wed Jan 29 13:43:51 2020 -0800

[SPARK-29543][SS][UI] Structured Streaming Web UI

### What changes were proposed in this pull request?

This PR adds two pages to Web UI for Structured Streaming:
   - "/streamingquery": Streaming Query Page, providing some aggregate 
information for running/completed streaming queries.
  - "/streamingquery/statistics": Streaming Query Statistics Page, 
providing detailed information for streaming query, including `Input Rate`, 
`Process Rate`, `Input Rows`, `Batch Duration` and `Operation Duration`

![Screen Shot 2020-01-29 at 1 38 00 
PM](https://user-images.githubusercontent.com/1000778/73399837-cd01cc80-429c-11ea-9d4b-1d200a41b8d5.png)
![Screen Shot 2020-01-29 at 1 39 16 
PM](https://user-images.githubusercontent.com/1000778/73399838-cd01cc80-429c-11ea-8185-4e56db6866bd.png)

### Why are the changes needed?

It helps users to better monitor Structured Streaming query.

### Does this PR introduce any user-facing change?

No

### How was this patch tested?

- new added and existing UTs
- manual test

Closes #26201 from uncleGen/SPARK-29543.

Lead-authored-by: uncleGen 
Co-authored-by: Yuanjian Li 
Co-authored-by: Genmao Yu 
Signed-off-by: Shixiong Zhu 
---
 .../org/apache/spark}/ui/static/streaming-page.css |   0
 .../org/apache/spark}/ui/static/streaming-page.js  |   0
 .../spark/ui/static/structured-streaming-page.js   | 171 +
 .../resources/org/apache/spark/ui/static/webui.js  |   2 +
 .../scala/org/apache/spark/ui/GraphUIData.scala| 169 +
 .../main/scala/org/apache/spark/ui/UIUtils.scala   |  91 +++
 .../scala/org/apache/spark/ui/jobs/StagePage.scala |  14 +-
 .../org/apache/spark/ui/jobs/StageTable.scala  |  14 +-
 project/MimaExcludes.scala |   5 +-
 .../org/apache/spark/sql/internal/SQLConf.scala|  16 ++
 .../sql/execution/streaming/ProgressReporter.scala |   5 +-
 .../sql/execution/streaming/StreamExecution.scala  |   3 +-
 .../apache/spark/sql/internal/SharedState.scala|  19 +-
 .../sql/streaming/StreamingQueryListener.scala |   4 +-
 .../sql/streaming/StreamingQueryManager.scala  |   6 +-
 .../org/apache/spark/sql/streaming/progress.scala  |   2 +
 .../sql/streaming/ui/StreamingQueryPage.scala  | 147 +++
 .../ui/StreamingQueryStatisticsPage.scala  | 271 +
 .../ui/StreamingQueryStatusListener.scala  | 122 ++
 .../spark/sql/streaming/ui/StreamingQueryTab.scala |  33 +--
 .../apache/spark/sql/streaming/ui/UIUtils.scala|  60 +
 .../streaming/StreamingQueryListenerSuite.scala|  10 +-
 .../StreamingQueryStatusAndProgressSuite.scala |   2 +
 .../spark/sql/streaming/StreamingQuerySuite.scala  |  14 +-
 .../sql/streaming/ui/StreamingQueryPageSuite.scala | 125 ++
 .../ui/StreamingQueryStatusListenerSuite.scala | 101 
 .../spark/sql/streaming/ui/UIUtilsSuite.scala  |  41 
 .../hive/thriftserver/ui/ThriftServerPage.scala|  16 +-
 .../apache/spark/streaming/dstream/DStream.scala   |   4 +-
 .../spark/streaming/scheduler/JobScheduler.scala   |   4 +-
 .../spark/streaming/ui/AllBatchesTable.scala   |   2 +-
 .../org/apache/spark/streaming/ui/BatchPage.scala  |   2 +-
 .../apache/spark/streaming/ui/StreamingPage.scala  | 125 +-
 .../apache/spark/streaming/ui/StreamingTab.scala   |   2 +-
 .../org/apache/spark/streaming/ui/UIUtils.scala|  71 +-
 .../apache/spark/streaming/DStreamScopeSuite.scala |   6 +-
 .../apache/spark/streaming/ui/UIUtilsSuite.scala   |  12 +-
 37 files changed, 1408 insertions(+), 283 deletions(-)

diff --git 
a/streaming/src/main/resources/org/apache/spark/streaming/ui/static/streaming-page.css
 b/core/src/main/resources/org/apache/spark/ui/static/streaming-page.css
similarity index 100%
rename from 
streaming/src/main/resources/org/apache/spark/streaming/ui/static/streaming-page.css
rename to core/src/main/resources/org/apache/spark/ui/static/streaming-page.css
diff --git 
a/streaming/src/main/resources/org/apache/spark/streaming/ui/static/streaming-page.js
 b/core/src/main/resources/org/apache/spark/ui/static/streaming-page.js
similarity index 100%
rename from 
streaming/src/main/resources/org/apache/spark/streaming/ui/static/streaming-page.js
rename to core/src/main/resources/org/apache/spark/ui/static/streaming-page.js
diff --git 
a/core/src/main/resources/org/apache/spark/ui/static/structured-streaming-page.

[spark] branch 2.0 created (now 0f3d744c)

2020-01-14 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a change to branch 2.0
in repository https://gitbox.apache.org/repos/asf/spark.git.


  at 0f3d744c [MINOR][TESTS] Remove unsupported `header` option in AvroSuite

No new revisions were added by this update.


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



[spark] branch master updated: Revert "[SPARK-21869][SS] Apply Apache Commons Pool to Kafka producer"

2019-12-10 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new cfd7ca9  Revert "[SPARK-21869][SS] Apply Apache Commons Pool to Kafka 
producer"
cfd7ca9 is described below

commit cfd7ca9a06161f7622b5179a777f965c11892afa
Author: Shixiong Zhu 
AuthorDate: Tue Dec 10 11:21:46 2019 -0800

Revert "[SPARK-21869][SS] Apply Apache Commons Pool to Kafka producer"

This reverts commit 3641c3dd69b2bd2beae028d52356450cc41f69ed.
---
 .../spark/sql/kafka010/CachedKafkaProducer.scala   | 118 +++-
 .../sql/kafka010/InternalKafkaConnectorPool.scala  | 210 -
 .../sql/kafka010/InternalKafkaConsumerPool.scala   | 210 ++---
 .../sql/kafka010/InternalKafkaProducerPool.scala   |  68 ---
 .../spark/sql/kafka010/KafkaDataConsumer.scala |   7 +-
 .../spark/sql/kafka010/KafkaDataWriter.scala   |  34 +---
 .../apache/spark/sql/kafka010/KafkaWriteTask.scala |  20 +-
 .../org/apache/spark/sql/kafka010/package.scala|  34 +---
 .../sql/kafka010/CachedKafkaProducerSuite.scala| 154 ---
 scala => InternalKafkaConsumerPoolSuite.scala} |   8 +-
 .../sql/kafka010/KafkaDataConsumerSuite.scala  |   6 +-
 .../org/apache/spark/sql/kafka010/KafkaTest.scala  |  10 +-
 .../kafka010/KafkaDataConsumerSuite.scala  |   7 +
 13 files changed, 332 insertions(+), 554 deletions(-)

diff --git 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaProducer.scala
 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaProducer.scala
index 907440a..fc177cd 100644
--- 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaProducer.scala
+++ 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaProducer.scala
@@ -18,68 +18,60 @@
 package org.apache.spark.sql.kafka010
 
 import java.{util => ju}
-import java.io.Closeable
+import java.util.concurrent.{ConcurrentMap, ExecutionException, TimeUnit}
 
+import com.google.common.cache._
+import com.google.common.util.concurrent.{ExecutionError, 
UncheckedExecutionException}
+import org.apache.kafka.clients.producer.KafkaProducer
 import scala.collection.JavaConverters._
 import scala.util.control.NonFatal
 
-import org.apache.kafka.clients.producer.{Callback, KafkaProducer, 
ProducerRecord}
-
 import org.apache.spark.SparkEnv
 import org.apache.spark.internal.Logging
 import org.apache.spark.kafka010.{KafkaConfigUpdater, KafkaRedactionUtil}
-import org.apache.spark.sql.kafka010.InternalKafkaProducerPool._
-import org.apache.spark.util.ShutdownHookManager
 
-private[kafka010] class CachedKafkaProducer(val kafkaParams: ju.Map[String, 
Object])
-  extends Closeable with Logging {
+private[kafka010] object CachedKafkaProducer extends Logging {
 
   private type Producer = KafkaProducer[Array[Byte], Array[Byte]]
 
-  private val producer = createProducer()
+  private val defaultCacheExpireTimeout = TimeUnit.MINUTES.toMillis(10)
 
-  private def createProducer(): Producer = {
-val producer: Producer = new Producer(kafkaParams)
-if (log.isDebugEnabled()) {
-  val redactedParamsSeq = 
KafkaRedactionUtil.redactParams(toCacheKey(kafkaParams))
-  logDebug(s"Created a new instance of kafka producer for 
$redactedParamsSeq.")
+  private lazy val cacheExpireTimeout: Long = Option(SparkEnv.get)
+.map(_.conf.get(PRODUCER_CACHE_TIMEOUT))
+.getOrElse(defaultCacheExpireTimeout)
+
+  private val cacheLoader = new CacheLoader[Seq[(String, Object)], Producer] {
+override def load(config: Seq[(String, Object)]): Producer = {
+  createKafkaProducer(config)
 }
-producer
   }
 
-  override def close(): Unit = {
-try {
-  if (log.isInfoEnabled()) {
-val redactedParamsSeq = 
KafkaRedactionUtil.redactParams(toCacheKey(kafkaParams))
-logInfo(s"Closing the KafkaProducer with params: 
${redactedParamsSeq.mkString("\n")}.")
+  private val removalListener = new RemovalListener[Seq[(String, Object)], 
Producer]() {
+override def onRemoval(
+notification: RemovalNotification[Seq[(String, Object)], Producer]): 
Unit = {
+  val paramsSeq: Seq[(String, Object)] = notification.getKey
+  val producer: Producer = notification.getValue
+  if (log.isDebugEnabled()) {
+val redactedParamsSeq = KafkaRedactionUtil.redactParams(paramsSeq)
+logDebug(s"Evicting kafka producer $producer params: 
$redactedParamsSeq, " +
+  s"due to ${notification.getCause}")
   }
-  producer.close()
-} catch {
-  case NonFatal(e) => logWarning("Error while closing kafka producer.", e)
+  close(paramsSeq, producer)
 }
   }
 
-  def send(record: ProducerRecord[Arra

[spark] branch master updated: [SPARK-29953][SS] Don't clean up source files for FileStreamSource if the files belong to the output of FileStreamSink

2019-12-05 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 25431d7  [SPARK-29953][SS] Don't clean up source files for 
FileStreamSource if the files belong to the output of FileStreamSink
25431d7 is described below

commit 25431d79f7daf2a68298701154eb505c2a4add80
Author: Jungtaek Lim (HeartSaVioR) 
AuthorDate: Thu Dec 5 21:46:28 2019 -0800

[SPARK-29953][SS] Don't clean up source files for FileStreamSource if the 
files belong to the output of FileStreamSink

### What changes were proposed in this pull request?

This patch prevents the cleanup operation in FileStreamSource if the source 
files belong to the FileStreamSink. This is needed because the output of 
FileStreamSink can be read with multiple Spark queries and queries will read 
the files based on the metadata log, which won't reflect the cleanup.

To simplify the logic, the patch only takes care of the case of when the 
source path without glob pattern refers to the output directory of 
FileStreamSink, via checking FileStreamSource to see whether it leverages 
metadata directory or not to list the source files.

### Why are the changes needed?

Without this patch, if end users turn on cleanup option with the path which 
is the output of FileStreamSink, there may be out of sync between metadata and 
available files which may break other queries reading the path.

### Does this PR introduce any user-facing change?

No

### How was this patch tested?

Added UT.

Closes #26590 from HeartSaVioR/SPARK-29953.

Authored-by: Jungtaek Lim (HeartSaVioR) 
Signed-off-by: Shixiong Zhu 
---
 docs/structured-streaming-programming-guide.md |  2 +-
 .../sql/execution/streaming/FileStreamSource.scala | 17 -
 .../sql/streaming/FileStreamSourceSuite.scala  | 83 +-
 3 files changed, 81 insertions(+), 21 deletions(-)

diff --git a/docs/structured-streaming-programming-guide.md 
b/docs/structured-streaming-programming-guide.md
index 01679e5..b91b930 100644
--- a/docs/structured-streaming-programming-guide.md
+++ b/docs/structured-streaming-programming-guide.md
@@ -551,7 +551,7 @@ Here are the details of all the sources in Spark.
 When "archive" is provided, additional option 
sourceArchiveDir must be provided as well. The value of 
"sourceArchiveDir" must have 2 subdirectories (so depth of directory is greater 
than 2). e.g. /archived/here. This will ensure archived files are 
never included as new source files.
 Spark will move source files respecting their own path. For example, 
if the path of source file is /a/b/dataset.txt and the path of 
archive directory is /archived/here, file will be moved to 
/archived/here/a/b/dataset.txt.
 NOTE: Both archiving (via moving) or deleting completed files will 
introduce overhead (slow down) in each micro-batch, so you need to understand 
the cost for each operation in your file system before enabling this option. On 
the other hand, enabling this option will reduce the cost to list source files 
which can be an expensive operation.
-NOTE 2: The source path should not be used from multiple sources or 
queries when enabling this option.
+NOTE 2: The source path should not be used from multiple sources or 
queries when enabling this option. Similarly, you must ensure the source path 
doesn't match to any files in output directory of file stream sink.
 NOTE 3: Both delete and move actions are best effort. Failing to 
delete or move files will not fail the streaming query.
 
 For file-format-specific options, see the related methods in 
DataStreamReader
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala
index 35d486c..f31fb32 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala
@@ -206,6 +206,17 @@ class FileStreamSource(
   CaseInsensitiveMap(options), None).allFiles()
   }
 
+  private def setSourceHasMetadata(newValue: Option[Boolean]): Unit = newValue 
match {
+case Some(true) =>
+  if (sourceCleaner.isDefined) {
+throw new UnsupportedOperationException("Clean up source files is not 
supported when" +
+  " reading from the output directory of FileStreamSink.")
+  }
+  sourceHasMetadata = Some(true)
+case _ =>
+  sourceHasMetadata = newValue
+  }
+
   /**
* Returns a list of files found, sorted by their timestamp.
*/
@@ -216,7 +227,7 @@ class FileStreamSource(
 

[spark] branch branch-2.4 updated: Revert "[SPARK-29494][SQL] Fix for ArrayOutofBoundsException while converting string to timestamp"

2019-10-18 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch branch-2.4
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-2.4 by this push:
 new b094774  Revert "[SPARK-29494][SQL] Fix for ArrayOutofBoundsException 
while converting string to timestamp"
b094774 is described below

commit b09477415f8cbff5292066c438f65af15622
Author: Shixiong Zhu 
AuthorDate: Fri Oct 18 15:21:35 2019 -0700

Revert "[SPARK-29494][SQL] Fix for ArrayOutofBoundsException while 
converting string to timestamp"

This reverts commit 4d476ed44a36eadb0b21b88d0f6420d52a80cc9d.
---
 .../scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala| 2 +-
 .../org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala | 6 --
 2 files changed, 1 insertion(+), 7 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala
index 5deb83e..cc3fcb2 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala
@@ -378,7 +378,7 @@ object DateTimeUtils {
 i += 1
   }
 } else {
-  if (i < segments.length && (b == ':' || b == ' ')) {
+  if (b == ':' || b == ' ') {
 segments(i) = currentSegmentValue
 currentSegmentValue = 0
 i += 1
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala
index 4496ec8..abdb916 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala
@@ -581,12 +581,6 @@ class DateTimeUtilsSuite extends SparkFunSuite {
 }
   }
 
-  test("trailing characters while converting string to timestamp") {
-val s = UTF8String.fromString("2019-10-31T10:59:23Z:::")
-val time = DateTimeUtils.stringToTimestamp(s, defaultZoneId)
-assert(time == None)
-  }
-
   test("truncTimestamp") {
 def testTrunc(
 level: Int,


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



[spark] branch master updated: [SPARK-27254][SS] Cleanup complete but invalid output files in ManifestFileCommitProtocol if job is aborted

2019-09-27 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new d72f398  [SPARK-27254][SS] Cleanup complete but invalid output files 
in ManifestFileCommitProtocol if job is aborted
d72f398 is described below

commit d72f39897b00d0bbd7a4db9de281a1256fcf908d
Author: Jungtaek Lim (HeartSaVioR) 
AuthorDate: Fri Sep 27 12:35:26 2019 -0700

[SPARK-27254][SS] Cleanup complete but invalid output files in 
ManifestFileCommitProtocol if job is aborted

## What changes were proposed in this pull request?

SPARK-27210 enables ManifestFileCommitProtocol to clean up incomplete 
output files in task level if task is aborted.

This patch extends the area of cleaning up, proposes 
ManifestFileCommitProtocol to clean up complete but invalid output files in job 
level if job aborts. Please note that this works as 'best-effort', not kind of 
guarantee, as we have in HadoopMapReduceCommitProtocol.

## How was this patch tested?

Added UT.

Closes #24186 from HeartSaVioR/SPARK-27254.

Lead-authored-by: Jungtaek Lim (HeartSaVioR) 
Co-authored-by: Jungtaek Lim (HeartSaVioR) 
Signed-off-by: Shixiong Zhu 
---
 .../streaming/ManifestFileCommitProtocol.scala | 37 ++-
 .../spark/sql/streaming/FileStreamSinkSuite.scala  | 74 ++
 2 files changed, 109 insertions(+), 2 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala
index 916bd2d..f6cc811 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala
@@ -17,6 +17,7 @@
 
 package org.apache.spark.sql.execution.streaming
 
+import java.io.IOException
 import java.util.UUID
 
 import scala.collection.mutable.ArrayBuffer
@@ -43,6 +44,8 @@ class ManifestFileCommitProtocol(jobId: String, path: String)
   @transient private var fileLog: FileStreamSinkLog = _
   private var batchId: Long = _
 
+  @transient private var pendingCommitFiles: ArrayBuffer[Path] = _
+
   /**
* Sets up the manifest log output and the batch id for this job.
* Must be called before any other function.
@@ -54,13 +57,21 @@ class ManifestFileCommitProtocol(jobId: String, path: 
String)
 
   override def setupJob(jobContext: JobContext): Unit = {
 require(fileLog != null, "setupManifestOptions must be called before this 
function")
-// Do nothing
+pendingCommitFiles = new ArrayBuffer[Path]
   }
 
   override def commitJob(jobContext: JobContext, taskCommits: 
Seq[TaskCommitMessage]): Unit = {
 require(fileLog != null, "setupManifestOptions must be called before this 
function")
 val fileStatuses = 
taskCommits.flatMap(_.obj.asInstanceOf[Seq[SinkFileStatus]]).toArray
 
+// We shouldn't remove the files if they're written to the metadata:
+// `fileLog.add(batchId, fileStatuses)` could fail AFTER writing files to 
the metadata
+// as well as there could be race
+// so for the safety we clean up the list before calling anything incurs 
exception.
+// The case is uncommon and we do best effort instead of guarantee, so the 
simplicity of
+// logic here would be OK, and safe for dealing with unexpected situations.
+pendingCommitFiles.clear()
+
 if (fileLog.add(batchId, fileStatuses)) {
   logInfo(s"Committed batch $batchId")
 } else {
@@ -70,7 +81,29 @@ class ManifestFileCommitProtocol(jobId: String, path: String)
 
   override def abortJob(jobContext: JobContext): Unit = {
 require(fileLog != null, "setupManifestOptions must be called before this 
function")
-// Do nothing
+// Best effort cleanup of complete files from failed job.
+// Since the file has UUID in its filename, we are safe to try deleting 
them
+// as the file will not conflict with file with another attempt on the 
same task.
+if (pendingCommitFiles.nonEmpty) {
+  pendingCommitFiles.foreach { path =>
+try {
+  val fs = path.getFileSystem(jobContext.getConfiguration)
+  // this is to make sure the file can be seen from driver as well
+  if (fs.exists(path)) {
+fs.delete(path, false)
+  }
+} catch {
+  case e: IOException =>
+logWarning(s"Fail to remove temporary file $path, continue 
removing next.", e)
+}
+  }
+  pendingCommitFiles.clear()
+}
+  }
+
+  override def onTaskCommit(taskCommit: TaskCommitMessage): Unit = {
+pendingCommitFiles ++= taskCommit.obj.asInstanceOf[Seq[SinkFileStatus]]
+  .map(

[spark] branch master updated: [SPARK-28976][CORE] Use KeyLock to simplify MapOutputTracker.getStatuses

2019-09-05 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 84a4d3a  [SPARK-28976][CORE] Use KeyLock to simplify 
MapOutputTracker.getStatuses
84a4d3a is described below

commit 84a4d3a17ccbf7e0cb75dffbbdc20a26715f7323
Author: Shixiong Zhu 
AuthorDate: Wed Sep 4 23:20:27 2019 -0700

[SPARK-28976][CORE] Use KeyLock to simplify MapOutputTracker.getStatuses

### What changes were proposed in this pull request?

Use `KeyLock` added in #25612 to simplify `MapOutputTracker.getStatuses`. 
It also has some improvement after the refactoring:
- `InterruptedException` is no longer sallowed.
- When a shuffle block is fetched, we don't need to wake up unrelated 
sleeping threads.

### Why are the changes needed?

`MapOutputTracker.getStatuses` is pretty hard to maintain right now because 
it has a special lock mechanism which we needs to pay attention to whenever 
updating this method. As we can use `KeyLock` to hide the complexity of locking 
behind a dedicated lock class, it's better to refactor it to make it easy to 
understand and maintain.

### Does this PR introduce any user-facing change?

No

### How was this patch tested?

Existing tests.

Closes #25680 from zsxwing/getStatuses.

Authored-by: Shixiong Zhu 
Signed-off-by: Shixiong Zhu 
---
 .../scala/org/apache/spark/MapOutputTracker.scala  | 50 +-
 1 file changed, 10 insertions(+), 40 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala 
b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
index 5c820f5..d878fc5 100644
--- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
+++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
@@ -678,8 +678,11 @@ private[spark] class MapOutputTrackerWorker(conf: 
SparkConf) extends MapOutputTr
   val mapStatuses: Map[Int, Array[MapStatus]] =
 new ConcurrentHashMap[Int, Array[MapStatus]]().asScala
 
-  /** Remembers which map output locations are currently being fetched on an 
executor. */
-  private val fetching = new HashSet[Int]
+  /**
+   * A [[KeyLock]] whose key is a shuffle id to ensure there is only one 
thread fetching
+   * the same shuffle block.
+   */
+  private val fetchingLock = new KeyLock[Int]
 
   // Get blocks sizes by executor Id. Note that zero-sized blocks are excluded 
in the result.
   override def getMapSizesByExecutorId(shuffleId: Int, startPartition: Int, 
endPartition: Int)
@@ -707,51 +710,18 @@ private[spark] class MapOutputTrackerWorker(conf: 
SparkConf) extends MapOutputTr
 if (statuses == null) {
   logInfo("Don't have map outputs for shuffle " + shuffleId + ", fetching 
them")
   val startTimeNs = System.nanoTime()
-  var fetchedStatuses: Array[MapStatus] = null
-  fetching.synchronized {
-// Someone else is fetching it; wait for them to be done
-while (fetching.contains(shuffleId)) {
-  try {
-fetching.wait()
-  } catch {
-case e: InterruptedException =>
-  }
-}
-
-// Either while we waited the fetch happened successfully, or
-// someone fetched it in between the get and the fetching.synchronized.
-fetchedStatuses = mapStatuses.get(shuffleId).orNull
+  fetchingLock.withLock(shuffleId) {
+var fetchedStatuses = mapStatuses.get(shuffleId).orNull
 if (fetchedStatuses == null) {
-  // We have to do the fetch, get others to wait for us.
-  fetching += shuffleId
-}
-  }
-
-  if (fetchedStatuses == null) {
-// We won the race to fetch the statuses; do so
-logInfo("Doing the fetch; tracker endpoint = " + trackerEndpoint)
-// This try-finally prevents hangs due to timeouts:
-try {
+  logInfo("Doing the fetch; tracker endpoint = " + trackerEndpoint)
   val fetchedBytes = 
askTracker[Array[Byte]](GetMapOutputStatuses(shuffleId))
   fetchedStatuses = 
MapOutputTracker.deserializeMapStatuses(fetchedBytes)
   logInfo("Got the output locations")
   mapStatuses.put(shuffleId, fetchedStatuses)
-} finally {
-  fetching.synchronized {
-fetching -= shuffleId
-fetching.notifyAll()
-  }
 }
-  }
-  logDebug(s"Fetching map output statuses for shuffle $shuffleId took " +
-s"${TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTimeNs)} 
ms")
-
-  if (fetchedStatuses != null) {
+logDebug(s"Fetching map output statuses for shuffle $shuffleId took " +
+  s"${TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTimeNs)} 
ms&qu

[spark] branch master updated: [SPARK-3137][CORE] Replace the global TorrentBroadcast lock with fine grained KeyLock

2019-09-03 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 8980093  [SPARK-3137][CORE] Replace the global TorrentBroadcast lock 
with fine grained KeyLock
8980093 is described below

commit 89800931aa8b565335e45e1d26ff60402e46c534
Author: Shixiong Zhu 
AuthorDate: Tue Sep 3 14:09:07 2019 -0700

[SPARK-3137][CORE] Replace the global TorrentBroadcast lock with fine 
grained KeyLock

### What changes were proposed in this pull request?

This PR provides a new lock mechanism `KeyLock` to lock  with a given key. 
Also use this new lock in `TorrentBroadcast` to avoid blocking tasks from 
fetching different broadcast values.

### Why are the changes needed?

`TorrentBroadcast.readObject` uses a global lock so only one task can be 
fetching the blocks at the same time. This is not optimal if we are running 
multiple stages concurrently because they should be able to independently fetch 
their own blocks.

### Does this PR introduce any user-facing change?

No.

### How was this patch tested?

Existing tests.

Closes #25612 from zsxwing/SPARK-3137.

Authored-by: Shixiong Zhu 
Signed-off-by: Shixiong Zhu 
---
 .../apache/spark/broadcast/BroadcastManager.scala  |   9 +-
 .../apache/spark/broadcast/TorrentBroadcast.scala  |  20 ++--
 .../main/scala/org/apache/spark/util/KeyLock.scala |  69 
 .../scala/org/apache/spark/util/KeyLockSuite.scala | 118 +
 4 files changed, 207 insertions(+), 9 deletions(-)

diff --git 
a/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala 
b/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala
index ed45043..9fa4745 100644
--- a/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala
+++ b/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala
@@ -17,6 +17,7 @@
 
 package org.apache.spark.broadcast
 
+import java.util.Collections
 import java.util.concurrent.atomic.AtomicLong
 
 import scala.reflect.ClassTag
@@ -55,9 +56,11 @@ private[spark] class BroadcastManager(
 
   private val nextBroadcastId = new AtomicLong(0)
 
-  private[broadcast] val cachedValues = {
-new ReferenceMap(AbstractReferenceMap.HARD, AbstractReferenceMap.WEAK)
-  }
+  private[broadcast] val cachedValues =
+Collections.synchronizedMap(
+  new ReferenceMap(AbstractReferenceMap.HARD, AbstractReferenceMap.WEAK)
+.asInstanceOf[java.util.Map[Any, Any]]
+)
 
   def newBroadcast[T: ClassTag](value_ : T, isLocal: Boolean): Broadcast[T] = {
 val bid = nextBroadcastId.getAndIncrement()
diff --git 
a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala 
b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala
index f416be8..1379314 100644
--- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala
+++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala
@@ -31,7 +31,7 @@ import org.apache.spark.internal.{config, Logging}
 import org.apache.spark.io.CompressionCodec
 import org.apache.spark.serializer.Serializer
 import org.apache.spark.storage._
-import org.apache.spark.util.Utils
+import org.apache.spark.util.{KeyLock, Utils}
 import org.apache.spark.util.io.{ChunkedByteBuffer, 
ChunkedByteBufferOutputStream}
 
 /**
@@ -167,7 +167,7 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, 
id: Long)
   bm.getLocalBytes(pieceId) match {
 case Some(block) =>
   blocks(pid) = block
-  releaseLock(pieceId)
+  releaseBlockManagerLock(pieceId)
 case None =>
   bm.getRemoteBytes(pieceId) match {
 case Some(b) =>
@@ -215,8 +215,10 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, 
id: Long)
   }
 
   private def readBroadcastBlock(): T = Utils.tryOrIOException {
-val broadcastCache = SparkEnv.get.broadcastManager.cachedValues
-broadcastCache.synchronized {
+TorrentBroadcast.torrentBroadcastLock.withLock(broadcastId) {
+  // As we only lock based on `broadcastId`, whenever using 
`broadcastCache`, we should only
+  // touch `broadcastId`.
+  val broadcastCache = SparkEnv.get.broadcastManager.cachedValues
 
   Option(broadcastCache.get(broadcastId)).map(_.asInstanceOf[T]).getOrElse 
{
 setConf(SparkEnv.get.conf)
@@ -225,7 +227,7 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, 
id: Long)
   case Some(blockResult) =>
 if (blockResult.data.hasNext) {
   val x = blockResult.data.next().asInstanceOf[T]
-  releaseLock(broadcastId)
+  releaseBlockManagerLock(broadcastId)
 
   if (x != null) {
 broadcastCache.put(broadcastId, x)
@@ -270,7 +272,7 @@ private[

[spark] branch master updated: [SPARK-28025][SS] Fix FileContextBasedCheckpointFileManager leaking crc files

2019-08-23 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 406c533  [SPARK-28025][SS] Fix FileContextBasedCheckpointFileManager 
leaking crc files
406c533 is described below

commit 406c5331ff8937120af465219c8f443ee00a97fb
Author: Jungtaek Lim (HeartSaVioR) 
AuthorDate: Thu Aug 22 23:10:16 2019 -0700

[SPARK-28025][SS] Fix FileContextBasedCheckpointFileManager leaking crc 
files

### What changes were proposed in this pull request?

This PR fixes the leak of crc files from CheckpointFileManager when 
FileContextBasedCheckpointFileManager is being used.

Spark hits the Hadoop bug, 
[HADOOP-16255](https://issues.apache.org/jira/browse/HADOOP-16255) which seems 
to be a long-standing issue.

This is there're two `renameInternal` methods:

```
public void renameInternal(Path src, Path dst)
public void renameInternal(final Path src, final Path dst, boolean 
overwrite)
```

which should be overridden to handle all cases but ChecksumFs only 
overrides method with 2 params, so when latter is called 
FilterFs.renameInternal(...) is called instead, and it will do rename with 
RawLocalFs as underlying filesystem.

The bug is related to FileContext, so FileSystemBasedCheckpointFileManager 
is not affected.

[SPARK-17475](https://issues.apache.org/jira/browse/SPARK-17475) took a 
workaround for this bug, but 
[SPARK-23966](https://issues.apache.org/jira/browse/SPARK-23966) seemed to 
bring regression.

This PR deletes crc file as "best-effort" when renaming, as failing to 
delete crc file is not that critical to fail the task.

### Why are the changes needed?

This PR prevents crc files not being cleaned up even purging batches. Too 
many files in same directory often hurts performance, as well as each crc file 
occupies more space than its own size so possible to occupy nontrivial amount 
of space when batches go up to 10+.

### Does this PR introduce any user-facing change?

No.

### How was this patch tested?

Some unit tests are modified to check leakage of crc files.

Closes #25488 from HeartSaVioR/SPARK-28025.

Authored-by: Jungtaek Lim (HeartSaVioR) 
Signed-off-by: Shixiong Zhu 
---
 .../streaming/CheckpointFileManager.scala  | 14 ++
 .../streaming/CheckpointFileManagerSuite.scala | 16 
 .../execution/streaming/HDFSMetadataLogSuite.scala | 30 ++
 3 files changed, 55 insertions(+), 5 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManager.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManager.scala
index fe6362d..26f42b6 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManager.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManager.scala
@@ -327,6 +327,8 @@ class FileContextBasedCheckpointFileManager(path: Path, 
hadoopConf: Configuratio
   override def renameTempFile(srcPath: Path, dstPath: Path, 
overwriteIfPossible: Boolean): Unit = {
 import Options.Rename._
 fc.rename(srcPath, dstPath, if (overwriteIfPossible) OVERWRITE else NONE)
+// TODO: this is a workaround of HADOOP-16255 - remove this when 
HADOOP-16255 is resolved
+mayRemoveCrcFile(srcPath)
   }
 
 
@@ -343,5 +345,17 @@ class FileContextBasedCheckpointFileManager(path: Path, 
hadoopConf: Configuratio
 case _: LocalFs | _: RawLocalFs => true // LocalFs = RawLocalFs + 
ChecksumFs
 case _ => false
   }
+
+  private def mayRemoveCrcFile(path: Path): Unit = {
+try {
+  val checksumFile = new Path(path.getParent, s".${path.getName}.crc")
+  if (exists(checksumFile)) {
+// checksum file exists, deleting it
+delete(checksumFile)
+  }
+} catch {
+  case NonFatal(_) => // ignore, we are removing crc file as "best-effort"
+}
+  }
 }
 
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManagerSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManagerSuite.scala
index c57b40c..79bcd49 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManagerSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManagerSuite.scala
@@ -78,6 +78,22 @@ abstract class CheckpointFileManagerTests extends 
SparkFunSuite with SQLHelper {
   assert(fm.exists(path))
   fm.createAtomic(path, overwriteIfPossible = true).close()  // should not 
throw exception
 
+  // crc file should not be leaked when origin file does

[spark] branch branch-2.4 updated: [SPARK-28650][SS][DOC] Correct explanation of guarantee for ForeachWriter

2019-08-20 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch branch-2.4
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-2.4 by this push:
 new aff5e2b  [SPARK-28650][SS][DOC] Correct explanation of guarantee for 
ForeachWriter
aff5e2b is described below

commit aff5e2bdca501fc24fb7d56f966d933c96a37b5b
Author: Jungtaek Lim (HeartSaVioR) 
AuthorDate: Tue Aug 20 00:56:53 2019 -0700

[SPARK-28650][SS][DOC] Correct explanation of guarantee for ForeachWriter

#  What changes were proposed in this pull request?

This patch modifies the explanation of guarantee for ForeachWriter as it 
doesn't guarantee same output for `(partitionId, epochId)`. Refer the 
description of [SPARK-28650](https://issues.apache.org/jira/browse/SPARK-28650) 
for more details.

Spark itself still guarantees same output for same epochId (batch) if the 
preconditions are met, 1) source is always providing the same input records for 
same offset request. 2) the query is idempotent in overall (indeterministic 
calculation like now(), random() can break this).

Assuming breaking preconditions as an exceptional case (the preconditions 
are implicitly required even before), we still can describe the guarantee with 
`epochId`, though it will be  harder to leverage the guarantee: 1) 
ForeachWriter should implement a feature to track whether all the partitions 
are written successfully for given `epochId` 2) There's pretty less chance to 
leverage the fact, as the chance for Spark to successfully write all partitions 
and fail to checkpoint the batch i [...]

Credit to zsxwing on discovering the broken guarantee.

## How was this patch tested?

This is just a documentation change, both on javadoc and guide doc.

Closes #25407 from HeartSaVioR/SPARK-28650.

Authored-by: Jungtaek Lim (HeartSaVioR) 
Signed-off-by: Shixiong Zhu 
(cherry picked from commit b37c8d5cea2e31e7821d848e42277f8fb7b68f30)
Signed-off-by: Shixiong Zhu 
---
 docs/structured-streaming-programming-guide.md | 14 ++
 .../main/scala/org/apache/spark/sql/ForeachWriter.scala| 13 +
 2 files changed, 11 insertions(+), 16 deletions(-)

diff --git a/docs/structured-streaming-programming-guide.md 
b/docs/structured-streaming-programming-guide.md
index 2c3348a..fa5664d 100644
--- a/docs/structured-streaming-programming-guide.md
+++ b/docs/structured-streaming-programming-guide.md
@@ -1827,7 +1827,7 @@ Here are the details of all the sinks in Spark.
 Foreach Sink
 Append, Update, Complete
 None
-Depends on ForeachWriter implementation
+Yes (at-least-once)
 More details in the next 
section
   
   
@@ -2235,13 +2235,11 @@ When the streaming query is started, Spark calls the 
function or the object’s
 
 - The close() method (if it exists) is called if an open() method exists and 
returns successfully (irrespective of the return value), except if the JVM or 
Python process crashes in the middle.
 
-- **Note:** The partitionId and epochId in the open() method can be used to 
deduplicate generated data 
-  when failures cause reprocessing of some input data. This depends on the 
execution mode of the query. 
-  If the streaming query is being executed in the micro-batch mode, then every 
partition represented 
-  by a unique tuple (partition_id, epoch_id) is guaranteed to have the same 
data. 
-  Hence, (partition_id, epoch_id) can be used to deduplicate and/or 
transactionally commit 
-  data and achieve exactly-once guarantees. However, if the streaming query is 
being executed 
-  in the continuous mode, then this guarantee does not hold and therefore 
should not be used for deduplication.
+- **Note:** Spark does not guarantee same output for (partitionId, epochId), 
so deduplication
+  cannot be achieved with (partitionId, epochId). e.g. source provides 
different number of
+  partitions for some reasons, Spark optimization changes number of 
partitions, etc.
+  See [SPARK-28650](https://issues.apache.org/jira/browse/SPARK-28650) for 
more details.
+  If you need deduplication on output, try out `foreachBatch` instead.
 
  Triggers
 The trigger settings of a streaming query defines the timing of streaming data 
processing, whether
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/ForeachWriter.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/ForeachWriter.scala
index 52b8c83..5cf294e 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/ForeachWriter.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/ForeachWriter.scala
@@ -50,14 +50,11 @@ import org.apache.spark.annotation.InterfaceStability
  *
  * Important points to note:
  * 
- * The `partitionId` and `epochId` can be used to deduplicate generated 
data when failures
- * cause reprocessing of some input data. This depends on the execution 
mode

[spark] branch master updated: [SPARK-28650][SS][DOC] Correct explanation of guarantee for ForeachWriter

2019-08-20 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new b37c8d5  [SPARK-28650][SS][DOC] Correct explanation of guarantee for 
ForeachWriter
b37c8d5 is described below

commit b37c8d5cea2e31e7821d848e42277f8fb7b68f30
Author: Jungtaek Lim (HeartSaVioR) 
AuthorDate: Tue Aug 20 00:56:53 2019 -0700

[SPARK-28650][SS][DOC] Correct explanation of guarantee for ForeachWriter

#  What changes were proposed in this pull request?

This patch modifies the explanation of guarantee for ForeachWriter as it 
doesn't guarantee same output for `(partitionId, epochId)`. Refer the 
description of [SPARK-28650](https://issues.apache.org/jira/browse/SPARK-28650) 
for more details.

Spark itself still guarantees same output for same epochId (batch) if the 
preconditions are met, 1) source is always providing the same input records for 
same offset request. 2) the query is idempotent in overall (indeterministic 
calculation like now(), random() can break this).

Assuming breaking preconditions as an exceptional case (the preconditions 
are implicitly required even before), we still can describe the guarantee with 
`epochId`, though it will be  harder to leverage the guarantee: 1) 
ForeachWriter should implement a feature to track whether all the partitions 
are written successfully for given `epochId` 2) There's pretty less chance to 
leverage the fact, as the chance for Spark to successfully write all partitions 
and fail to checkpoint the batch i [...]

Credit to zsxwing on discovering the broken guarantee.

## How was this patch tested?

This is just a documentation change, both on javadoc and guide doc.

Closes #25407 from HeartSaVioR/SPARK-28650.

Authored-by: Jungtaek Lim (HeartSaVioR) 
Signed-off-by: Shixiong Zhu 
---
 docs/structured-streaming-programming-guide.md | 14 ++
 .../main/scala/org/apache/spark/sql/ForeachWriter.scala| 13 +
 2 files changed, 11 insertions(+), 16 deletions(-)

diff --git a/docs/structured-streaming-programming-guide.md 
b/docs/structured-streaming-programming-guide.md
index e07a0e5..b0d3e16 100644
--- a/docs/structured-streaming-programming-guide.md
+++ b/docs/structured-streaming-programming-guide.md
@@ -1843,7 +1843,7 @@ Here are the details of all the sinks in Spark.
 Foreach Sink
 Append, Update, Complete
 None
-Depends on ForeachWriter implementation
+Yes (at-least-once)
 More details in the next 
section
   
   
@@ -2251,13 +2251,11 @@ When the streaming query is started, Spark calls the 
function or the object’s
 
 - The close() method (if it exists) is called if an open() method exists and 
returns successfully (irrespective of the return value), except if the JVM or 
Python process crashes in the middle.
 
-- **Note:** The partitionId and epochId in the open() method can be used to 
deduplicate generated data 
-  when failures cause reprocessing of some input data. This depends on the 
execution mode of the query. 
-  If the streaming query is being executed in the micro-batch mode, then every 
partition represented 
-  by a unique tuple (partition_id, epoch_id) is guaranteed to have the same 
data. 
-  Hence, (partition_id, epoch_id) can be used to deduplicate and/or 
transactionally commit 
-  data and achieve exactly-once guarantees. However, if the streaming query is 
being executed 
-  in the continuous mode, then this guarantee does not hold and therefore 
should not be used for deduplication.
+- **Note:** Spark does not guarantee same output for (partitionId, epochId), 
so deduplication
+  cannot be achieved with (partitionId, epochId). e.g. source provides 
different number of
+  partitions for some reasons, Spark optimization changes number of 
partitions, etc.
+  See [SPARK-28650](https://issues.apache.org/jira/browse/SPARK-28650) for 
more details.
+  If you need deduplication on output, try out `foreachBatch` instead.
 
  Triggers
 The trigger settings of a streaming query define the timing of streaming data 
processing, whether
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/ForeachWriter.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/ForeachWriter.scala
index 5c0fe79..a0b0a34 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/ForeachWriter.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/ForeachWriter.scala
@@ -50,14 +50,11 @@ import org.apache.spark.annotation.Evolving
  *
  * Important points to note:
  * 
- * The `partitionId` and `epochId` can be used to deduplicate generated 
data when failures
- * cause reprocessing of some input data. This depends on the execution 
mode of the query. If
- * the streaming query is being executed in the micro-batch mode, then 
every partition

[spark] branch master updated: [SPARK-28574][CORE] Allow to config different sizes for event queues

2019-08-02 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new c212c9d  [SPARK-28574][CORE] Allow to config different sizes for event 
queues
c212c9d is described below

commit c212c9d9ed7375cd1ea16c118733edd84037ec0d
Author: yunzoud 
AuthorDate: Fri Aug 2 15:27:33 2019 -0700

[SPARK-28574][CORE] Allow to config different sizes for event queues

## What changes were proposed in this pull request?
Add configuration spark.scheduler.listenerbus.eventqueue.${name}.capacity 
to allow configuration of different event queue size.

## How was this patch tested?
Unit test in 
core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala

Closes #25307 from yunzoud/SPARK-28574.

Authored-by: yunzoud 
Signed-off-by: Shixiong Zhu 
---
 .../apache/spark/scheduler/AsyncEventQueue.scala   | 14 +--
 .../apache/spark/scheduler/LiveListenerBus.scala   |  4 
 .../spark/scheduler/SparkListenerSuite.scala   | 28 ++
 3 files changed, 44 insertions(+), 2 deletions(-)

diff --git 
a/core/src/main/scala/org/apache/spark/scheduler/AsyncEventQueue.scala 
b/core/src/main/scala/org/apache/spark/scheduler/AsyncEventQueue.scala
index 7cd2b86..11e2c47 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/AsyncEventQueue.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/AsyncEventQueue.scala
@@ -46,8 +46,18 @@ private class AsyncEventQueue(
 
   // Cap the capacity of the queue so we get an explicit error (rather than an 
OOM exception) if
   // it's perpetually being added to more quickly than it's being drained.
-  private val eventQueue = new LinkedBlockingQueue[SparkListenerEvent](
-conf.get(LISTENER_BUS_EVENT_QUEUE_CAPACITY))
+  // The capacity can be configured by 
spark.scheduler.listenerbus.eventqueue.${name}.capacity,
+  // if no such conf is specified, use the value specified in
+  // LISTENER_BUS_EVENT_QUEUE_CAPACITY
+  private[scheduler] def capacity: Int = {
+val queuesize = 
conf.getInt(s"spark.scheduler.listenerbus.eventqueue.${name}.capacity",
+conf.get(LISTENER_BUS_EVENT_QUEUE_CAPACITY))
+assert(queuesize > 0, s"capacity for event queue $name must be greater 
than 0, " +
+  s"but $queuesize is configured.")
+queuesize
+  }
+
+  private val eventQueue = new 
LinkedBlockingQueue[SparkListenerEvent](capacity)
 
   // Keep the event count separately, so that waitUntilEmpty() can be 
implemented properly;
   // this allows that method to return only when the events in the queue have 
been fully
diff --git 
a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala 
b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala
index d135190..302ebd3 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala
@@ -236,6 +236,10 @@ private[spark] class LiveListenerBus(conf: SparkConf) {
 queues.asScala.map(_.name).toSet
   }
 
+  // For testing only.
+  private[scheduler] def getQueueCapacity(name: String): Option[Int] = {
+queues.asScala.find(_.name == name).map(_.capacity)
+  }
 }
 
 private[spark] object LiveListenerBus {
diff --git 
a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala 
b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala
index a7869d3..8903e10 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala
@@ -532,6 +532,34 @@ class SparkListenerSuite extends SparkFunSuite with 
LocalSparkContext with Match
 }
   }
 
+  test("event queue size can be configued through spark conf") {
+// configure the shared queue size to be 1, event log queue size to be 2,
+// and listner bus event queue size to be 5
+val conf = new SparkConf(false)
+  .set(LISTENER_BUS_EVENT_QUEUE_CAPACITY, 5)
+  .set(s"spark.scheduler.listenerbus.eventqueue.${SHARED_QUEUE}.capacity", 
"1")
+  
.set(s"spark.scheduler.listenerbus.eventqueue.${EVENT_LOG_QUEUE}.capacity", "2")
+
+val bus = new LiveListenerBus(conf)
+val counter1 = new BasicJobCounter()
+val counter2 = new BasicJobCounter()
+val counter3 = new BasicJobCounter()
+
+// add a new shared, status and event queue
+bus.addToSharedQueue(counter1)
+bus.addToStatusQueue(counter2)
+bus.addToEventLogQueue(counter3)
+
+assert(bus.activeQueues() === Set(SHARED_QUEUE, APP_STATUS_QUEUE, 
EVENT_LOG_QUEUE))
+// check the size of shared queue is 1 as configured
+assert(bus.getQueueCapacity(SHARED_QUEUE) == So

[spark] branch master updated: [SPARK-20547][REPL] Throw RemoteClassLoadedError for transient errors in ExecutorClassLoader

2019-05-28 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 04f142d  [SPARK-20547][REPL] Throw RemoteClassLoadedError for 
transient errors in ExecutorClassLoader
04f142d is described below

commit 04f142db9c4f87699053eb3aa777c08aca57b524
Author: Shixiong Zhu 
AuthorDate: Tue May 28 12:56:14 2019 -0700

[SPARK-20547][REPL] Throw RemoteClassLoadedError for transient errors in 
ExecutorClassLoader

## What changes were proposed in this pull request?

`ExecutorClassLoader`'s `findClass` may fail to fetch a class due to 
transient exceptions. For example, when a task is interrupted, if 
`ExecutorClassLoader` is fetching a class, you may see `InterruptedException` 
or `IOException` wrapped by `ClassNotFoundException`, even if this class can be 
loaded. Then the result of `findClass` will be cached by JVM, and later when 
the same class is being loaded in the same executor, it will just throw 
NoClassDefFoundError even if the class can be loaded.

I found JVM only caches `LinkageError` and `ClassNotFoundException`. Hence 
in this PR, I changed ExecutorClassLoader to throw `RemoteClassLoadedError` if 
we cannot get a response from driver.

## How was this patch tested?

New unit tests.

Closes #24683 from zsxwing/SPARK-20547-fix.

Authored-by: Shixiong Zhu 
Signed-off-by: Shixiong Zhu 
---
 .../network/server/TransportRequestHandler.java|   2 +
 .../apache/spark/repl/ExecutorClassLoader.scala|  45 ++-
 .../spark/repl/ExecutorClassLoaderSuite.scala  | 145 -
 .../scala/org/apache/spark/repl/ReplSuite.scala|  17 ++-
 .../org/apache/spark/repl/SingletonReplSuite.scala |  16 +++
 5 files changed, 214 insertions(+), 11 deletions(-)

diff --git 
a/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java
 
b/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java
index 3e089b4..0792b58 100644
--- 
a/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java
+++ 
b/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java
@@ -140,6 +140,8 @@ public class TransportRequestHandler extends 
MessageHandler {
 streamManager.streamSent(req.streamId);
   });
 } else {
+  // org.apache.spark.repl.ExecutorClassLoader.STREAM_NOT_FOUND_REGEX 
should also be updated
+  // when the following error message is changed.
   respond(new StreamFailure(req.streamId, String.format(
 "Stream '%s' was not found.", req.streamId)));
 }
diff --git 
a/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala 
b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala
index 177bce2..0cfd961 100644
--- a/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala
+++ b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala
@@ -21,6 +21,8 @@ import java.io.{ByteArrayOutputStream, FileNotFoundException, 
FilterInputStream,
 import java.net.{URI, URL, URLEncoder}
 import java.nio.channels.Channels
 
+import scala.util.control.NonFatal
+
 import org.apache.hadoop.fs.{FileSystem, Path}
 import org.apache.xbean.asm7._
 import org.apache.xbean.asm7.Opcodes._
@@ -106,7 +108,17 @@ class ExecutorClassLoader(
 parentLoader.loadClass(name)
   } catch {
 case e: ClassNotFoundException =>
-  val classOption = findClassLocally(name)
+  val classOption = try {
+findClassLocally(name)
+  } catch {
+case e: RemoteClassLoaderError =>
+  throw e
+case NonFatal(e) =>
+  // Wrap the error to include the class name
+  // scalastyle:off throwerror
+  throw new RemoteClassLoaderError(name, e)
+  // scalastyle:on throwerror
+  }
   classOption match {
 case None => throw new ClassNotFoundException(name, e)
 case Some(a) => a
@@ -115,14 +127,15 @@ class ExecutorClassLoader(
 }
   }
 
+  // See 
org.apache.spark.network.server.TransportRequestHandler.processStreamRequest.
+  private val STREAM_NOT_FOUND_REGEX = s"Stream '.*' was not found.".r.pattern
+
   private def getClassFileInputStreamFromSparkRPC(path: String): InputStream = 
{
-val channel = env.rpcEnv.openChannel(s"$classUri/$path")
+val channel = env.rpcEnv.openChannel(s"$classUri/${urlEncode(path)}")
 new FilterInputStream(Channels.newInputStream(channel)) {
 
   override def read(): Int = toClassNotFound(super.read())
 
-  override def read(b: Array[Byte]): Int = toClassNotFound(super.read(b))
-
   override def read(b: Array[B

[spark] branch branch-2.4 updated: [SPARK-27394][WEBUI] Flush LiveEntity if necessary when receiving SparkListenerExecutorMetricsUpdate (backport 2.4)

2019-04-10 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch branch-2.4
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-2.4 by this push:
 new a8a2ba1  [SPARK-27394][WEBUI] Flush LiveEntity if necessary when 
receiving SparkListenerExecutorMetricsUpdate (backport 2.4)
a8a2ba1 is described below

commit a8a2ba11ac10051423e58920062b50f328b06421
Author: Shixiong Zhu 
AuthorDate: Wed Apr 10 15:17:04 2019 -0700

[SPARK-27394][WEBUI] Flush LiveEntity if necessary when receiving 
SparkListenerExecutorMetricsUpdate (backport 2.4)

## What changes were proposed in this pull request?

This PR backports #24303 to 2.4.

## How was this patch tested?

Jenkins

Closes #24328 from zsxwing/SPARK-27394-2.4.

Authored-by: Shixiong Zhu 
Signed-off-by: Shixiong Zhu 
---
 .../apache/spark/status/AppStatusListener.scala| 40 --
 .../scala/org/apache/spark/status/config.scala |  6 
 .../org/apache/spark/ui/UISeleniumSuite.scala  | 35 +--
 docs/configuration.md  |  8 +
 4 files changed, 75 insertions(+), 14 deletions(-)

diff --git 
a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala 
b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala
index c4dd47d..cb7ab7f 100644
--- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala
+++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala
@@ -58,6 +58,12 @@ private[spark] class AppStatusListener(
   // operations that we can live without when rapidly processing incoming task 
events.
   private val liveUpdatePeriodNs = if (live) 
conf.get(LIVE_ENTITY_UPDATE_PERIOD) else -1L
 
+  /**
+   * Minimum time elapsed before stale UI data is flushed. This avoids UI 
staleness when incoming
+   * task events are not fired frequently.
+   */
+  private val liveUpdateMinFlushPeriod = 
conf.get(LIVE_ENTITY_UPDATE_MIN_FLUSH_PERIOD)
+
   private val maxTasksPerStage = conf.get(MAX_RETAINED_TASKS_PER_STAGE)
   private val maxGraphRootNodes = conf.get(MAX_RETAINED_ROOT_NODES)
 
@@ -73,6 +79,9 @@ private[spark] class AppStatusListener(
   // around liveExecutors.
   @volatile private var activeExecutorCount = 0
 
+  /** The last time when flushing `LiveEntity`s. This is to avoid flushing too 
frequently. */
+  private var lastFlushTimeNs = System.nanoTime()
+
   kvstore.addTrigger(classOf[ExecutorSummaryWrapper], 
conf.get(MAX_RETAINED_DEAD_EXECUTORS))
 { count => cleanupExecutors(count) }
 
@@ -86,7 +95,8 @@ private[spark] class AppStatusListener(
 
   kvstore.onFlush {
 if (!live) {
-  flush()
+  val now = System.nanoTime()
+  flush(update(_, now))
 }
   }
 
@@ -744,6 +754,15 @@ private[spark] class AppStatusListener(
 }
   }
 }
+
+// Flush updates if necessary. Executor heartbeat is an event that happens 
periodically. Flush
+// here to ensure the staleness of Spark UI doesn't last more than
+// `max(heartbeat interval, liveUpdateMinFlushPeriod)`.
+if (now - lastFlushTimeNs > liveUpdateMinFlushPeriod) {
+  flush(maybeUpdate(_, now))
+  // Re-get the current system time because `flush` may be slow and `now` 
is stale.
+  lastFlushTimeNs = System.nanoTime()
+}
   }
 
   override def onBlockUpdated(event: SparkListenerBlockUpdated): Unit = {
@@ -755,18 +774,17 @@ private[spark] class AppStatusListener(
 }
   }
 
-  /** Flush all live entities' data to the underlying store. */
-  private def flush(): Unit = {
-val now = System.nanoTime()
+  /** Go through all `LiveEntity`s and use `entityFlushFunc(entity)` to flush 
them. */
+  private def flush(entityFlushFunc: LiveEntity => Unit): Unit = {
 liveStages.values.asScala.foreach { stage =>
-  update(stage, now)
-  stage.executorSummaries.values.foreach(update(_, now))
+  entityFlushFunc(stage)
+  stage.executorSummaries.values.foreach(entityFlushFunc)
 }
-liveJobs.values.foreach(update(_, now))
-liveExecutors.values.foreach(update(_, now))
-liveTasks.values.foreach(update(_, now))
-liveRDDs.values.foreach(update(_, now))
-pools.values.foreach(update(_, now))
+liveJobs.values.foreach(entityFlushFunc)
+liveExecutors.values.foreach(entityFlushFunc)
+liveTasks.values.foreach(entityFlushFunc)
+liveRDDs.values.foreach(entityFlushFunc)
+pools.values.foreach(entityFlushFunc)
   }
 
   /**
diff --git a/core/src/main/scala/org/apache/spark/status/config.scala 
b/core/src/main/scala/org/apache/spark/status/config.scala
index 67801b8..87204fd 100644
--- a/core/src/main/scala/org/apache/spark/status/config.scala
+++ b/core/src/main/scala/org/apache/spark/status/config.scala
@@ -31,6 +31,12 @@ private[spark] object config {
 .timeConf(TimeUnit.NANOSECONDS)
 .createWithDefaultString("

[spark] branch master updated: [SPARK-27210][SS] Cleanup incomplete output files in ManifestFileCommitProtocol if task is aborted

2019-03-22 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 78d546f  [SPARK-27210][SS] Cleanup incomplete output files in 
ManifestFileCommitProtocol if task is aborted
78d546f is described below

commit 78d546fe15aebcbf4b671c44383ddcf82b05b8a7
Author: Jungtaek Lim (HeartSaVioR) 
AuthorDate: Fri Mar 22 11:26:53 2019 -0700

[SPARK-27210][SS] Cleanup incomplete output files in 
ManifestFileCommitProtocol if task is aborted

## What changes were proposed in this pull request?

This patch proposes ManifestFileCommitProtocol to clean up incomplete 
output files in task level if task aborts. Please note that this works as 
'best-effort', not kind of guarantee, as we have in 
HadoopMapReduceCommitProtocol.

## How was this patch tested?

Added UT.

Closes #24154 from HeartSaVioR/SPARK-27210.

Authored-by: Jungtaek Lim (HeartSaVioR) 
Signed-off-by: Shixiong Zhu 
---
 .../streaming/ManifestFileCommitProtocol.scala |  7 --
 .../spark/sql/streaming/FileStreamSinkSuite.scala  | 29 ++
 2 files changed, 34 insertions(+), 2 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala
index 92191c8..916bd2d 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala
@@ -114,7 +114,10 @@ class ManifestFileCommitProtocol(jobId: String, path: 
String)
   }
 
   override def abortTask(taskContext: TaskAttemptContext): Unit = {
-// Do nothing
-// TODO: we can also try delete the addedFiles as a best-effort cleanup.
+// best effort cleanup of incomplete files
+if (addedFiles.nonEmpty) {
+  val fs = new 
Path(addedFiles.head).getFileSystem(taskContext.getConfiguration)
+  addedFiles.foreach { file => fs.delete(new Path(file), false) }
+}
   }
 }
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala
index 619d118..020ab23 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala
@@ -18,8 +18,11 @@
 package org.apache.spark.sql.streaming
 
 import java.io.File
+import java.nio.file.Files
 import java.util.Locale
 
+import scala.collection.JavaConverters._
+
 import org.apache.hadoop.fs.Path
 
 import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd}
@@ -478,4 +481,30 @@ class FileStreamSinkSuite extends StreamTest {
   checkDatasetUnorderly(outputDf, 1, 2, 3)
 }
   }
+
+  testQuietly("cleanup incomplete output for aborted task") {
+withTempDir { tempDir =>
+  val checkpointDir = new File(tempDir, "chk")
+  val outputDir = new File(tempDir, "output")
+  val inputData = MemoryStream[Int]
+  inputData.addData(1, 2, 3)
+  val q = inputData.toDS().map(_ / 0)
+.writeStream
+.option("checkpointLocation", checkpointDir.getCanonicalPath)
+.format("parquet")
+.start(outputDir.getCanonicalPath)
+
+  intercept[StreamingQueryException] {
+try {
+  q.processAllAvailable()
+} finally {
+  q.stop()
+}
+  }
+
+  val outputFiles = Files.walk(outputDir.toPath).iterator().asScala
+.filter(_.toString.endsWith(".parquet"))
+  assert(outputFiles.toList.isEmpty, "Incomplete files should be cleaned 
up.")
+}
+  }
 }


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



[spark] branch branch-2.3 updated: [SPARK-27111][SS] Fix a race that a continuous query may fail with InterruptedException

2019-03-09 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch branch-2.3
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-2.3 by this push:
 new 4d1d0a4  [SPARK-27111][SS] Fix a race that a continuous query may fail 
with InterruptedException
4d1d0a4 is described below

commit 4d1d0a41a862c234acb9b8b68e96da7bf079eb8d
Author: Shixiong Zhu 
AuthorDate: Sat Mar 9 14:26:58 2019 -0800

[SPARK-27111][SS] Fix a race that a continuous query may fail with 
InterruptedException

Before a Kafka consumer gets assigned with partitions, its offset will 
contain 0 partitions. However, runContinuous will still run and launch a Spark 
job having 0 partitions. In this case, there is a race that epoch may interrupt 
the query execution thread after `lastExecution.toRdd`, and either 
`epochEndpoint.askSync[Unit](StopContinuousExecutionWrites)` or the next 
`runContinuous` will get interrupted unintentionally.

To handle this case, this PR has the following changes:

- Clean up the resources in `queryExecutionThread.runUninterruptibly`. This 
may increase the waiting time of `stop` but should be minor because the 
operations here are very fast (just sending an RPC message in the same process 
and stopping a very simple thread).
- Clear the interrupted status at the end so that it won't impact the 
`runContinuous` call. We may clear the interrupted status set by `stop`, but it 
doesn't affect the query termination because `runActivatedStream` will check 
`state` and exit accordingly.

I also updated the clean up codes to make sure exceptions thrown from 
`epochEndpoint.askSync[Unit](StopContinuousExecutionWrites)` won't stop the 
clean up.

Jenkins

Closes #24034 from zsxwing/SPARK-27111.

Authored-by: Shixiong Zhu 
Signed-off-by: Shixiong Zhu 
(cherry picked from commit 6e1c0827ece1cdc615196e60cb11c76b917b8eeb)
Signed-off-by: Shixiong Zhu 
---
 .../streaming/continuous/ContinuousExecution.scala | 32 --
 1 file changed, 24 insertions(+), 8 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala
index 62adedb..dad7f9b 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala
@@ -270,14 +270,30 @@ class ContinuousExecution(
 logInfo(s"Query $id ignoring exception from reconfiguring: $t")
 // interrupted by reconfiguration - swallow exception so we can 
restart the query
 } finally {
-  epochEndpoint.askSync[Unit](StopContinuousExecutionWrites)
-  SparkEnv.get.rpcEnv.stop(epochEndpoint)
-
-  epochUpdateThread.interrupt()
-  epochUpdateThread.join()
-
-  stopSources()
-  sparkSession.sparkContext.cancelJobGroup(runId.toString)
+  // The above execution may finish before getting interrupted, for 
example, a Spark job having
+  // 0 partitions will complete immediately. Then the interrupted status 
will sneak here.
+  //
+  // To handle this case, we do the two things here:
+  //
+  // 1. Clean up the resources in 
`queryExecutionThread.runUninterruptibly`. This may increase
+  //the waiting time of `stop` but should be minor because the 
operations here are very fast
+  //(just sending an RPC message in the same process and stopping a 
very simple thread).
+  // 2. Clear the interrupted status at the end so that it won't impact 
the `runContinuous`
+  //call. We may clear the interrupted status set by `stop`, but it 
doesn't affect the query
+  //termination because `runActivatedStream` will check `state` and 
exit accordingly.
+  queryExecutionThread.runUninterruptibly {
+try {
+  epochEndpoint.askSync[Unit](StopContinuousExecutionWrites)
+} finally {
+  SparkEnv.get.rpcEnv.stop(epochEndpoint)
+  epochUpdateThread.interrupt()
+  epochUpdateThread.join()
+  stopSources()
+  // The following line must be the last line because it may fail if 
SparkContext is stopped
+  sparkSession.sparkContext.cancelJobGroup(runId.toString)
+}
+  }
+  Thread.interrupted()
 }
   }
 


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



[spark] branch branch-2.4 updated: [SPARK-27111][SS] Fix a race that a continuous query may fail with InterruptedException

2019-03-09 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch branch-2.4
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-2.4 by this push:
 new 53590f2  [SPARK-27111][SS] Fix a race that a continuous query may fail 
with InterruptedException
53590f2 is described below

commit 53590f275a7ebcd015120b576905ce999e50331e
Author: Shixiong Zhu 
AuthorDate: Sat Mar 9 14:26:58 2019 -0800

[SPARK-27111][SS] Fix a race that a continuous query may fail with 
InterruptedException

Before a Kafka consumer gets assigned with partitions, its offset will 
contain 0 partitions. However, runContinuous will still run and launch a Spark 
job having 0 partitions. In this case, there is a race that epoch may interrupt 
the query execution thread after `lastExecution.toRdd`, and either 
`epochEndpoint.askSync[Unit](StopContinuousExecutionWrites)` or the next 
`runContinuous` will get interrupted unintentionally.

To handle this case, this PR has the following changes:

- Clean up the resources in `queryExecutionThread.runUninterruptibly`. This 
may increase the waiting time of `stop` but should be minor because the 
operations here are very fast (just sending an RPC message in the same process 
and stopping a very simple thread).
- Clear the interrupted status at the end so that it won't impact the 
`runContinuous` call. We may clear the interrupted status set by `stop`, but it 
doesn't affect the query termination because `runActivatedStream` will check 
`state` and exit accordingly.

I also updated the clean up codes to make sure exceptions thrown from 
`epochEndpoint.askSync[Unit](StopContinuousExecutionWrites)` won't stop the 
clean up.

Jenkins

Closes #24034 from zsxwing/SPARK-27111.

Authored-by: Shixiong Zhu 
Signed-off-by: Shixiong Zhu 
(cherry picked from commit 6e1c0827ece1cdc615196e60cb11c76b917b8eeb)
Signed-off-by: Shixiong Zhu 
---
 .../streaming/continuous/ContinuousExecution.scala | 32 --
 1 file changed, 24 insertions(+), 8 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala
index 2e24fa6..3037c01 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala
@@ -272,14 +272,30 @@ class ContinuousExecution(
 logInfo(s"Query $id ignoring exception from reconfiguring: $t")
 // interrupted by reconfiguration - swallow exception so we can 
restart the query
 } finally {
-  epochEndpoint.askSync[Unit](StopContinuousExecutionWrites)
-  SparkEnv.get.rpcEnv.stop(epochEndpoint)
-
-  epochUpdateThread.interrupt()
-  epochUpdateThread.join()
-
-  stopSources()
-  sparkSession.sparkContext.cancelJobGroup(runId.toString)
+  // The above execution may finish before getting interrupted, for 
example, a Spark job having
+  // 0 partitions will complete immediately. Then the interrupted status 
will sneak here.
+  //
+  // To handle this case, we do the two things here:
+  //
+  // 1. Clean up the resources in 
`queryExecutionThread.runUninterruptibly`. This may increase
+  //the waiting time of `stop` but should be minor because the 
operations here are very fast
+  //(just sending an RPC message in the same process and stopping a 
very simple thread).
+  // 2. Clear the interrupted status at the end so that it won't impact 
the `runContinuous`
+  //call. We may clear the interrupted status set by `stop`, but it 
doesn't affect the query
+  //termination because `runActivatedStream` will check `state` and 
exit accordingly.
+  queryExecutionThread.runUninterruptibly {
+try {
+  epochEndpoint.askSync[Unit](StopContinuousExecutionWrites)
+} finally {
+  SparkEnv.get.rpcEnv.stop(epochEndpoint)
+  epochUpdateThread.interrupt()
+  epochUpdateThread.join()
+  stopSources()
+  // The following line must be the last line because it may fail if 
SparkContext is stopped
+  sparkSession.sparkContext.cancelJobGroup(runId.toString)
+}
+  }
+  Thread.interrupted()
 }
   }
 


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



[spark] branch master updated: [SPARK-27111][SS] Fix a race that a continuous query may fail with InterruptedException

2019-03-09 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 6e1c082  [SPARK-27111][SS] Fix a race that a continuous query may fail 
with InterruptedException
6e1c082 is described below

commit 6e1c0827ece1cdc615196e60cb11c76b917b8eeb
Author: Shixiong Zhu 
AuthorDate: Sat Mar 9 14:26:58 2019 -0800

[SPARK-27111][SS] Fix a race that a continuous query may fail with 
InterruptedException

## What changes were proposed in this pull request?

Before a Kafka consumer gets assigned with partitions, its offset will 
contain 0 partitions. However, runContinuous will still run and launch a Spark 
job having 0 partitions. In this case, there is a race that epoch may interrupt 
the query execution thread after `lastExecution.toRdd`, and either 
`epochEndpoint.askSync[Unit](StopContinuousExecutionWrites)` or the next 
`runContinuous` will get interrupted unintentionally.

To handle this case, this PR has the following changes:

- Clean up the resources in `queryExecutionThread.runUninterruptibly`. This 
may increase the waiting time of `stop` but should be minor because the 
operations here are very fast (just sending an RPC message in the same process 
and stopping a very simple thread).
- Clear the interrupted status at the end so that it won't impact the 
`runContinuous` call. We may clear the interrupted status set by `stop`, but it 
doesn't affect the query termination because `runActivatedStream` will check 
`state` and exit accordingly.

I also updated the clean up codes to make sure exceptions thrown from 
`epochEndpoint.askSync[Unit](StopContinuousExecutionWrites)` won't stop the 
clean up.

## How was this patch tested?

Jenkins

Closes #24034 from zsxwing/SPARK-27111.

Authored-by: Shixiong Zhu 
Signed-off-by: Shixiong Zhu 
---
 .../streaming/continuous/ContinuousExecution.scala | 30 +-
 1 file changed, 23 insertions(+), 7 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala
index 26b5642..aef556d 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala
@@ -268,13 +268,29 @@ class ContinuousExecution(
 logInfo(s"Query $id ignoring exception from reconfiguring: $t")
 // interrupted by reconfiguration - swallow exception so we can 
restart the query
 } finally {
-  epochEndpoint.askSync[Unit](StopContinuousExecutionWrites)
-  SparkEnv.get.rpcEnv.stop(epochEndpoint)
-
-  epochUpdateThread.interrupt()
-  epochUpdateThread.join()
-
-  sparkSession.sparkContext.cancelJobGroup(runId.toString)
+  // The above execution may finish before getting interrupted, for 
example, a Spark job having
+  // 0 partitions will complete immediately. Then the interrupted status 
will sneak here.
+  //
+  // To handle this case, we do the two things here:
+  //
+  // 1. Clean up the resources in 
`queryExecutionThread.runUninterruptibly`. This may increase
+  //the waiting time of `stop` but should be minor because the 
operations here are very fast
+  //(just sending an RPC message in the same process and stopping a 
very simple thread).
+  // 2. Clear the interrupted status at the end so that it won't impact 
the `runContinuous`
+  //call. We may clear the interrupted status set by `stop`, but it 
doesn't affect the query
+  //termination because `runActivatedStream` will check `state` and 
exit accordingly.
+  queryExecutionThread.runUninterruptibly {
+try {
+  epochEndpoint.askSync[Unit](StopContinuousExecutionWrites)
+} finally {
+  SparkEnv.get.rpcEnv.stop(epochEndpoint)
+  epochUpdateThread.interrupt()
+  epochUpdateThread.join()
+  // The following line must be the last line because it may fail if 
SparkContext is stopped
+  sparkSession.sparkContext.cancelJobGroup(runId.toString)
+}
+  }
+  Thread.interrupted()
 }
   }
 


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



[spark] branch master updated: [SPARK-26824][SS] Fix the checkpoint location and _spark_metadata when it contains special chars

2019-02-20 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 77b99af  [SPARK-26824][SS] Fix the checkpoint location and 
_spark_metadata when it contains special chars
77b99af is described below

commit 77b99af57330cf2e5016a6acc69642d54041b041
Author: Shixiong Zhu 
AuthorDate: Wed Feb 20 15:44:20 2019 -0800

[SPARK-26824][SS] Fix the checkpoint location and _spark_metadata when it 
contains special chars

## What changes were proposed in this pull request?

When a user specifies a checkpoint location or a file sink output using a 
path containing special chars that need to be escaped in a path, the streaming 
query will store checkpoint and file sink metadata in a wrong place. In this 
PR, I uploaded a checkpoint that was generated by the following codes using 
Spark 2.4.0 to show this issue:

```
implicit val s = spark.sqlContext
val input = org.apache.spark.sql.execution.streaming.MemoryStream[Int]
input.addData(1, 2, 3)
val q = 
input.toDF.writeStream.format("parquet").option("checkpointLocation", ".../chk 
%#chk").start(".../output %#output")
q.stop()
```
Here is the structure of the directory:
```
sql/core/src/test/resources/structured-streaming/escaped-path-2.4.0
├── chk%252520%252525%252523chk
│   ├── commits
│   │   └── 0
│   ├── metadata
│   └── offsets
│   └── 0
├── output %#output
│   └── part-0-97f675a2-bb82-4201-8245-05f3dae4c372-c000.snappy.parquet
└── output%20%25%23output
└── _spark_metadata
└── 0
```

In this checkpoint, the user specified checkpoint location is `.../chk 
%#chk` but the real path to store the checkpoint is 
`.../chk%252520%252525%252523chk` (this is generated by escaping the original 
path three times). The user specified output path is `.../output %#output` but 
the path to store `_spark_metadata` is 
`.../output%20%25%23output/_spark_metadata` (this is generated by escaping the 
original path once). The data files are still in the correct path (such as 
`.../output %#ou [...]

This checkpoint will be used in unit tests in this PR.

The fix is just simply removing improper `Path.toUri` calls to fix the 
issue.

However, as the user may not read the release note and is not aware of this 
checkpoint location change, if they upgrade Spark without moving checkpoint to 
the new location, their query will just start from the scratch. In order to not 
surprise the users, this PR also adds a check to **detect the impacted paths 
and throws an error** to include the migration guide. This check can be turned 
off by an internal sql conf 
`spark.sql.streaming.checkpoint.escapedPathCheck.enabled`. Here are ex [...]

- Streaming checkpoint error:
```
Error: we detected a possible problem with the location of your checkpoint 
and you
likely need to move it before restarting this query.

Earlier version of Spark incorrectly escaped paths when writing out 
checkpoints for
structured streaming. While this was corrected in Spark 3.0, it appears 
that your
query was started using an earlier version that incorrectly handled the 
checkpoint
path.

Correct Checkpoint Directory: /.../chk %#chk
Incorrect Checkpoint Directory: /.../chk%252520%252525%252523chk

Please move the data from the incorrect directory to the correct one, 
delete the
incorrect directory, and then restart this query. If you believe you are 
receiving
this message in error, you can disable it with the SQL conf
spark.sql.streaming.checkpoint.escapedPathCheck.enabled.
```

- File sink error (`_spark_metadata`):
```
Error: we detected a possible problem with the location of your 
"_spark_metadata"
directory and you likely need to move it before restarting this query.

Earlier version of Spark incorrectly escaped paths when writing out the
"_spark_metadata" directory for structured streaming. While this was 
corrected in
Spark 3.0, it appears that your query was started using an earlier version 
that
incorrectly handled the "_spark_metadata" path.

Correct "_spark_metadata" Directory: /.../output %#output/_spark_metadata
Incorrect "_spark_metadata" Directory: 
/.../output%20%25%23output/_spark_metadata

Please move the data from the incorrect directory to the correct one, 
delete the
incorrect directory, and then restart this query. If you believe you are 
receiving
this message in error, you can disable it with the SQL conf
spark.sql.streaming.checkpoint.escapedPathCheck.enabled.
```

    ## How was this patch tested?

The new un

[spark] branch branch-2.2 updated: [SPARK-26806][SS] EventTimeStats.merge should handle zeros correctly

2019-02-01 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch branch-2.2
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-2.2 by this push:
 new 7c7d7f6  [SPARK-26806][SS] EventTimeStats.merge should handle zeros 
correctly
7c7d7f6 is described below

commit 7c7d7f6a878b02ece881266ee538f3e1443aa8c1
Author: Shixiong Zhu 
AuthorDate: Fri Feb 1 11:15:05 2019 -0800

[SPARK-26806][SS] EventTimeStats.merge should handle zeros correctly

## What changes were proposed in this pull request?

Right now, EventTimeStats.merge doesn't handle `zero.merge(zero)` 
correctly. This will make `avg` become `NaN`. And whatever gets merged with the 
result of `zero.merge(zero)`, `avg` will still be `NaN`. Then finally, we call 
`NaN.toLong` and get `0`, and the user will see the following incorrect report:

```
"eventTime" : {
"avg" : "1970-01-01T00:00:00.000Z",
"max" : "2019-01-31T12:57:00.000Z",
"min" : "2019-01-30T18:44:04.000Z",
"watermark" : "1970-01-01T00:00:00.000Z"
  }
```

This issue was reported by liancheng .

This PR fixes the above issue.

    ## How was this patch tested?

The new unit tests.

Closes #23718 from zsxwing/merge-zero.

Authored-by: Shixiong Zhu 
Signed-off-by: Shixiong Zhu 
(cherry picked from commit 03a928cbecaf38bbbab3e6b957fcbb542771cfbd)
Signed-off-by: Shixiong Zhu 
---
 .../streaming/EventTimeWatermarkExec.scala | 17 +---
 .../sql/streaming/EventTimeWatermarkSuite.scala| 32 --
 2 files changed, 42 insertions(+), 7 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/EventTimeWatermarkExec.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/EventTimeWatermarkExec.scala
index 55e7508..4069633 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/EventTimeWatermarkExec.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/EventTimeWatermarkExec.scala
@@ -36,10 +36,19 @@ case class EventTimeStats(var max: Long, var min: Long, var 
avg: Double, var cou
   }
 
   def merge(that: EventTimeStats): Unit = {
-this.max = math.max(this.max, that.max)
-this.min = math.min(this.min, that.min)
-this.count += that.count
-this.avg += (that.avg - this.avg) * that.count / this.count
+if (that.count == 0) {
+  // no-op
+} else if (this.count == 0) {
+  this.max = that.max
+  this.min = that.min
+  this.count = that.count
+  this.avg = that.avg
+} else {
+  this.max = math.max(this.max, that.max)
+  this.min = math.min(this.min, that.min)
+  this.count += that.count
+  this.avg += (that.avg - this.avg) * that.count / this.count
+}
   }
 }
 
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala
index 4f19fa0..14a193f 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala
@@ -38,9 +38,9 @@ class EventTimeWatermarkSuite extends StreamTest with 
BeforeAndAfter with Matche
 sqlContext.streams.active.foreach(_.stop())
   }
 
-  test("EventTimeStats") {
-val epsilon = 10E-6
+  private val epsilon = 10E-6
 
+  test("EventTimeStats") {
 val stats = EventTimeStats(max = 100, min = 10, avg = 20.0, count = 5)
 stats.add(80L)
 stats.max should be (100)
@@ -57,7 +57,6 @@ class EventTimeWatermarkSuite extends StreamTest with 
BeforeAndAfter with Matche
   }
 
   test("EventTimeStats: avg on large values") {
-val epsilon = 10E-6
 val largeValue = 100L // 10B
 // Make sure `largeValue` will cause overflow if we use a Long sum to calc 
avg.
 assert(largeValue * largeValue != BigInt(largeValue) * BigInt(largeValue))
@@ -75,6 +74,33 @@ class EventTimeWatermarkSuite extends StreamTest with 
BeforeAndAfter with Matche
 stats.avg should be ((largeValue + 0.5) +- epsilon)
   }
 
+  test("EventTimeStats: zero merge zero") {
+val stats = EventTimeStats.zero
+val stats2 = EventTimeStats.zero
+stats.merge(stats2)
+stats should be (EventTimeStats.zero)
+  }
+
+  test("EventTimeStats: non-zero merge zero") {
+val stats = EventTimeStats(max = 10, min = 1, avg = 5.0, count = 3)
+val stats2 = EventTimeStats.zero
+stats.merge(stats2)
+stats.max should be (10L)
+stats.min should be (1L)
+stats.avg should be (5.0 +- epsilon)
+stats.count should be (3L)
+  }
+
+  test("EventTimeStats: zero merge no

[spark] branch branch-2.3 updated: [SPARK-26806][SS] EventTimeStats.merge should handle zeros correctly

2019-02-01 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch branch-2.3
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-2.3 by this push:
 new a5d22da  [SPARK-26806][SS] EventTimeStats.merge should handle zeros 
correctly
a5d22da is described below

commit a5d22da1888b8110b490d52d2c36b3fc907254f6
Author: Shixiong Zhu 
AuthorDate: Fri Feb 1 11:15:05 2019 -0800

[SPARK-26806][SS] EventTimeStats.merge should handle zeros correctly

## What changes were proposed in this pull request?

Right now, EventTimeStats.merge doesn't handle `zero.merge(zero)` 
correctly. This will make `avg` become `NaN`. And whatever gets merged with the 
result of `zero.merge(zero)`, `avg` will still be `NaN`. Then finally, we call 
`NaN.toLong` and get `0`, and the user will see the following incorrect report:

```
"eventTime" : {
"avg" : "1970-01-01T00:00:00.000Z",
"max" : "2019-01-31T12:57:00.000Z",
"min" : "2019-01-30T18:44:04.000Z",
"watermark" : "1970-01-01T00:00:00.000Z"
  }
```

This issue was reported by liancheng .

This PR fixes the above issue.

    ## How was this patch tested?

The new unit tests.

Closes #23718 from zsxwing/merge-zero.

Authored-by: Shixiong Zhu 
Signed-off-by: Shixiong Zhu 
(cherry picked from commit 03a928cbecaf38bbbab3e6b957fcbb542771cfbd)
Signed-off-by: Shixiong Zhu 
---
 .../streaming/EventTimeWatermarkExec.scala | 17 +---
 .../sql/streaming/EventTimeWatermarkSuite.scala| 32 --
 2 files changed, 42 insertions(+), 7 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/EventTimeWatermarkExec.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/EventTimeWatermarkExec.scala
index b161651..6fa7ee0 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/EventTimeWatermarkExec.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/EventTimeWatermarkExec.scala
@@ -36,10 +36,19 @@ case class EventTimeStats(var max: Long, var min: Long, var 
avg: Double, var cou
   }
 
   def merge(that: EventTimeStats): Unit = {
-this.max = math.max(this.max, that.max)
-this.min = math.min(this.min, that.min)
-this.count += that.count
-this.avg += (that.avg - this.avg) * that.count / this.count
+if (that.count == 0) {
+  // no-op
+} else if (this.count == 0) {
+  this.max = that.max
+  this.min = that.min
+  this.count = that.count
+  this.avg = that.avg
+} else {
+  this.max = math.max(this.max, that.max)
+  this.min = math.min(this.min, that.min)
+  this.count += that.count
+  this.avg += (that.avg - this.avg) * that.count / this.count
+}
   }
 }
 
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala
index d6bef9c..a51f086 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala
@@ -38,9 +38,9 @@ class EventTimeWatermarkSuite extends StreamTest with 
BeforeAndAfter with Matche
 sqlContext.streams.active.foreach(_.stop())
   }
 
-  test("EventTimeStats") {
-val epsilon = 10E-6
+  private val epsilon = 10E-6
 
+  test("EventTimeStats") {
 val stats = EventTimeStats(max = 100, min = 10, avg = 20.0, count = 5)
 stats.add(80L)
 stats.max should be (100)
@@ -57,7 +57,6 @@ class EventTimeWatermarkSuite extends StreamTest with 
BeforeAndAfter with Matche
   }
 
   test("EventTimeStats: avg on large values") {
-val epsilon = 10E-6
 val largeValue = 100L // 10B
 // Make sure `largeValue` will cause overflow if we use a Long sum to calc 
avg.
 assert(largeValue * largeValue != BigInt(largeValue) * BigInt(largeValue))
@@ -75,6 +74,33 @@ class EventTimeWatermarkSuite extends StreamTest with 
BeforeAndAfter with Matche
 stats.avg should be ((largeValue + 0.5) +- epsilon)
   }
 
+  test("EventTimeStats: zero merge zero") {
+val stats = EventTimeStats.zero
+val stats2 = EventTimeStats.zero
+stats.merge(stats2)
+stats should be (EventTimeStats.zero)
+  }
+
+  test("EventTimeStats: non-zero merge zero") {
+val stats = EventTimeStats(max = 10, min = 1, avg = 5.0, count = 3)
+val stats2 = EventTimeStats.zero
+stats.merge(stats2)
+stats.max should be (10L)
+stats.min should be (1L)
+stats.avg should be (5.0 +- epsilon)
+stats.count should be (3L)
+  }
+
+  test("EventTimeStats: zero merge no

[spark] branch branch-2.4 updated: [SPARK-26806][SS] EventTimeStats.merge should handle zeros correctly

2019-02-01 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch branch-2.4
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-2.4 by this push:
 new bd4ce51  [SPARK-26806][SS] EventTimeStats.merge should handle zeros 
correctly
bd4ce51 is described below

commit bd4ce51e699da306bc36db0c7b0303b6e6c3d4df
Author: Shixiong Zhu 
AuthorDate: Fri Feb 1 11:15:05 2019 -0800

[SPARK-26806][SS] EventTimeStats.merge should handle zeros correctly

## What changes were proposed in this pull request?

Right now, EventTimeStats.merge doesn't handle `zero.merge(zero)` 
correctly. This will make `avg` become `NaN`. And whatever gets merged with the 
result of `zero.merge(zero)`, `avg` will still be `NaN`. Then finally, we call 
`NaN.toLong` and get `0`, and the user will see the following incorrect report:

```
"eventTime" : {
"avg" : "1970-01-01T00:00:00.000Z",
"max" : "2019-01-31T12:57:00.000Z",
"min" : "2019-01-30T18:44:04.000Z",
"watermark" : "1970-01-01T00:00:00.000Z"
  }
```

This issue was reported by liancheng .

This PR fixes the above issue.

    ## How was this patch tested?

The new unit tests.

Closes #23718 from zsxwing/merge-zero.

Authored-by: Shixiong Zhu 
Signed-off-by: Shixiong Zhu 
(cherry picked from commit 03a928cbecaf38bbbab3e6b957fcbb542771cfbd)
Signed-off-by: Shixiong Zhu 
---
 .../streaming/EventTimeWatermarkExec.scala | 17 +---
 .../sql/streaming/EventTimeWatermarkSuite.scala| 32 --
 2 files changed, 42 insertions(+), 7 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/EventTimeWatermarkExec.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/EventTimeWatermarkExec.scala
index b161651..6fa7ee0 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/EventTimeWatermarkExec.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/EventTimeWatermarkExec.scala
@@ -36,10 +36,19 @@ case class EventTimeStats(var max: Long, var min: Long, var 
avg: Double, var cou
   }
 
   def merge(that: EventTimeStats): Unit = {
-this.max = math.max(this.max, that.max)
-this.min = math.min(this.min, that.min)
-this.count += that.count
-this.avg += (that.avg - this.avg) * that.count / this.count
+if (that.count == 0) {
+  // no-op
+} else if (this.count == 0) {
+  this.max = that.max
+  this.min = that.min
+  this.count = that.count
+  this.avg = that.avg
+} else {
+  this.max = math.max(this.max, that.max)
+  this.min = math.min(this.min, that.min)
+  this.count += that.count
+  this.avg += (that.avg - this.avg) * that.count / this.count
+}
   }
 }
 
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala
index 026af17..091b9a1 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala
@@ -43,9 +43,9 @@ class EventTimeWatermarkSuite extends StreamTest with 
BeforeAndAfter with Matche
 sqlContext.streams.active.foreach(_.stop())
   }
 
-  test("EventTimeStats") {
-val epsilon = 10E-6
+  private val epsilon = 10E-6
 
+  test("EventTimeStats") {
 val stats = EventTimeStats(max = 100, min = 10, avg = 20.0, count = 5)
 stats.add(80L)
 stats.max should be (100)
@@ -62,7 +62,6 @@ class EventTimeWatermarkSuite extends StreamTest with 
BeforeAndAfter with Matche
   }
 
   test("EventTimeStats: avg on large values") {
-val epsilon = 10E-6
 val largeValue = 100L // 10B
 // Make sure `largeValue` will cause overflow if we use a Long sum to calc 
avg.
 assert(largeValue * largeValue != BigInt(largeValue) * BigInt(largeValue))
@@ -80,6 +79,33 @@ class EventTimeWatermarkSuite extends StreamTest with 
BeforeAndAfter with Matche
 stats.avg should be ((largeValue + 0.5) +- epsilon)
   }
 
+  test("EventTimeStats: zero merge zero") {
+val stats = EventTimeStats.zero
+val stats2 = EventTimeStats.zero
+stats.merge(stats2)
+stats should be (EventTimeStats.zero)
+  }
+
+  test("EventTimeStats: non-zero merge zero") {
+val stats = EventTimeStats(max = 10, min = 1, avg = 5.0, count = 3)
+val stats2 = EventTimeStats.zero
+stats.merge(stats2)
+stats.max should be (10L)
+stats.min should be (1L)
+stats.avg should be (5.0 +- epsilon)
+stats.count should be (3L)
+  }
+
+  test("EventTimeStats: zero merge no

[spark] branch master updated: [SPARK-26806][SS] EventTimeStats.merge should handle zeros correctly

2019-02-01 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 03a928c  [SPARK-26806][SS] EventTimeStats.merge should handle zeros 
correctly
03a928c is described below

commit 03a928cbecaf38bbbab3e6b957fcbb542771cfbd
Author: Shixiong Zhu 
AuthorDate: Fri Feb 1 11:15:05 2019 -0800

[SPARK-26806][SS] EventTimeStats.merge should handle zeros correctly

## What changes were proposed in this pull request?

Right now, EventTimeStats.merge doesn't handle `zero.merge(zero)` 
correctly. This will make `avg` become `NaN`. And whatever gets merged with the 
result of `zero.merge(zero)`, `avg` will still be `NaN`. Then finally, we call 
`NaN.toLong` and get `0`, and the user will see the following incorrect report:

```
"eventTime" : {
"avg" : "1970-01-01T00:00:00.000Z",
"max" : "2019-01-31T12:57:00.000Z",
"min" : "2019-01-30T18:44:04.000Z",
"watermark" : "1970-01-01T00:00:00.000Z"
  }
```

This issue was reported by liancheng .

This PR fixes the above issue.

    ## How was this patch tested?

The new unit tests.

Closes #23718 from zsxwing/merge-zero.

Authored-by: Shixiong Zhu 
Signed-off-by: Shixiong Zhu 
---
 .../streaming/EventTimeWatermarkExec.scala | 17 +---
 .../sql/streaming/EventTimeWatermarkSuite.scala| 32 --
 2 files changed, 42 insertions(+), 7 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/EventTimeWatermarkExec.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/EventTimeWatermarkExec.scala
index b161651..6fa7ee0 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/EventTimeWatermarkExec.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/EventTimeWatermarkExec.scala
@@ -36,10 +36,19 @@ case class EventTimeStats(var max: Long, var min: Long, var 
avg: Double, var cou
   }
 
   def merge(that: EventTimeStats): Unit = {
-this.max = math.max(this.max, that.max)
-this.min = math.min(this.min, that.min)
-this.count += that.count
-this.avg += (that.avg - this.avg) * that.count / this.count
+if (that.count == 0) {
+  // no-op
+} else if (this.count == 0) {
+  this.max = that.max
+  this.min = that.min
+  this.count = that.count
+  this.avg = that.avg
+} else {
+  this.max = math.max(this.max, that.max)
+  this.min = math.min(this.min, that.min)
+  this.count += that.count
+  this.avg += (that.avg - this.avg) * that.count / this.count
+}
   }
 }
 
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala
index c696204..b79770a 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala
@@ -43,9 +43,9 @@ class EventTimeWatermarkSuite extends StreamTest with 
BeforeAndAfter with Matche
 sqlContext.streams.active.foreach(_.stop())
   }
 
-  test("EventTimeStats") {
-val epsilon = 10E-6
+  private val epsilon = 10E-6
 
+  test("EventTimeStats") {
 val stats = EventTimeStats(max = 100, min = 10, avg = 20.0, count = 5)
 stats.add(80L)
 stats.max should be (100)
@@ -62,7 +62,6 @@ class EventTimeWatermarkSuite extends StreamTest with 
BeforeAndAfter with Matche
   }
 
   test("EventTimeStats: avg on large values") {
-val epsilon = 10E-6
 val largeValue = 100L // 10B
 // Make sure `largeValue` will cause overflow if we use a Long sum to calc 
avg.
 assert(largeValue * largeValue != BigInt(largeValue) * BigInt(largeValue))
@@ -80,6 +79,33 @@ class EventTimeWatermarkSuite extends StreamTest with 
BeforeAndAfter with Matche
 stats.avg should be ((largeValue + 0.5) +- epsilon)
   }
 
+  test("EventTimeStats: zero merge zero") {
+val stats = EventTimeStats.zero
+val stats2 = EventTimeStats.zero
+stats.merge(stats2)
+stats should be (EventTimeStats.zero)
+  }
+
+  test("EventTimeStats: non-zero merge zero") {
+val stats = EventTimeStats(max = 10, min = 1, avg = 5.0, count = 3)
+val stats2 = EventTimeStats.zero
+stats.merge(stats2)
+stats.max should be (10L)
+stats.min should be (1L)
+stats.avg should be (5.0 +- epsilon)
+stats.count should be (3L)
+  }
+
+  test("EventTimeStats: zero merge non-zero") {
+val stats = EventTimeStats.zero
+val stats2 = EventTimeStats(max = 10, min = 1

[spark-website] branch asf-site updated: Add Jose Torres to committers list

2019-01-28 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch asf-site
in repository https://gitbox.apache.org/repos/asf/spark-website.git


The following commit(s) were added to refs/heads/asf-site by this push:
 new fb1a7b4  Add Jose Torres to committers list
fb1a7b4 is described below

commit fb1a7b407e149e133e35bb506d48cfe034a4d351
Author: Jose Torres 
AuthorDate: Mon Jan 28 15:59:37 2019 -0800

Add Jose Torres to committers list

Author: Jose Torres 

Closes #176 from jose-torres/addjose.
---
 committers.md| 1 +
 site/committers.html | 4 
 2 files changed, 5 insertions(+)

diff --git a/committers.md b/committers.md
index c3daf10..8049106 100644
--- a/committers.md
+++ b/committers.md
@@ -65,6 +65,7 @@ navigation:
 |Saisai Shao|Tencent|
 |Prashant Sharma|IBM|
 |Ram Sriharsha|Databricks|
+|Jose Torres|Databricks|
 |DB Tsai|Apple|
 |Takuya Ueshin|Databricks|
 |Marcelo Vanzin|Cloudera|
diff --git a/site/committers.html b/site/committers.html
index ec5814b..3066b5d 100644
--- a/site/committers.html
+++ b/site/committers.html
@@ -431,6 +431,10 @@
   Databricks
 
 
+  Jose Torres
+  Databricks
+
+
   DB Tsai
   Apple
 


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



[spark] branch branch-2.3 updated: [SPARK-26665][CORE] Fix a bug that BlockTransferService.fetchBlockSync may hang forever

2019-01-22 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch branch-2.3
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-2.3 by this push:
 new b88067b  [SPARK-26665][CORE] Fix a bug that 
BlockTransferService.fetchBlockSync may hang forever
b88067b is described below

commit b88067bd0f7b9466a89ce6458cb7766a24283b13
Author: Shixiong Zhu 
AuthorDate: Tue Jan 22 09:00:52 2019 -0800

[SPARK-26665][CORE] Fix a bug that BlockTransferService.fetchBlockSync may 
hang forever

## What changes were proposed in this pull request?

`ByteBuffer.allocate` may throw `OutOfMemoryError` when the block is large 
but no enough memory is available. However, when this happens, right now 
BlockTransferService.fetchBlockSync will just hang forever as its 
`BlockFetchingListener. onBlockFetchSuccess` doesn't complete `Promise`.

This PR catches `Throwable` and uses the error to complete `Promise`.

## How was this patch tested?

Added a unit test. Since I cannot make `ByteBuffer.allocate` throw 
`OutOfMemoryError`, I passed a negative size to make `ByteBuffer.allocate` 
fail. Although the error type is different, it should trigger the same code 
path.

Closes #23590 from zsxwing/SPARK-26665.

Authored-by: Shixiong Zhu 
Signed-off-by: Shixiong Zhu 
---
 .../spark/network/BlockTransferService.scala   |  12 ++-
 .../spark/network/BlockTransferServiceSuite.scala  | 104 +
 2 files changed, 112 insertions(+), 4 deletions(-)

diff --git 
a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala 
b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala
index eef8c31..875e4fc 100644
--- a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala
+++ b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala
@@ -105,10 +105,14 @@ abstract class BlockTransferService extends ShuffleClient 
with Closeable with Lo
 case f: FileSegmentManagedBuffer =>
   result.success(f)
 case _ =>
-  val ret = ByteBuffer.allocate(data.size.toInt)
-  ret.put(data.nioByteBuffer())
-  ret.flip()
-  result.success(new NioManagedBuffer(ret))
+  try {
+val ret = ByteBuffer.allocate(data.size.toInt)
+ret.put(data.nioByteBuffer())
+ret.flip()
+result.success(new NioManagedBuffer(ret))
+  } catch {
+case e: Throwable => result.failure(e)
+  }
   }
 }
   }, tempFileManager)
diff --git 
a/core/src/test/scala/org/apache/spark/network/BlockTransferServiceSuite.scala 
b/core/src/test/scala/org/apache/spark/network/BlockTransferServiceSuite.scala
new file mode 100644
index 000..d7e4b91
--- /dev/null
+++ 
b/core/src/test/scala/org/apache/spark/network/BlockTransferServiceSuite.scala
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network
+
+import java.io.InputStream
+import java.nio.ByteBuffer
+
+import scala.concurrent.Future
+import scala.concurrent.duration._
+import scala.reflect.ClassTag
+
+import org.scalatest.concurrent._
+
+import org.apache.spark.{SparkException, SparkFunSuite}
+import org.apache.spark.network.buffer.ManagedBuffer
+import org.apache.spark.network.shuffle.{BlockFetchingListener, 
DownloadFileManager}
+import org.apache.spark.storage.{BlockId, StorageLevel}
+
+class BlockTransferServiceSuite extends SparkFunSuite with TimeLimits {
+
+  implicit val defaultSignaler: Signaler = ThreadSignaler
+
+  test("fetchBlockSync should not hang when 
BlockFetchingListener.onBlockFetchSuccess fails") {
+// Create a mocked `BlockTransferService` to call 
`BlockFetchingListener.onBlockFetchSuccess`
+// with a bad `ManagedBuffer` which will trigger an exception in 
`onBlockFetchSuccess`.
+val blockTransferService = new BlockTransferService {
+  override def init(blockDataManager: BlockDataManager): Unit = {}
+
+  override def c

[spark] branch branch-2.4 updated: [SPARK-26665][CORE] Fix a bug that BlockTransferService.fetchBlockSync may hang forever

2019-01-22 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch branch-2.4
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-2.4 by this push:
 new 9814108  [SPARK-26665][CORE] Fix a bug that 
BlockTransferService.fetchBlockSync may hang forever
9814108 is described below

commit 9814108a4f51aeb281f14a8421ac1d735c85
Author: Shixiong Zhu 
AuthorDate: Tue Jan 22 09:00:52 2019 -0800

[SPARK-26665][CORE] Fix a bug that BlockTransferService.fetchBlockSync may 
hang forever

## What changes were proposed in this pull request?

`ByteBuffer.allocate` may throw `OutOfMemoryError` when the block is large 
but no enough memory is available. However, when this happens, right now 
BlockTransferService.fetchBlockSync will just hang forever as its 
`BlockFetchingListener. onBlockFetchSuccess` doesn't complete `Promise`.

This PR catches `Throwable` and uses the error to complete `Promise`.

## How was this patch tested?

Added a unit test. Since I cannot make `ByteBuffer.allocate` throw 
`OutOfMemoryError`, I passed a negative size to make `ByteBuffer.allocate` 
fail. Although the error type is different, it should trigger the same code 
path.

Closes #23590 from zsxwing/SPARK-26665.

Authored-by: Shixiong Zhu 
Signed-off-by: Shixiong Zhu 
(cherry picked from commit 66450bbc1bb4397f06ca9a6ecba4d16c82d711fd)
Signed-off-by: Shixiong Zhu 
---
 .../spark/network/BlockTransferService.scala   |  12 ++-
 .../spark/network/BlockTransferServiceSuite.scala  | 104 +
 2 files changed, 112 insertions(+), 4 deletions(-)

diff --git 
a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala 
b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala
index a58c8fa..51ced69 100644
--- a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala
+++ b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala
@@ -107,10 +107,14 @@ abstract class BlockTransferService extends ShuffleClient 
with Closeable with Lo
 case e: EncryptedManagedBuffer =>
   result.success(e)
 case _ =>
-  val ret = ByteBuffer.allocate(data.size.toInt)
-  ret.put(data.nioByteBuffer())
-  ret.flip()
-  result.success(new NioManagedBuffer(ret))
+  try {
+val ret = ByteBuffer.allocate(data.size.toInt)
+ret.put(data.nioByteBuffer())
+ret.flip()
+result.success(new NioManagedBuffer(ret))
+  } catch {
+case e: Throwable => result.failure(e)
+  }
   }
 }
   }, tempFileManager)
diff --git 
a/core/src/test/scala/org/apache/spark/network/BlockTransferServiceSuite.scala 
b/core/src/test/scala/org/apache/spark/network/BlockTransferServiceSuite.scala
new file mode 100644
index 000..d7e4b91
--- /dev/null
+++ 
b/core/src/test/scala/org/apache/spark/network/BlockTransferServiceSuite.scala
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network
+
+import java.io.InputStream
+import java.nio.ByteBuffer
+
+import scala.concurrent.Future
+import scala.concurrent.duration._
+import scala.reflect.ClassTag
+
+import org.scalatest.concurrent._
+
+import org.apache.spark.{SparkException, SparkFunSuite}
+import org.apache.spark.network.buffer.ManagedBuffer
+import org.apache.spark.network.shuffle.{BlockFetchingListener, 
DownloadFileManager}
+import org.apache.spark.storage.{BlockId, StorageLevel}
+
+class BlockTransferServiceSuite extends SparkFunSuite with TimeLimits {
+
+  implicit val defaultSignaler: Signaler = ThreadSignaler
+
+  test("fetchBlockSync should not hang when 
BlockFetchingListener.onBlockFetchSuccess fails") {
+// Create a mocked `BlockTransferService` to call 
`BlockFetchingListener.onBlockFetchSuccess`
+// with a bad `ManagedBuffer` which will trigger an exception in 
`onBlockFetchSuccess`.
+val blockTransferService = new BlockTra

[spark] branch master updated: [SPARK-26665][CORE] Fix a bug that BlockTransferService.fetchBlockSync may hang forever

2019-01-22 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 66450bb  [SPARK-26665][CORE] Fix a bug that 
BlockTransferService.fetchBlockSync may hang forever
66450bb is described below

commit 66450bbc1bb4397f06ca9a6ecba4d16c82d711fd
Author: Shixiong Zhu 
AuthorDate: Tue Jan 22 09:00:52 2019 -0800

[SPARK-26665][CORE] Fix a bug that BlockTransferService.fetchBlockSync may 
hang forever

## What changes were proposed in this pull request?

`ByteBuffer.allocate` may throw `OutOfMemoryError` when the block is large 
but no enough memory is available. However, when this happens, right now 
BlockTransferService.fetchBlockSync will just hang forever as its 
`BlockFetchingListener. onBlockFetchSuccess` doesn't complete `Promise`.

This PR catches `Throwable` and uses the error to complete `Promise`.

## How was this patch tested?

Added a unit test. Since I cannot make `ByteBuffer.allocate` throw 
`OutOfMemoryError`, I passed a negative size to make `ByteBuffer.allocate` 
fail. Although the error type is different, it should trigger the same code 
path.

Closes #23590 from zsxwing/SPARK-26665.

Authored-by: Shixiong Zhu 
Signed-off-by: Shixiong Zhu 
---
 .../spark/network/BlockTransferService.scala   |  12 ++-
 .../spark/network/BlockTransferServiceSuite.scala  | 104 +
 2 files changed, 112 insertions(+), 4 deletions(-)

diff --git 
a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala 
b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala
index a58c8fa..51ced69 100644
--- a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala
+++ b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala
@@ -107,10 +107,14 @@ abstract class BlockTransferService extends ShuffleClient 
with Closeable with Lo
 case e: EncryptedManagedBuffer =>
   result.success(e)
 case _ =>
-  val ret = ByteBuffer.allocate(data.size.toInt)
-  ret.put(data.nioByteBuffer())
-  ret.flip()
-  result.success(new NioManagedBuffer(ret))
+  try {
+val ret = ByteBuffer.allocate(data.size.toInt)
+ret.put(data.nioByteBuffer())
+ret.flip()
+result.success(new NioManagedBuffer(ret))
+  } catch {
+case e: Throwable => result.failure(e)
+  }
   }
 }
   }, tempFileManager)
diff --git 
a/core/src/test/scala/org/apache/spark/network/BlockTransferServiceSuite.scala 
b/core/src/test/scala/org/apache/spark/network/BlockTransferServiceSuite.scala
new file mode 100644
index 000..d7e4b91
--- /dev/null
+++ 
b/core/src/test/scala/org/apache/spark/network/BlockTransferServiceSuite.scala
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network
+
+import java.io.InputStream
+import java.nio.ByteBuffer
+
+import scala.concurrent.Future
+import scala.concurrent.duration._
+import scala.reflect.ClassTag
+
+import org.scalatest.concurrent._
+
+import org.apache.spark.{SparkException, SparkFunSuite}
+import org.apache.spark.network.buffer.ManagedBuffer
+import org.apache.spark.network.shuffle.{BlockFetchingListener, 
DownloadFileManager}
+import org.apache.spark.storage.{BlockId, StorageLevel}
+
+class BlockTransferServiceSuite extends SparkFunSuite with TimeLimits {
+
+  implicit val defaultSignaler: Signaler = ThreadSignaler
+
+  test("fetchBlockSync should not hang when 
BlockFetchingListener.onBlockFetchSuccess fails") {
+// Create a mocked `BlockTransferService` to call 
`BlockFetchingListener.onBlockFetchSuccess`
+// with a bad `ManagedBuffer` which will trigger an exception in 
`onBlockFetchSuccess`.
+val blockTransferService = new BlockTransferService {
+  override def init(blockDataManager: BlockDataManager): Unit = {}
+
+  override def close(): Un

[spark] branch branch-2.3 updated: Revert "[SPARK-26629][SS] Fixed error with multiple file stream in a query + restart on a batch that has no data for one file stream"

2019-01-16 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch branch-2.3
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-2.3 by this push:
 new c0fc6d0  Revert "[SPARK-26629][SS] Fixed error with multiple file 
stream in a query + restart on a batch that has no data for one file stream"
c0fc6d0 is described below

commit c0fc6d0d8dbd890a817176eb1da6e98252c2e0c0
Author: Shixiong Zhu 
AuthorDate: Wed Jan 16 10:03:21 2019 -0800

Revert "[SPARK-26629][SS] Fixed error with multiple file stream in a query 
+ restart on a batch that has no data for one file stream"

This reverts commit 5a50ae37f4c41099c174459603966ee25f21ac75.
---
 .../execution/streaming/FileStreamSourceLog.scala  |  4 +-
 .../sql/execution/streaming/HDFSMetadataLog.scala  |  3 +-
 .../execution/streaming/HDFSMetadataLogSuite.scala |  6 --
 .../sql/streaming/FileStreamSourceSuite.scala  | 75 ++
 4 files changed, 8 insertions(+), 80 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceLog.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceLog.scala
index 7b2ea96..8628471 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceLog.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceLog.scala
@@ -117,9 +117,7 @@ class FileStreamSourceLog(
 
 val batches =
   (existedBatches ++ retrievedBatches).map(i => i._1 -> 
i._2.get).toArray.sortBy(_._1)
-if (startBatchId <= endBatchId) {
-  HDFSMetadataLog.verifyBatchIds(batches.map(_._1), startId, endId)
-}
+HDFSMetadataLog.verifyBatchIds(batches.map(_._1), startId, endId)
 batches
   }
 }
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala
index d4cfbb3..00bc215 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala
@@ -457,8 +457,7 @@ object HDFSMetadataLog {
   }
 
   /**
-   * Verify if batchIds are continuous and between `startId` and `endId` (both 
inclusive and
-   * startId assumed to be <= endId).
+   * Verify if batchIds are continuous and between `startId` and `endId`.
*
* @param batchIds the sorted ids to verify.
* @param startId the start id. If it's set, batchIds should start with this 
id.
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala
index 57a0343..4677769 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala
@@ -275,12 +275,6 @@ class HDFSMetadataLogSuite extends SparkFunSuite with 
SharedSQLContext {
 intercept[IllegalStateException](verifyBatchIds(Seq(2, 3, 4), None, 
Some(5L)))
 intercept[IllegalStateException](verifyBatchIds(Seq(2, 3, 4), Some(1L), 
Some(5L)))
 intercept[IllegalStateException](verifyBatchIds(Seq(1, 2, 4, 5), Some(1L), 
Some(5L)))
-
-// Related to SPARK-26629, this capatures the behavior for verifyBatchIds 
when startId > endId
-intercept[IllegalStateException](verifyBatchIds(Seq(), Some(2L), Some(1L)))
-intercept[AssertionError](verifyBatchIds(Seq(2), Some(2L), Some(1L)))
-intercept[AssertionError](verifyBatchIds(Seq(1), Some(2L), Some(1L)))
-intercept[AssertionError](verifyBatchIds(Seq(0), Some(2L), Some(1L)))
   }
 }
 
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala
index fb0b365..d4bd9c7 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala
@@ -48,33 +48,21 @@ abstract class FileStreamSourceTest
* `FileStreamSource` actually being used in the execution.
*/
   abstract class AddFileData extends AddData {
-private val _qualifiedBasePath = PrivateMethod[Path]('qualifiedBasePath)
-
-private def isSamePath(fileSource: FileStreamSource, srcPath: File): 
Boolean = {
-  val path = (fileSource invokePrivate 
_qualifiedBasePath()).toString.stripPrefix("file:")
-  path == srcPath.getCanonicalPath
-}
-
 override def addData(query: Option[StreamExecution]): (Source, Offset) = {
   require(
 query.nonEmpty,
 "Cannot add data when there is n

[spark] branch branch-2.3 updated: [SPARK-26629][SS] Fixed error with multiple file stream in a query + restart on a batch that has no data for one file stream

2019-01-16 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch branch-2.3
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-2.3 by this push:
 new 5a50ae3  [SPARK-26629][SS] Fixed error with multiple file stream in a 
query + restart on a batch that has no data for one file stream
5a50ae3 is described below

commit 5a50ae37f4c41099c174459603966ee25f21ac75
Author: Tathagata Das 
AuthorDate: Wed Jan 16 09:42:14 2019 -0800

[SPARK-26629][SS] Fixed error with multiple file stream in a query + 
restart on a batch that has no data for one file stream

## What changes were proposed in this pull request?
When a streaming query has multiple file streams, and there is a batch 
where one of the file streams dont have data in that batch, then if the query 
has to restart from that, it will throw the following error.
```
java.lang.IllegalStateException: batch 1 doesn't exist
at 
org.apache.spark.sql.execution.streaming.HDFSMetadataLog$.verifyBatchIds(HDFSMetadataLog.scala:300)
at 
org.apache.spark.sql.execution.streaming.FileStreamSourceLog.get(FileStreamSourceLog.scala:120)
at 
org.apache.spark.sql.execution.streaming.FileStreamSource.getBatch(FileStreamSource.scala:181)
at 
org.apache.spark.sql.execution.streaming.MicroBatchExecution$$anonfun$org$apache$spark$sql$execution$streaming$MicroBatchExecution$$populateStartOffsets$2.apply(MicroBatchExecution.scala:294)
at 
org.apache.spark.sql.execution.streaming.MicroBatchExecution$$anonfun$org$apache$spark$sql$execution$streaming$MicroBatchExecution$$populateStartOffsets$2.apply(MicroBatchExecution.scala:291)
at scala.collection.Iterator$class.foreach(Iterator.scala:891)
at scala.collection.AbstractIterator.foreach(Iterator.scala:1334)
at scala.collection.IterableLike$class.foreach(IterableLike.scala:72)
at 
org.apache.spark.sql.execution.streaming.StreamProgress.foreach(StreamProgress.scala:25)
at 
org.apache.spark.sql.execution.streaming.MicroBatchExecution.org$apache$spark$sql$execution$streaming$MicroBatchExecution$$populateStartOffsets(MicroBatchExecution.scala:291)
at 
org.apache.spark.sql.execution.streaming.MicroBatchExecution$$anonfun$runActivatedStream$1$$anonfun$apply$mcZ$sp$1.apply$mcV$sp(MicroBatchExecution.scala:178)
at 
org.apache.spark.sql.execution.streaming.MicroBatchExecution$$anonfun$runActivatedStream$1$$anonfun$apply$mcZ$sp$1.apply(MicroBatchExecution.scala:175)
at 
org.apache.spark.sql.execution.streaming.MicroBatchExecution$$anonfun$runActivatedStream$1$$anonfun$apply$mcZ$sp$1.apply(MicroBatchExecution.scala:175)
at 
org.apache.spark.sql.execution.streaming.ProgressReporter$class.reportTimeTaken(ProgressReporter.scala:251)
at 
org.apache.spark.sql.execution.streaming.StreamExecution.reportTimeTaken(StreamExecution.scala:61)
at 
org.apache.spark.sql.execution.streaming.MicroBatchExecution$$anonfun$runActivatedStream$1.apply$mcZ$sp(MicroBatchExecution.scala:175)
at 
org.apache.spark.sql.execution.streaming.ProcessingTimeExecutor.execute(TriggerExecutor.scala:56)
at 
org.apache.spark.sql.execution.streaming.MicroBatchExecution.runActivatedStream(MicroBatchExecution.scala:169)
at 
org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runStream(StreamExecution.scala:295)
at 
org.apache.spark.sql.execution.streaming.StreamExecution$$anon$1.run(StreamExecution.scala:205)
```

Existing `HDFSMetadata.verifyBatchIds` threw error whenever the `batchIds` 
list was empty. In the context of `FileStreamSource.getBatch` (where verify is 
called) and `FileStreamSourceLog` (subclass of `HDFSMetadata`), this is usually 
okay because, in a streaming query with one file stream, the `batchIds` can 
never be empty:
- A batch is planned only when the `FileStreamSourceLog` has seen new 
offset (that is, there are new data files).
- So `FileStreamSource.getBatch` will be called on X to Y where X will 
always be > Y. This calls internally`HDFSMetadata.verifyBatchIds (X+1, Y)` with 
X+1-Y ids.

For example.,`FileStreamSource.getBatch(4, 5)` will call `verify(batchIds = 
Seq(5), start = 5, end = 5)`. However, the invariant of X > Y is not true when 
there are two file stream sources, as a batch may be planned even when only one 
of the file streams has data. So one of the file stream may not have data, 
which can call `FileStreamSource.getBatch(X, X)` -> `verify(batchIds = 
Seq.empty, start = X+1, end = X)` -> failure.

Note that `FileStreamSource.getBatch(X, X)` gets called **only when 
restarting a query in a batch where a file source did not have data**. This is 
because in normal planning of batches, `MicroBatchExecution` avoids calling 
`FileStreamSource.getBatch(X, X)` w

[spark] branch branch-2.4 updated: [SPARK-26629][SS] Fixed error with multiple file stream in a query + restart on a batch that has no data for one file stream

2019-01-16 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch branch-2.4
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-2.4 by this push:
 new 1843c16  [SPARK-26629][SS] Fixed error with multiple file stream in a 
query + restart on a batch that has no data for one file stream
1843c16 is described below

commit 1843c16fda09a3e9373e8f7b3ff5f73455c50442
Author: Tathagata Das 
AuthorDate: Wed Jan 16 09:42:14 2019 -0800

[SPARK-26629][SS] Fixed error with multiple file stream in a query + 
restart on a batch that has no data for one file stream

## What changes were proposed in this pull request?
When a streaming query has multiple file streams, and there is a batch 
where one of the file streams dont have data in that batch, then if the query 
has to restart from that, it will throw the following error.
```
java.lang.IllegalStateException: batch 1 doesn't exist
at 
org.apache.spark.sql.execution.streaming.HDFSMetadataLog$.verifyBatchIds(HDFSMetadataLog.scala:300)
at 
org.apache.spark.sql.execution.streaming.FileStreamSourceLog.get(FileStreamSourceLog.scala:120)
at 
org.apache.spark.sql.execution.streaming.FileStreamSource.getBatch(FileStreamSource.scala:181)
at 
org.apache.spark.sql.execution.streaming.MicroBatchExecution$$anonfun$org$apache$spark$sql$execution$streaming$MicroBatchExecution$$populateStartOffsets$2.apply(MicroBatchExecution.scala:294)
at 
org.apache.spark.sql.execution.streaming.MicroBatchExecution$$anonfun$org$apache$spark$sql$execution$streaming$MicroBatchExecution$$populateStartOffsets$2.apply(MicroBatchExecution.scala:291)
at scala.collection.Iterator$class.foreach(Iterator.scala:891)
at scala.collection.AbstractIterator.foreach(Iterator.scala:1334)
at scala.collection.IterableLike$class.foreach(IterableLike.scala:72)
at 
org.apache.spark.sql.execution.streaming.StreamProgress.foreach(StreamProgress.scala:25)
at 
org.apache.spark.sql.execution.streaming.MicroBatchExecution.org$apache$spark$sql$execution$streaming$MicroBatchExecution$$populateStartOffsets(MicroBatchExecution.scala:291)
at 
org.apache.spark.sql.execution.streaming.MicroBatchExecution$$anonfun$runActivatedStream$1$$anonfun$apply$mcZ$sp$1.apply$mcV$sp(MicroBatchExecution.scala:178)
at 
org.apache.spark.sql.execution.streaming.MicroBatchExecution$$anonfun$runActivatedStream$1$$anonfun$apply$mcZ$sp$1.apply(MicroBatchExecution.scala:175)
at 
org.apache.spark.sql.execution.streaming.MicroBatchExecution$$anonfun$runActivatedStream$1$$anonfun$apply$mcZ$sp$1.apply(MicroBatchExecution.scala:175)
at 
org.apache.spark.sql.execution.streaming.ProgressReporter$class.reportTimeTaken(ProgressReporter.scala:251)
at 
org.apache.spark.sql.execution.streaming.StreamExecution.reportTimeTaken(StreamExecution.scala:61)
at 
org.apache.spark.sql.execution.streaming.MicroBatchExecution$$anonfun$runActivatedStream$1.apply$mcZ$sp(MicroBatchExecution.scala:175)
at 
org.apache.spark.sql.execution.streaming.ProcessingTimeExecutor.execute(TriggerExecutor.scala:56)
at 
org.apache.spark.sql.execution.streaming.MicroBatchExecution.runActivatedStream(MicroBatchExecution.scala:169)
at 
org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runStream(StreamExecution.scala:295)
at 
org.apache.spark.sql.execution.streaming.StreamExecution$$anon$1.run(StreamExecution.scala:205)
```

Existing `HDFSMetadata.verifyBatchIds` threw error whenever the `batchIds` 
list was empty. In the context of `FileStreamSource.getBatch` (where verify is 
called) and `FileStreamSourceLog` (subclass of `HDFSMetadata`), this is usually 
okay because, in a streaming query with one file stream, the `batchIds` can 
never be empty:
- A batch is planned only when the `FileStreamSourceLog` has seen new 
offset (that is, there are new data files).
- So `FileStreamSource.getBatch` will be called on X to Y where X will 
always be > Y. This calls internally`HDFSMetadata.verifyBatchIds (X+1, Y)` with 
X+1-Y ids.

For example.,`FileStreamSource.getBatch(4, 5)` will call `verify(batchIds = 
Seq(5), start = 5, end = 5)`. However, the invariant of X > Y is not true when 
there are two file stream sources, as a batch may be planned even when only one 
of the file streams has data. So one of the file stream may not have data, 
which can call `FileStreamSource.getBatch(X, X)` -> `verify(batchIds = 
Seq.empty, start = X+1, end = X)` -> failure.

Note that `FileStreamSource.getBatch(X, X)` gets called **only when 
restarting a query in a batch where a file source did not have data**. This is 
because in normal planning of batches, `MicroBatchExecution` avoids calling 
`FileStreamSource.getBatch(X, X)` w

[spark] branch master updated: [SPARK-26629][SS] Fixed error with multiple file stream in a query + restart on a batch that has no data for one file stream

2019-01-16 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 06d5b17  [SPARK-26629][SS] Fixed error with multiple file stream in a 
query + restart on a batch that has no data for one file stream
06d5b17 is described below

commit 06d5b173b687c23aa53e293ed6e12ec746393876
Author: Tathagata Das 
AuthorDate: Wed Jan 16 09:42:14 2019 -0800

[SPARK-26629][SS] Fixed error with multiple file stream in a query + 
restart on a batch that has no data for one file stream

## What changes were proposed in this pull request?
When a streaming query has multiple file streams, and there is a batch 
where one of the file streams dont have data in that batch, then if the query 
has to restart from that, it will throw the following error.
```
java.lang.IllegalStateException: batch 1 doesn't exist
at 
org.apache.spark.sql.execution.streaming.HDFSMetadataLog$.verifyBatchIds(HDFSMetadataLog.scala:300)
at 
org.apache.spark.sql.execution.streaming.FileStreamSourceLog.get(FileStreamSourceLog.scala:120)
at 
org.apache.spark.sql.execution.streaming.FileStreamSource.getBatch(FileStreamSource.scala:181)
at 
org.apache.spark.sql.execution.streaming.MicroBatchExecution$$anonfun$org$apache$spark$sql$execution$streaming$MicroBatchExecution$$populateStartOffsets$2.apply(MicroBatchExecution.scala:294)
at 
org.apache.spark.sql.execution.streaming.MicroBatchExecution$$anonfun$org$apache$spark$sql$execution$streaming$MicroBatchExecution$$populateStartOffsets$2.apply(MicroBatchExecution.scala:291)
at scala.collection.Iterator$class.foreach(Iterator.scala:891)
at scala.collection.AbstractIterator.foreach(Iterator.scala:1334)
at scala.collection.IterableLike$class.foreach(IterableLike.scala:72)
at 
org.apache.spark.sql.execution.streaming.StreamProgress.foreach(StreamProgress.scala:25)
at 
org.apache.spark.sql.execution.streaming.MicroBatchExecution.org$apache$spark$sql$execution$streaming$MicroBatchExecution$$populateStartOffsets(MicroBatchExecution.scala:291)
at 
org.apache.spark.sql.execution.streaming.MicroBatchExecution$$anonfun$runActivatedStream$1$$anonfun$apply$mcZ$sp$1.apply$mcV$sp(MicroBatchExecution.scala:178)
at 
org.apache.spark.sql.execution.streaming.MicroBatchExecution$$anonfun$runActivatedStream$1$$anonfun$apply$mcZ$sp$1.apply(MicroBatchExecution.scala:175)
at 
org.apache.spark.sql.execution.streaming.MicroBatchExecution$$anonfun$runActivatedStream$1$$anonfun$apply$mcZ$sp$1.apply(MicroBatchExecution.scala:175)
at 
org.apache.spark.sql.execution.streaming.ProgressReporter$class.reportTimeTaken(ProgressReporter.scala:251)
at 
org.apache.spark.sql.execution.streaming.StreamExecution.reportTimeTaken(StreamExecution.scala:61)
at 
org.apache.spark.sql.execution.streaming.MicroBatchExecution$$anonfun$runActivatedStream$1.apply$mcZ$sp(MicroBatchExecution.scala:175)
at 
org.apache.spark.sql.execution.streaming.ProcessingTimeExecutor.execute(TriggerExecutor.scala:56)
at 
org.apache.spark.sql.execution.streaming.MicroBatchExecution.runActivatedStream(MicroBatchExecution.scala:169)
at 
org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runStream(StreamExecution.scala:295)
at 
org.apache.spark.sql.execution.streaming.StreamExecution$$anon$1.run(StreamExecution.scala:205)
```

Existing `HDFSMetadata.verifyBatchIds` threw error whenever the `batchIds` 
list was empty. In the context of `FileStreamSource.getBatch` (where verify is 
called) and `FileStreamSourceLog` (subclass of `HDFSMetadata`), this is usually 
okay because, in a streaming query with one file stream, the `batchIds` can 
never be empty:
- A batch is planned only when the `FileStreamSourceLog` has seen new 
offset (that is, there are new data files).
- So `FileStreamSource.getBatch` will be called on X to Y where X will 
always be > Y. This calls internally`HDFSMetadata.verifyBatchIds (X+1, Y)` with 
X+1-Y ids.

For example.,`FileStreamSource.getBatch(4, 5)` will call `verify(batchIds = 
Seq(5), start = 5, end = 5)`. However, the invariant of X > Y is not true when 
there are two file stream sources, as a batch may be planned even when only one 
of the file streams has data. So one of the file stream may not have data, 
which can call `FileStreamSource.getBatch(X, X)` -> `verify(batchIds = 
Seq.empty, start = X+1, end = X)` -> failure.

Note that `FileStreamSource.getBatch(X, X)` gets called **only when 
restarting a query in a batch where a file source did not have data**. This is 
because in normal planning of batches, `MicroBatchExecution` avoids calling 
`FileStreamSource.getBatch(X, X)` when 

[spark] branch master updated: [SPARK-26350][FOLLOWUP] Add actual verification on new UT introduced on SPARK-26350

2019-01-15 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 2ebb79b  [SPARK-26350][FOLLOWUP] Add actual verification on new UT 
introduced on SPARK-26350
2ebb79b is described below

commit 2ebb79b2a607aa25ea22826d9c5d6af18c97a7f2
Author: Jungtaek Lim (HeartSaVioR) 
AuthorDate: Tue Jan 15 14:21:51 2019 -0800

[SPARK-26350][FOLLOWUP] Add actual verification on new UT introduced on 
SPARK-26350

## What changes were proposed in this pull request?

This patch adds the check to verify consumer group id is given correctly 
when custom group id is provided to Kafka parameter.

## How was this patch tested?

Modified UT.

Closes #23544 from 
HeartSaVioR/SPARK-26350-follow-up-actual-verification-on-UT.

Authored-by: Jungtaek Lim (HeartSaVioR) 
Signed-off-by: Shixiong Zhu 
---
 .../spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala| 14 --
 .../org/apache/spark/sql/kafka010/KafkaRelationSuite.scala | 13 -
 .../org/apache/spark/sql/kafka010/KafkaTestUtils.scala |  6 +-
 3 files changed, 29 insertions(+), 4 deletions(-)

diff --git 
a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
 
b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
index 6402088..cb45384 100644
--- 
a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
+++ 
b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
@@ -28,6 +28,7 @@ import scala.collection.JavaConverters._
 import scala.io.Source
 import scala.util.Random
 
+import org.apache.kafka.clients.admin.{AdminClient, ConsumerGroupListing}
 import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord, 
RecordMetadata}
 import org.apache.kafka.common.TopicPartition
 import org.scalatest.concurrent.PatienceConfiguration.Timeout
@@ -638,10 +639,11 @@ abstract class KafkaMicroBatchSourceSuiteBase extends 
KafkaSourceSuiteBase {
 testUtils.sendMessages(topic, (11 to 20).map(_.toString).toArray, Some(1))
 testUtils.sendMessages(topic, (21 to 30).map(_.toString).toArray, Some(2))
 
+val customGroupId = "id-" + Random.nextInt()
 val dsKafka = spark
   .readStream
   .format("kafka")
-  .option("kafka.group.id", "id-" + Random.nextInt())
+  .option("kafka.group.id", customGroupId)
   .option("kafka.bootstrap.servers", testUtils.brokerAddress)
   .option("subscribe", topic)
   .option("startingOffsets", "earliest")
@@ -652,7 +654,15 @@ abstract class KafkaMicroBatchSourceSuiteBase extends 
KafkaSourceSuiteBase {
 
 testStream(dsKafka)(
   makeSureGetOffsetCalled,
-  CheckAnswer(1 to 30: _*)
+  CheckAnswer(1 to 30: _*),
+  Execute { _ =>
+val consumerGroups = testUtils.listConsumerGroups()
+val validGroups = consumerGroups.valid().get()
+val validGroupsId = validGroups.asScala.map(_.groupId())
+assert(validGroupsId.exists(_ === customGroupId), "Valid consumer 
groups don't " +
+  s"contain the expected group id - Valid consumer groups: 
$validGroupsId / " +
+  s"expected group id: $customGroupId")
+  }
 )
   }
 
diff --git 
a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala
 
b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala
index efe7385..2cd13a9 100644
--- 
a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala
+++ 
b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala
@@ -20,6 +20,9 @@ package org.apache.spark.sql.kafka010
 import java.util.Locale
 import java.util.concurrent.atomic.AtomicInteger
 
+import scala.collection.JavaConverters._
+import scala.util.Random
+
 import org.apache.kafka.clients.producer.ProducerRecord
 import org.apache.kafka.common.TopicPartition
 
@@ -247,8 +250,16 @@ class KafkaRelationSuite extends QueryTest with 
SharedSQLContext with KafkaTest
 testUtils.sendMessages(topic, (11 to 20).map(_.toString).toArray, Some(1))
 testUtils.sendMessages(topic, (21 to 30).map(_.toString).toArray, Some(2))
 
-val df = createDF(topic, withOptions = Map("kafka.group.id" -> "custom"))
+val customGroupId = "id-" + Random.nextInt()
+val df = createDF(topic, withOptions = Map("kafka.group.id" -> 
customGroupId))
 checkAnswer(df, (1 to 30).map(_.toString).toDF())
+
+val consumerGroups = testUtils.listConsumerGroups()
+v

[spark] branch branch-2.4 updated: [SPARK-26586][SS] Fix race condition that causes streams to run with unexpected confs

2019-01-11 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch branch-2.4
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-2.4 by this push:
 new bbf61eb  [SPARK-26586][SS] Fix race condition that causes streams to 
run with unexpected confs
bbf61eb is described below

commit bbf61eb4222d7b46e71dc91eeedf82d27226fc2c
Author: Mukul Murthy 
AuthorDate: Fri Jan 11 11:46:14 2019 -0800

[SPARK-26586][SS] Fix race condition that causes streams to run with 
unexpected confs

## What changes were proposed in this pull request?

Fix race condition where streams can have unexpected conf values.

New streaming queries should run with isolated SparkSessions so that they 
aren't affected by conf updates after they are started. In StreamExecution, the 
parent SparkSession is cloned and used to run each batch, but this cloning 
happens in a separate thread and may happen after DataStreamWriter.start() 
returns. If a stream is started and a conf key is set immediately after, the 
stream is likely to have the new value.

## How was this patch tested?

New unit test that fails prior to the production change and passes with it.

Please review http://spark.apache.org/contributing.html before opening a 
pull request.

Closes #23513 from mukulmurthy/26586.

Authored-by: Mukul Murthy 
Signed-off-by: Shixiong Zhu 
(cherry picked from commit ae382c94dd10ff494dde4de44e66182bf6dbe8f8)
Signed-off-by: Shixiong Zhu 
---
 .../sql/execution/streaming/StreamExecution.scala  |  5 +++--
 .../test/DataStreamReaderWriterSuite.scala | 24 ++
 2 files changed, 27 insertions(+), 2 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala
index de33844..c1aa98a 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala
@@ -176,6 +176,9 @@ abstract class StreamExecution(
   lazy val streamMetrics = new MetricsReporter(
 this, s"spark.streaming.${Option(name).getOrElse(id)}")
 
+  /** Isolated spark session to run the batches with. */
+  private val sparkSessionForStream = sparkSession.cloneSession()
+
   /**
* The thread that runs the micro-batches of this stream. Note that this 
thread must be
* [[org.apache.spark.util.UninterruptibleThread]] to workaround KAFKA-1894: 
interrupting a
@@ -265,8 +268,6 @@ abstract class StreamExecution(
   // force initialization of the logical plan so that the sources can be 
created
   logicalPlan
 
-  // Isolated spark session to run the batches with.
-  val sparkSessionForStream = sparkSession.cloneSession()
   // Adaptive execution can change num shuffle partitions, disallow
   sparkSessionForStream.conf.set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, 
"false")
   // Disable cost-based join optimization as we do not want stateful 
operations to be rearranged
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala
index 8212fb9..569114a 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala
@@ -18,6 +18,7 @@
 package org.apache.spark.sql.streaming.test
 
 import java.io.File
+import java.util.ConcurrentModificationException
 import java.util.Locale
 import java.util.concurrent.TimeUnit
 
@@ -651,4 +652,27 @@ class DataStreamReaderWriterSuite extends StreamTest with 
BeforeAndAfter {
 
 LastOptions.clear()
   }
+
+  test("SPARK-26586: Streams should have isolated confs") {
+import testImplicits._
+val input = MemoryStream[Int]
+input.addData(1 to 10)
+spark.conf.set("testKey1", 0)
+val queries = (1 to 10).map { i =>
+  spark.conf.set("testKey1", i)
+  input.toDF().writeStream
+.foreachBatch { (df: Dataset[Row], id: Long) =>
+  val v = df.sparkSession.conf.get("testKey1").toInt
+  if (i != v) {
+throw new ConcurrentModificationException(s"Stream $i has the 
wrong conf value $v")
+  }
+}
+.start()
+}
+try {
+  queries.foreach(_.processAllAvailable())
+} finally {
+  queries.foreach(_.stop())
+}
+  }
 }


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



[spark] branch master updated: [SPARK-26586][SS] Fix race condition that causes streams to run with unexpected confs

2019-01-11 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new ae382c9  [SPARK-26586][SS] Fix race condition that causes streams to 
run with unexpected confs
ae382c9 is described below

commit ae382c94dd10ff494dde4de44e66182bf6dbe8f8
Author: Mukul Murthy 
AuthorDate: Fri Jan 11 11:46:14 2019 -0800

[SPARK-26586][SS] Fix race condition that causes streams to run with 
unexpected confs

## What changes were proposed in this pull request?

Fix race condition where streams can have unexpected conf values.

New streaming queries should run with isolated SparkSessions so that they 
aren't affected by conf updates after they are started. In StreamExecution, the 
parent SparkSession is cloned and used to run each batch, but this cloning 
happens in a separate thread and may happen after DataStreamWriter.start() 
returns. If a stream is started and a conf key is set immediately after, the 
stream is likely to have the new value.

## How was this patch tested?

New unit test that fails prior to the production change and passes with it.

Please review http://spark.apache.org/contributing.html before opening a 
pull request.

Closes #23513 from mukulmurthy/26586.

Authored-by: Mukul Murthy 
Signed-off-by: Shixiong Zhu 
---
 .../sql/execution/streaming/StreamExecution.scala  |  5 +++--
 .../test/DataStreamReaderWriterSuite.scala | 24 ++
 2 files changed, 27 insertions(+), 2 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala
index 83824f4..90f7b47 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala
@@ -181,6 +181,9 @@ abstract class StreamExecution(
   lazy val streamMetrics = new MetricsReporter(
 this, s"spark.streaming.${Option(name).getOrElse(id)}")
 
+  /** Isolated spark session to run the batches with. */
+  private val sparkSessionForStream = sparkSession.cloneSession()
+
   /**
* The thread that runs the micro-batches of this stream. Note that this 
thread must be
* [[org.apache.spark.util.UninterruptibleThread]] to workaround KAFKA-1894: 
interrupting a
@@ -270,8 +273,6 @@ abstract class StreamExecution(
   // force initialization of the logical plan so that the sources can be 
created
   logicalPlan
 
-  // Isolated spark session to run the batches with.
-  val sparkSessionForStream = sparkSession.cloneSession()
   // Adaptive execution can change num shuffle partitions, disallow
   sparkSessionForStream.conf.set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, 
"false")
   // Disable cost-based join optimization as we do not want stateful 
operations to be rearranged
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala
index 4d3a54a..74ea0bf 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala
@@ -18,6 +18,7 @@
 package org.apache.spark.sql.streaming.test
 
 import java.io.File
+import java.util.ConcurrentModificationException
 import java.util.Locale
 import java.util.concurrent.TimeUnit
 
@@ -651,4 +652,27 @@ class DataStreamReaderWriterSuite extends StreamTest with 
BeforeAndAfter {
 
 LastOptions.clear()
   }
+
+  test("SPARK-26586: Streams should have isolated confs") {
+import testImplicits._
+val input = MemoryStream[Int]
+input.addData(1 to 10)
+spark.conf.set("testKey1", 0)
+val queries = (1 to 10).map { i =>
+  spark.conf.set("testKey1", i)
+  input.toDF().writeStream
+.foreachBatch { (df: Dataset[Row], id: Long) =>
+  val v = df.sparkSession.conf.get("testKey1").toInt
+  if (i != v) {
+throw new ConcurrentModificationException(s"Stream $i has the 
wrong conf value $v")
+  }
+}
+.start()
+}
+try {
+  queries.foreach(_.processAllAvailable())
+} finally {
+  queries.foreach(_.stop())
+}
+  }
 }


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



[spark] branch branch-2.4 updated: [SPARK-26267][SS] Retry when detecting incorrect offsets from Kafka (2.4)

2019-01-07 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch branch-2.4
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-2.4 by this push:
 new faa4c28  [SPARK-26267][SS] Retry when detecting incorrect offsets from 
Kafka (2.4)
faa4c28 is described below

commit faa4c2823b69c1643d7678ee1cb0b7295c611334
Author: Shixiong Zhu 
AuthorDate: Mon Jan 7 16:53:07 2019 -0800

[SPARK-26267][SS] Retry when detecting incorrect offsets from Kafka (2.4)

## What changes were proposed in this pull request?

Backport #23324 to branch-2.4.

## How was this patch tested?

Jenkins

Closes #23365 from zsxwing/SPARK-26267-2.4.

Authored-by: Shixiong Zhu 
Signed-off-by: Shixiong Zhu 
---
 .../spark/sql/kafka010/KafkaContinuousReader.scala |  4 +-
 .../spark/sql/kafka010/KafkaMicroBatchReader.scala | 20 --
 .../sql/kafka010/KafkaOffsetRangeCalculator.scala  |  2 +
 .../spark/sql/kafka010/KafkaOffsetReader.scala | 80 --
 .../apache/spark/sql/kafka010/KafkaSource.scala|  5 +-
 .../sql/kafka010/KafkaMicroBatchSourceSuite.scala  | 48 +
 6 files changed, 146 insertions(+), 13 deletions(-)

diff --git 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReader.scala
 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReader.scala
index 8ce56a2..561d501 100644
--- 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReader.scala
+++ 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReader.scala
@@ -73,7 +73,7 @@ class KafkaContinuousReader(
 offset = start.orElse {
   val offsets = initialOffsets match {
 case EarliestOffsetRangeLimit => 
KafkaSourceOffset(offsetReader.fetchEarliestOffsets())
-case LatestOffsetRangeLimit => 
KafkaSourceOffset(offsetReader.fetchLatestOffsets())
+case LatestOffsetRangeLimit => 
KafkaSourceOffset(offsetReader.fetchLatestOffsets(None))
 case SpecificOffsetRangeLimit(p) => 
offsetReader.fetchSpecificOffsets(p, reportDataLoss)
   }
   logInfo(s"Initial offsets: $offsets")
@@ -128,7 +128,7 @@ class KafkaContinuousReader(
   }
 
   override def needsReconfiguration(): Boolean = {
-knownPartitions != null && offsetReader.fetchLatestOffsets().keySet != 
knownPartitions
+knownPartitions != null && offsetReader.fetchLatestOffsets(None).keySet != 
knownPartitions
   }
 
   override def toString(): String = s"KafkaSource[$offsetReader]"
diff --git 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReader.scala
 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReader.scala
index 8cc989f..b6c8035 100644
--- 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReader.scala
+++ 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReader.scala
@@ -93,7 +93,8 @@ private[kafka010] class KafkaMicroBatchReader(
 endPartitionOffsets = Option(end.orElse(null))
 .map(_.asInstanceOf[KafkaSourceOffset].partitionToOffsets)
 .getOrElse {
-  val latestPartitionOffsets = kafkaOffsetReader.fetchLatestOffsets()
+  val latestPartitionOffsets =
+kafkaOffsetReader.fetchLatestOffsets(Some(startPartitionOffsets))
   maxOffsetsPerTrigger.map { maxOffsets =>
 rateLimit(maxOffsets, startPartitionOffsets, 
latestPartitionOffsets)
   }.getOrElse {
@@ -132,10 +133,21 @@ private[kafka010] class KafkaMicroBatchReader(
 }.toSeq
 logDebug("TopicPartitions: " + topicPartitions.mkString(", "))
 
+val fromOffsets = startPartitionOffsets ++ newPartitionInitialOffsets
+val untilOffsets = endPartitionOffsets
+untilOffsets.foreach { case (tp, untilOffset) =>
+  fromOffsets.get(tp).foreach { fromOffset =>
+if (untilOffset < fromOffset) {
+  reportDataLoss(s"Partition $tp's offset was changed from " +
+s"$fromOffset to $untilOffset, some data may have been missed")
+}
+  }
+}
+
 // Calculate offset ranges
 val offsetRanges = rangeCalculator.getRanges(
-  fromOffsets = startPartitionOffsets ++ newPartitionInitialOffsets,
-  untilOffsets = endPartitionOffsets,
+  fromOffsets = fromOffsets,
+  untilOffsets = untilOffsets,
   executorLocations = getSortedExecutorList())
 
 // Reuse Kafka consumers only when all the offset ranges have distinct 
TopicPartitions,
@@ -192,7 +204,7 @@ private[kafka010] class KafkaMicroBatchReader(
 case EarliestOffsetRangeLimit =>
   KafkaSourceOffset(kafkaOffsetReader.fetchEarliestOffsets())
  

[spark] branch master updated: [SPARK-26267][SS] Retry when detecting incorrect offsets from Kafka

2018-12-21 Thread zsxwing
This is an automated email from the ASF dual-hosted git repository.

zsxwing pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 8e76d66  [SPARK-26267][SS] Retry when detecting incorrect offsets from 
Kafka
8e76d66 is described below

commit 8e76d6621aaddb8b73443b14ea2c6eebe9089893
Author: Shixiong Zhu 
AuthorDate: Fri Dec 21 10:41:25 2018 -0800

[SPARK-26267][SS] Retry when detecting incorrect offsets from Kafka

## What changes were proposed in this pull request?

Due to [KAFKA-7703](https://issues.apache.org/jira/browse/KAFKA-7703), 
Kafka may return an earliest offset when we are request a latest offset. This 
will cause Spark to reprocess data.

As per suggestion in KAFKA-7703, we put a position call between poll and 
seekToEnd to block the fetch request triggered by `poll` before calling 
`seekToEnd`.

In addition, to avoid other unknown issues, we also use the previous known 
offsets to audit the latest offsets returned by Kafka. If we find some 
incorrect offsets (a latest offset is less than an offset in `knownOffsets`), 
we will retry at most `maxOffsetFetchAttempts` times.

## How was this patch tested?

Jenkins

Closes #23324 from zsxwing/SPARK-26267.

Authored-by: Shixiong Zhu 
Signed-off-by: Shixiong Zhu 
---
 .../sql/kafka010/KafkaContinuousReadSupport.scala  |  4 +-
 .../sql/kafka010/KafkaMicroBatchReadSupport.scala  | 19 +++--
 .../sql/kafka010/KafkaOffsetRangeCalculator.scala  |  2 +
 .../spark/sql/kafka010/KafkaOffsetReader.scala | 80 --
 .../apache/spark/sql/kafka010/KafkaSource.scala|  5 +-
 .../sql/kafka010/KafkaMicroBatchSourceSuite.scala  | 48 +
 6 files changed, 145 insertions(+), 13 deletions(-)

diff --git 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReadSupport.scala
 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReadSupport.scala
index 1753a28..02dfb9c 100644
--- 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReadSupport.scala
+++ 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReadSupport.scala
@@ -60,7 +60,7 @@ class KafkaContinuousReadSupport(
   override def initialOffset(): Offset = {
 val offsets = initialOffsets match {
   case EarliestOffsetRangeLimit => 
KafkaSourceOffset(offsetReader.fetchEarliestOffsets())
-  case LatestOffsetRangeLimit => 
KafkaSourceOffset(offsetReader.fetchLatestOffsets())
+  case LatestOffsetRangeLimit => 
KafkaSourceOffset(offsetReader.fetchLatestOffsets(None))
   case SpecificOffsetRangeLimit(p) => offsetReader.fetchSpecificOffsets(p, 
reportDataLoss)
 }
 logInfo(s"Initial offsets: $offsets")
@@ -107,7 +107,7 @@ class KafkaContinuousReadSupport(
 
   override def needsReconfiguration(config: ScanConfig): Boolean = {
 val knownPartitions = 
config.asInstanceOf[KafkaContinuousScanConfig].knownPartitions
-offsetReader.fetchLatestOffsets().keySet != knownPartitions
+offsetReader.fetchLatestOffsets(None).keySet != knownPartitions
   }
 
   override def toString(): String = s"KafkaSource[$offsetReader]"
diff --git 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReadSupport.scala
 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReadSupport.scala
index bb4de67..b4f042e 100644
--- 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReadSupport.scala
+++ 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReadSupport.scala
@@ -84,7 +84,7 @@ private[kafka010] class KafkaMicroBatchReadSupport(
 
   override def latestOffset(start: Offset): Offset = {
 val startPartitionOffsets = 
start.asInstanceOf[KafkaSourceOffset].partitionToOffsets
-val latestPartitionOffsets = kafkaOffsetReader.fetchLatestOffsets()
+val latestPartitionOffsets = 
kafkaOffsetReader.fetchLatestOffsets(Some(startPartitionOffsets))
 endPartitionOffsets = KafkaSourceOffset(maxOffsetsPerTrigger.map { 
maxOffsets =>
   rateLimit(maxOffsets, startPartitionOffsets, latestPartitionOffsets)
 }.getOrElse {
@@ -133,10 +133,21 @@ private[kafka010] class KafkaMicroBatchReadSupport(
 }.toSeq
 logDebug("TopicPartitions: " + topicPartitions.mkString(", "))
 
+val fromOffsets = startPartitionOffsets ++ newPartitionInitialOffsets
+val untilOffsets = endPartitionOffsets
+untilOffsets.foreach { case (tp, untilOffset) =>
+  fromOffsets.get(tp).foreach { fromOffset =>
+if (untilOffset < fromOffset) {
+  reportDataLoss(s"Partition $tp's offset was changed from " +

spark git commit: [SPARK-26092][SS] Use CheckpointFileManager to write the streaming metadata file

2018-11-16 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/branch-2.4 77c0629cb -> c23b801d3


[SPARK-26092][SS] Use CheckpointFileManager to write the streaming metadata file

## What changes were proposed in this pull request?

Use CheckpointFileManager to write the streaming `metadata` file so that the 
`metadata` file will never be a partial file.

## How was this patch tested?

Jenkins

Closes #23060 from zsxwing/SPARK-26092.

Authored-by: Shixiong Zhu 
Signed-off-by: Shixiong Zhu 
(cherry picked from commit 058c4602b000b24deb764a810ef8b43c41fe63ae)
Signed-off-by: Shixiong Zhu 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/c23b801d
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/c23b801d
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/c23b801d

Branch: refs/heads/branch-2.4
Commit: c23b801d3c87b12e729b98910833b441db05bd45
Parents: 77c0629
Author: Shixiong Zhu 
Authored: Fri Nov 16 15:43:27 2018 -0800
Committer: Shixiong Zhu 
Committed: Fri Nov 16 15:43:44 2018 -0800

--
 .../streaming/CheckpointFileManager.scala   |  2 +-
 .../execution/streaming/StreamExecution.scala   |  1 +
 .../execution/streaming/StreamMetadata.scala| 23 ++--
 3 files changed, 18 insertions(+), 8 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/c23b801d/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManager.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManager.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManager.scala
index 606ba25..b3e4240 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManager.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManager.scala
@@ -56,7 +56,7 @@ trait CheckpointFileManager {
* @param overwriteIfPossible If true, then the implementations must do a 
best-effort attempt to
*overwrite the file if it already exists. It 
should not throw
*any exception if the file exists. However, if 
false, then the
-   *implementation must not overwrite if the file 
alraedy exists and
+   *implementation must not overwrite if the file 
already exists and
*must throw `FileAlreadyExistsException` in 
that case.
*/
   def createAtomic(path: Path, overwriteIfPossible: Boolean): 
CancellableFSDataOutputStream

http://git-wip-us.apache.org/repos/asf/spark/blob/c23b801d/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala
index f6c60c1..de33844 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala
@@ -87,6 +87,7 @@ abstract class StreamExecution(
   val resolvedCheckpointRoot = {
 val checkpointPath = new Path(checkpointRoot)
 val fs = 
checkpointPath.getFileSystem(sparkSession.sessionState.newHadoopConf())
+fs.mkdirs(checkpointPath)
 checkpointPath.makeQualified(fs.getUri, 
fs.getWorkingDirectory).toUri.toString
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/c23b801d/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetadata.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetadata.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetadata.scala
index 0bc54ea..516afbe 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetadata.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetadata.scala
@@ -19,16 +19,18 @@ package org.apache.spark.sql.execution.streaming
 
 import java.io.{InputStreamReader, OutputStreamWriter}
 import java.nio.charset.StandardCharsets
+import java.util.ConcurrentModificationException
 
 import scala.util.control.NonFatal
 
 import org.apache.commons.io.IOUtils
 import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.{FileSystem, FSDataInputStream, 
FSDataOutputStream, Path}
+import org.apache.hadoop.fs.{FileAlreadyExistsException, FSDataInputStream, 
Path}
 import org.json4s.NoTypeHints
 imp

spark git commit: [SPARK-26092][SS] Use CheckpointFileManager to write the streaming metadata file

2018-11-16 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/master 99cbc51b3 -> 058c4602b


[SPARK-26092][SS] Use CheckpointFileManager to write the streaming metadata file

## What changes were proposed in this pull request?

Use CheckpointFileManager to write the streaming `metadata` file so that the 
`metadata` file will never be a partial file.

## How was this patch tested?

Jenkins

Closes #23060 from zsxwing/SPARK-26092.

Authored-by: Shixiong Zhu 
Signed-off-by: Shixiong Zhu 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/058c4602
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/058c4602
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/058c4602

Branch: refs/heads/master
Commit: 058c4602b000b24deb764a810ef8b43c41fe63ae
Parents: 99cbc51
Author: Shixiong Zhu 
Authored: Fri Nov 16 15:43:27 2018 -0800
Committer: Shixiong Zhu 
Committed: Fri Nov 16 15:43:27 2018 -0800

--
 .../streaming/CheckpointFileManager.scala   |  2 +-
 .../execution/streaming/StreamExecution.scala   |  1 +
 .../execution/streaming/StreamMetadata.scala| 23 ++--
 3 files changed, 18 insertions(+), 8 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/058c4602/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManager.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManager.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManager.scala
index 606ba25..b3e4240 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManager.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManager.scala
@@ -56,7 +56,7 @@ trait CheckpointFileManager {
* @param overwriteIfPossible If true, then the implementations must do a 
best-effort attempt to
*overwrite the file if it already exists. It 
should not throw
*any exception if the file exists. However, if 
false, then the
-   *implementation must not overwrite if the file 
alraedy exists and
+   *implementation must not overwrite if the file 
already exists and
*must throw `FileAlreadyExistsException` in 
that case.
*/
   def createAtomic(path: Path, overwriteIfPossible: Boolean): 
CancellableFSDataOutputStream

http://git-wip-us.apache.org/repos/asf/spark/blob/058c4602/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala
index 631a6eb..89b4f40 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala
@@ -88,6 +88,7 @@ abstract class StreamExecution(
   val resolvedCheckpointRoot = {
 val checkpointPath = new Path(checkpointRoot)
 val fs = 
checkpointPath.getFileSystem(sparkSession.sessionState.newHadoopConf())
+fs.mkdirs(checkpointPath)
 checkpointPath.makeQualified(fs.getUri, 
fs.getWorkingDirectory).toUri.toString
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/058c4602/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetadata.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetadata.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetadata.scala
index 0bc54ea..516afbe 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetadata.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetadata.scala
@@ -19,16 +19,18 @@ package org.apache.spark.sql.execution.streaming
 
 import java.io.{InputStreamReader, OutputStreamWriter}
 import java.nio.charset.StandardCharsets
+import java.util.ConcurrentModificationException
 
 import scala.util.control.NonFatal
 
 import org.apache.commons.io.IOUtils
 import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.{FileSystem, FSDataInputStream, 
FSDataOutputStream, Path}
+import org.apache.hadoop.fs.{FileAlreadyExistsException, FSDataInputStream, 
Path}
 import org.json4s.NoTypeHints
 import org.json4s.jackson.Serialization
 
 import org.apache.spark.internal.Logging
+imp

spark git commit: [SPARK-26069][TESTS] Fix flaky test: RpcIntegrationSuite.sendRpcWithStreamFailures

2018-11-16 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/branch-2.4 be42bfe5c -> 77c0629cb


[SPARK-26069][TESTS] Fix flaky test: 
RpcIntegrationSuite.sendRpcWithStreamFailures

## What changes were proposed in this pull request?

The test failure is because `assertErrorAndClosed` misses one possible error 
message: `java.nio.channels.ClosedChannelException`. This happens when the 
second `uploadStream` is called after the channel has been closed. This can be 
reproduced by adding `Thread.sleep(1000)` below this line: 
https://github.com/apache/spark/blob/03306a6df39c9fd6cb581401c13c4dfc6bbd632e/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java#L217

This PR fixes the above issue and also improves the test failure messages of 
`assertErrorAndClosed`.

## How was this patch tested?

Jenkins

Closes #23041 from zsxwing/SPARK-26069.

Authored-by: Shixiong Zhu 
Signed-off-by: Shixiong Zhu 
(cherry picked from commit 99cbc51b3250c07a3e8cc95c9b74e9d1725bac77)
Signed-off-by: Shixiong Zhu 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/77c0629c
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/77c0629c
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/77c0629c

Branch: refs/heads/branch-2.4
Commit: 77c0629cbac6046cc3f2ea1025c43d86af344d62
Parents: be42bfe
Author: Shixiong Zhu 
Authored: Fri Nov 16 09:51:41 2018 -0800
Committer: Shixiong Zhu 
Committed: Fri Nov 16 09:52:03 2018 -0800

--
 .../apache/spark/network/RpcIntegrationSuite.java | 18 --
 1 file changed, 12 insertions(+), 6 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/77c0629c/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java
--
diff --git 
a/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java
 
b/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java
index 1f4d75c..45f4a18 100644
--- 
a/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java
+++ 
b/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java
@@ -371,7 +371,10 @@ public class RpcIntegrationSuite {
 
   private void assertErrorAndClosed(RpcResult result, String expectedError) {
 assertTrue("unexpected success: " + result.successMessages, 
result.successMessages.isEmpty());
-// we expect 1 additional error, which contains *either* "closed" or 
"Connection reset"
+// we expect 1 additional error, which should contain one of the follow 
messages:
+// - "closed"
+// - "Connection reset"
+// - "java.nio.channels.ClosedChannelException"
 Set errors = result.errorMessages;
 assertEquals("Expected 2 errors, got " + errors.size() + "errors: " +
 errors, 2, errors.size());
@@ -379,15 +382,18 @@ public class RpcIntegrationSuite {
 Set containsAndClosed = Sets.newHashSet(expectedError);
 containsAndClosed.add("closed");
 containsAndClosed.add("Connection reset");
+containsAndClosed.add("java.nio.channels.ClosedChannelException");
 
 Pair, Set> r = checkErrorsContain(errors, 
containsAndClosed);
 
-Set errorsNotFound = r.getRight();
-assertEquals(1, errorsNotFound.size());
-String err = errorsNotFound.iterator().next();
-assertTrue(err.equals("closed") || err.equals("Connection reset"));
+assertTrue("Got a non-empty set " + r.getLeft(), r.getLeft().isEmpty());
 
-assertTrue(r.getLeft().isEmpty());
+Set errorsNotFound = r.getRight();
+assertEquals(
+"The size of " + errorsNotFound.toString() + " was not 2", 2, 
errorsNotFound.size());
+for (String err: errorsNotFound) {
+  assertTrue("Found a wrong error " + err, 
containsAndClosed.contains(err));
+}
   }
 
   private Pair, Set> checkErrorsContain(


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



spark git commit: [SPARK-26069][TESTS] Fix flaky test: RpcIntegrationSuite.sendRpcWithStreamFailures

2018-11-16 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/master a2fc48c28 -> 99cbc51b3


[SPARK-26069][TESTS] Fix flaky test: 
RpcIntegrationSuite.sendRpcWithStreamFailures

## What changes were proposed in this pull request?

The test failure is because `assertErrorAndClosed` misses one possible error 
message: `java.nio.channels.ClosedChannelException`. This happens when the 
second `uploadStream` is called after the channel has been closed. This can be 
reproduced by adding `Thread.sleep(1000)` below this line: 
https://github.com/apache/spark/blob/03306a6df39c9fd6cb581401c13c4dfc6bbd632e/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java#L217

This PR fixes the above issue and also improves the test failure messages of 
`assertErrorAndClosed`.

## How was this patch tested?

Jenkins

Closes #23041 from zsxwing/SPARK-26069.

Authored-by: Shixiong Zhu 
Signed-off-by: Shixiong Zhu 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/99cbc51b
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/99cbc51b
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/99cbc51b

Branch: refs/heads/master
Commit: 99cbc51b3250c07a3e8cc95c9b74e9d1725bac77
Parents: a2fc48c
Author: Shixiong Zhu 
Authored: Fri Nov 16 09:51:41 2018 -0800
Committer: Shixiong Zhu 
Committed: Fri Nov 16 09:51:41 2018 -0800

--
 .../apache/spark/network/RpcIntegrationSuite.java | 18 --
 1 file changed, 12 insertions(+), 6 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/99cbc51b/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java
--
diff --git 
a/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java
 
b/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java
index 1f4d75c..45f4a18 100644
--- 
a/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java
+++ 
b/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java
@@ -371,7 +371,10 @@ public class RpcIntegrationSuite {
 
   private void assertErrorAndClosed(RpcResult result, String expectedError) {
 assertTrue("unexpected success: " + result.successMessages, 
result.successMessages.isEmpty());
-// we expect 1 additional error, which contains *either* "closed" or 
"Connection reset"
+// we expect 1 additional error, which should contain one of the follow 
messages:
+// - "closed"
+// - "Connection reset"
+// - "java.nio.channels.ClosedChannelException"
 Set errors = result.errorMessages;
 assertEquals("Expected 2 errors, got " + errors.size() + "errors: " +
 errors, 2, errors.size());
@@ -379,15 +382,18 @@ public class RpcIntegrationSuite {
 Set containsAndClosed = Sets.newHashSet(expectedError);
 containsAndClosed.add("closed");
 containsAndClosed.add("Connection reset");
+containsAndClosed.add("java.nio.channels.ClosedChannelException");
 
 Pair, Set> r = checkErrorsContain(errors, 
containsAndClosed);
 
-Set errorsNotFound = r.getRight();
-assertEquals(1, errorsNotFound.size());
-String err = errorsNotFound.iterator().next();
-assertTrue(err.equals("closed") || err.equals("Connection reset"));
+assertTrue("Got a non-empty set " + r.getLeft(), r.getLeft().isEmpty());
 
-assertTrue(r.getLeft().isEmpty());
+Set errorsNotFound = r.getRight();
+assertEquals(
+"The size of " + errorsNotFound.toString() + " was not 2", 2, 
errorsNotFound.size());
+for (String err: errorsNotFound) {
+  assertTrue("Found a wrong error " + err, 
containsAndClosed.contains(err));
+}
   }
 
   private Pair, Set> checkErrorsContain(


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



spark git commit: [SPARK-26042][SS][TESTS] Fix a potential hang in KafkaContinuousSourceTopicDeletionSuite

2018-11-14 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/master 5f11e8c4c -> 4035c98a0


[SPARK-26042][SS][TESTS] Fix a potential hang in 
KafkaContinuousSourceTopicDeletionSuite

## What changes were proposed in this pull request?

As initializing lazy vals shares the same lock, a thread is trying to 
initialize `executedPlan` when `isRDD` is running, this thread will hang 
forever.

This PR just materializes `executedPlan` so that accessing it when `toRdd` is 
running doesn't need to wait for a lock

## How was this patch tested?

Jenkins

Closes #23023 from zsxwing/SPARK-26042.

Authored-by: Shixiong Zhu 
Signed-off-by: Shixiong Zhu 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/4035c98a
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/4035c98a
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/4035c98a

Branch: refs/heads/master
Commit: 4035c98a0c03cf61d1fb9a9916df513ab1081a9b
Parents: 5f11e8c
Author: Shixiong Zhu 
Authored: Wed Nov 14 10:19:20 2018 -0800
Committer: Shixiong Zhu 
Committed: Wed Nov 14 10:19:20 2018 -0800

--
 .../execution/streaming/continuous/ContinuousExecution.scala  | 7 ++-
 1 file changed, 6 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/4035c98a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala
index f009c52..4a7df73 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala
@@ -262,7 +262,12 @@ class ContinuousExecution(
 
   reportTimeTaken("runContinuous") {
 SQLExecution.withNewExecutionId(
-  sparkSessionForQuery, lastExecution)(lastExecution.toRdd)
+  sparkSessionForQuery, lastExecution) {
+  // Materialize `executedPlan` so that accessing it when `toRdd` is 
running doesn't need to
+  // wait for a lock
+  lastExecution.executedPlan
+  lastExecution.toRdd
+}
   }
 } catch {
   case t: Throwable if StreamExecution.isInterruptionException(t, 
sparkSession.sparkContext) &&


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



spark git commit: [SPARK-26042][SS][TESTS] Fix a potential hang in KafkaContinuousSourceTopicDeletionSuite

2018-11-14 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/branch-2.4 e2e1f0ad8 -> ca426bfa5


[SPARK-26042][SS][TESTS] Fix a potential hang in 
KafkaContinuousSourceTopicDeletionSuite

## What changes were proposed in this pull request?

As initializing lazy vals shares the same lock, a thread is trying to 
initialize `executedPlan` when `isRDD` is running, this thread will hang 
forever.

This PR just materializes `executedPlan` so that accessing it when `toRdd` is 
running doesn't need to wait for a lock

## How was this patch tested?

Jenkins

Closes #23023 from zsxwing/SPARK-26042.

Authored-by: Shixiong Zhu 
Signed-off-by: Shixiong Zhu 
(cherry picked from commit 4035c98a0c03cf61d1fb9a9916df513ab1081a9b)
Signed-off-by: Shixiong Zhu 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/ca426bfa
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/ca426bfa
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/ca426bfa

Branch: refs/heads/branch-2.4
Commit: ca426bfa56045f01de0ea14480a375753073e025
Parents: e2e1f0a
Author: Shixiong Zhu 
Authored: Wed Nov 14 10:19:20 2018 -0800
Committer: Shixiong Zhu 
Committed: Wed Nov 14 10:19:37 2018 -0800

--
 .../execution/streaming/continuous/ContinuousExecution.scala  | 7 ++-
 1 file changed, 6 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/ca426bfa/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala
index f104422..2e24fa6 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala
@@ -259,7 +259,12 @@ class ContinuousExecution(
 
   reportTimeTaken("runContinuous") {
 SQLExecution.withNewExecutionId(
-  sparkSessionForQuery, lastExecution)(lastExecution.toRdd)
+  sparkSessionForQuery, lastExecution) {
+  // Materialize `executedPlan` so that accessing it when `toRdd` is 
running doesn't need to
+  // wait for a lock
+  lastExecution.executedPlan
+  lastExecution.toRdd
+}
   }
 } catch {
   case t: Throwable


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



spark git commit: [SPARK-25773][CORE] Cancel zombie tasks in a result stage when the job finishes

2018-10-30 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/master 94de5609b -> c36537fcf


[SPARK-25773][CORE] Cancel zombie tasks in a result stage when the job finishes

## What changes were proposed in this pull request?

When a job finishes, there may be some zombie tasks still running due to stage 
retry. Since a result stage will never be used by other jobs, running these 
tasks are just wasting the cluster resource. This PR just asks TaskScheduler to 
cancel the running tasks of a result stage when it's already finished. Credits 
go to srinathshankar who suggested this idea to me.

This PR also fixes two minor issues while I'm touching DAGScheduler:
- Invalid spark.job.interruptOnCancel should not crash DAGScheduler.
- Non fatal errors should not crash DAGScheduler.

## How was this patch tested?

The new unit tests.

Closes #22771 from zsxwing/SPARK-25773.

Lead-authored-by: Shixiong Zhu 
Co-authored-by: Shixiong Zhu 
Signed-off-by: Shixiong Zhu 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/c36537fc
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/c36537fc
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/c36537fc

Branch: refs/heads/master
Commit: c36537fcfddc1eae1581b1b84d9d4384c5985c26
Parents: 94de560
Author: Shixiong Zhu 
Authored: Tue Oct 30 10:48:04 2018 -0700
Committer: Shixiong Zhu 
Committed: Tue Oct 30 10:48:04 2018 -0700

--
 .../apache/spark/scheduler/DAGScheduler.scala   | 48 +++---
 .../org/apache/spark/SparkContextSuite.scala| 53 +++-
 .../spark/scheduler/DAGSchedulerSuite.scala | 51 +--
 3 files changed, 129 insertions(+), 23 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/c36537fc/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
--
diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala 
b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
index 34b1160..06966e7 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
@@ -1296,6 +1296,27 @@ private[spark] class DAGScheduler(
   }
 
   /**
+   * Check [[SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL]] in job properties to 
see if we should
+   * interrupt running tasks. Returns `false` if the property value is not a 
boolean value
+   */
+  private def shouldInterruptTaskThread(job: ActiveJob): Boolean = {
+if (job.properties == null) {
+  false
+} else {
+  val shouldInterruptThread =
+job.properties.getProperty(SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL, 
"false")
+  try {
+shouldInterruptThread.toBoolean
+  } catch {
+case e: IllegalArgumentException =>
+  logWarning(s"${SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL} in Job 
${job.jobId} " +
+s"is invalid: $shouldInterruptThread. Using 'false' instead", e)
+  false
+  }
+}
+  }
+
+  /**
* Responds to a task finishing. This is called inside the event loop so it 
assumes that it can
* modify the scheduler's internal state. Use taskEnded() to post a task end 
event from outside.
*/
@@ -1364,6 +1385,21 @@ private[spark] class DAGScheduler(
   if (job.numFinished == job.numPartitions) {
 markStageAsFinished(resultStage)
 cleanupStateForJobAndIndependentStages(job)
+try {
+  // killAllTaskAttempts will fail if a SchedulerBackend 
does not implement
+  // killTask.
+  logInfo(s"Job ${job.jobId} is finished. Cancelling 
potential speculative " +
+"or zombie tasks for this job")
+  // ResultStage is only used by this job. It's safe to 
kill speculative or
+  // zombie tasks in this stage.
+  taskScheduler.killAllTaskAttempts(
+stageId,
+shouldInterruptTaskThread(job),
+reason = "Stage finished")
+} catch {
+  case e: UnsupportedOperationException =>
+logWarning(s"Could not cancel tasks for stage 
$stageId", e)
+}
 listenerBus.post(
   SparkListenerJobEnd(job.jobId, clock.getTimeMillis(), 
JobSucceeded))
   }
@@ -1373,7 +1409,7 @@ private[spark] class DAGScheduler(
   try {
 job.listener.taskSucceeded(rt.outputId, event.result)
   } catch {
-

spark git commit: [SPARK-25771][PYSPARK] Fix improper synchronization in PythonWorkerFactory

2018-10-22 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/master 81a305dd0 -> bd66c7302


[SPARK-25771][PYSPARK] Fix improper synchronization in PythonWorkerFactory

## What changes were proposed in this pull request?

Fix the following issues in PythonWorkerFactory
1. MonitorThread.run uses a wrong lock.
2. `createSimpleWorker` misses `synchronized` when updating `simpleWorkers`.

Other changes are just to improve the code style to make the thread-safe 
contract clear.

## How was this patch tested?

Jenkins

Closes #22770 from zsxwing/pwf.

Authored-by: Shixiong Zhu 
Signed-off-by: Shixiong Zhu 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/bd66c730
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/bd66c730
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/bd66c730

Branch: refs/heads/master
Commit: bd66c73025c0b947be230178a737fd53812b78dd
Parents: 81a305d
Author: Shixiong Zhu 
Authored: Mon Oct 22 10:07:11 2018 -0700
Committer: Shixiong Zhu 
Committed: Mon Oct 22 10:07:11 2018 -0700

--
 .../spark/api/python/PythonWorkerFactory.scala  | 75 +++-
 1 file changed, 43 insertions(+), 32 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/bd66c730/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala
--
diff --git 
a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala 
b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala
index 6afa37a..1f2f503 100644
--- a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala
+++ b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala
@@ -21,6 +21,7 @@ import java.io.{DataInputStream, DataOutputStream, 
EOFException, InputStream, Ou
 import java.net.{InetAddress, ServerSocket, Socket, SocketException}
 import java.nio.charset.StandardCharsets
 import java.util.Arrays
+import javax.annotation.concurrent.GuardedBy
 
 import scala.collection.JavaConverters._
 import scala.collection.mutable
@@ -31,7 +32,7 @@ import org.apache.spark.security.SocketAuthHelper
 import org.apache.spark.util.{RedirectThread, Utils}
 
 private[spark] class PythonWorkerFactory(pythonExec: String, envVars: 
Map[String, String])
-  extends Logging {
+  extends Logging { self =>
 
   import PythonWorkerFactory._
 
@@ -39,7 +40,7 @@ private[spark] class PythonWorkerFactory(pythonExec: String, 
envVars: Map[String
   // pyspark/daemon.py (by default) and tell it to fork new workers for our 
tasks. This daemon
   // currently only works on UNIX-based systems now because it uses signals 
for child management,
   // so we can also fall back to launching workers, pyspark/worker.py (by 
default) directly.
-  val useDaemon = {
+  private val useDaemon = {
 val useDaemonEnabled = 
SparkEnv.get.conf.getBoolean("spark.python.use.daemon", true)
 
 // This flag is ignored on Windows as it's unable to fork.
@@ -51,44 +52,52 @@ private[spark] class PythonWorkerFactory(pythonExec: 
String, envVars: Map[String
   // as expert-only option, and shouldn't be used before knowing what it means 
exactly.
 
   // This configuration indicates the module to run the daemon to execute its 
Python workers.
-  val daemonModule = 
SparkEnv.get.conf.getOption("spark.python.daemon.module").map { value =>
-logInfo(
-  s"Python daemon module in PySpark is set to [$value] in 
'spark.python.daemon.module', " +
-  "using this to start the daemon up. Note that this configuration only 
has an effect when " +
-  "'spark.python.use.daemon' is enabled and the platform is not Windows.")
-value
-  }.getOrElse("pyspark.daemon")
+  private val daemonModule =
+SparkEnv.get.conf.getOption("spark.python.daemon.module").map { value =>
+  logInfo(
+s"Python daemon module in PySpark is set to [$value] in 
'spark.python.daemon.module', " +
+"using this to start the daemon up. Note that this configuration only 
has an effect when " +
+"'spark.python.use.daemon' is enabled and the platform is not 
Windows.")
+  value
+}.getOrElse("pyspark.daemon")
 
   // This configuration indicates the module to run each Python worker.
-  val workerModule = 
SparkEnv.get.conf.getOption("spark.python.worker.module").map { value =>
-logInfo(
-  s"Python worker module in PySpark is set to [$value] in 
'spark.python.worker.module', " +
-  "using this to start the worker up. Note that this configuration only 
has an effect when " +
-  "'spark.python.use.daemon' is disabled or the platform is Windows.")
-value
-  }.getO

spark git commit: [SPARK-25644][SS] Fix java foreachBatch in DataStreamWriter

2018-10-05 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/branch-2.4 2c700ee30 -> 0a70afdc0


[SPARK-25644][SS] Fix java foreachBatch in DataStreamWriter

## What changes were proposed in this pull request?

The java `foreachBatch` API in `DataStreamWriter` should accept 
`java.lang.Long` rather `scala.Long`.

## How was this patch tested?

New java test.

Closes #22633 from zsxwing/fix-java-foreachbatch.

Authored-by: Shixiong Zhu 
Signed-off-by: Shixiong Zhu 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/0a70afdc
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/0a70afdc
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/0a70afdc

Branch: refs/heads/branch-2.4
Commit: 0a70afdc08d76f84c59ec50f2f92144f54271602
Parents: 2c700ee
Author: Shixiong Zhu 
Authored: Fri Oct 5 10:45:15 2018 -0700
Committer: Shixiong Zhu 
Committed: Fri Oct 5 11:18:49 2018 -0700

--
 .../spark/sql/streaming/DataStreamWriter.scala  |  2 +-
 .../JavaDataStreamReaderWriterSuite.java| 89 
 2 files changed, 90 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/0a70afdc/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
index 735fd17..4eb2918 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
@@ -379,7 +379,7 @@ final class DataStreamWriter[T] private[sql](ds: 
Dataset[T]) {
* @since 2.4.0
*/
   @InterfaceStability.Evolving
-  def foreachBatch(function: VoidFunction2[Dataset[T], Long]): 
DataStreamWriter[T] = {
+  def foreachBatch(function: VoidFunction2[Dataset[T], java.lang.Long]): 
DataStreamWriter[T] = {
 foreachBatch((batchDs: Dataset[T], batchId: Long) => 
function.call(batchDs, batchId))
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/0a70afdc/sql/core/src/test/java/test/org/apache/spark/sql/streaming/JavaDataStreamReaderWriterSuite.java
--
diff --git 
a/sql/core/src/test/java/test/org/apache/spark/sql/streaming/JavaDataStreamReaderWriterSuite.java
 
b/sql/core/src/test/java/test/org/apache/spark/sql/streaming/JavaDataStreamReaderWriterSuite.java
new file mode 100644
index 000..48cdb26
--- /dev/null
+++ 
b/sql/core/src/test/java/test/org/apache/spark/sql/streaming/JavaDataStreamReaderWriterSuite.java
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package test.org.apache.spark.sql.streaming;
+
+import java.io.File;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.spark.api.java.function.VoidFunction2;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.ForeachWriter;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.streaming.StreamingQuery;
+import org.apache.spark.sql.test.TestSparkSession;
+import org.apache.spark.util.Utils;
+
+public class JavaDataStreamReaderWriterSuite {
+  private SparkSession spark;
+  private String input;
+
+  @Before
+  public void setUp() {
+spark = new TestSparkSession();
+input = Utils.createTempDir(System.getProperty("java.io.tmpdir"), 
"input").toString();
+  }
+
+  @After
+  public void tearDown() {
+try {
+  Utils.deleteRecursively(new File(input));
+} finally {
+  spark.stop();
+  spark = null;
+}
+  }
+
+  @Test
+  public void testForeachBatchAPI() {
+StreamingQuery query = spark
+  .readStream()
+  .textFile(input)
+  .writeStream()
+  .foreachBatch(new VoidFunction2, Long>() {
+@Override
+public void call(Dataset v1, Long v2) throws Exception {}
+  })
+  .star

spark git commit: [SPARK-25644][SS] Fix java foreachBatch in DataStreamWriter

2018-10-05 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/master 434ada12a -> 7dcc90fbb


[SPARK-25644][SS] Fix java foreachBatch in DataStreamWriter

## What changes were proposed in this pull request?

The java `foreachBatch` API in `DataStreamWriter` should accept 
`java.lang.Long` rather `scala.Long`.

## How was this patch tested?

New java test.

Closes #22633 from zsxwing/fix-java-foreachbatch.

Authored-by: Shixiong Zhu 
Signed-off-by: Shixiong Zhu 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/7dcc90fb
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/7dcc90fb
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/7dcc90fb

Branch: refs/heads/master
Commit: 7dcc90fbb8dc75077819a5d8c42652f0c84424b5
Parents: 434ada1
Author: Shixiong Zhu 
Authored: Fri Oct 5 10:45:15 2018 -0700
Committer: Shixiong Zhu 
Committed: Fri Oct 5 10:45:15 2018 -0700

--
 .../spark/sql/streaming/DataStreamWriter.scala  |  2 +-
 .../JavaDataStreamReaderWriterSuite.java| 89 
 2 files changed, 90 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/7dcc90fb/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
index e9a1521..b23e86a 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
@@ -380,7 +380,7 @@ final class DataStreamWriter[T] private[sql](ds: 
Dataset[T]) {
* @since 2.4.0
*/
   @InterfaceStability.Evolving
-  def foreachBatch(function: VoidFunction2[Dataset[T], Long]): 
DataStreamWriter[T] = {
+  def foreachBatch(function: VoidFunction2[Dataset[T], java.lang.Long]): 
DataStreamWriter[T] = {
 foreachBatch((batchDs: Dataset[T], batchId: Long) => 
function.call(batchDs, batchId))
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/7dcc90fb/sql/core/src/test/java/test/org/apache/spark/sql/streaming/JavaDataStreamReaderWriterSuite.java
--
diff --git 
a/sql/core/src/test/java/test/org/apache/spark/sql/streaming/JavaDataStreamReaderWriterSuite.java
 
b/sql/core/src/test/java/test/org/apache/spark/sql/streaming/JavaDataStreamReaderWriterSuite.java
new file mode 100644
index 000..48cdb26
--- /dev/null
+++ 
b/sql/core/src/test/java/test/org/apache/spark/sql/streaming/JavaDataStreamReaderWriterSuite.java
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package test.org.apache.spark.sql.streaming;
+
+import java.io.File;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.spark.api.java.function.VoidFunction2;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.ForeachWriter;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.streaming.StreamingQuery;
+import org.apache.spark.sql.test.TestSparkSession;
+import org.apache.spark.util.Utils;
+
+public class JavaDataStreamReaderWriterSuite {
+  private SparkSession spark;
+  private String input;
+
+  @Before
+  public void setUp() {
+spark = new TestSparkSession();
+input = Utils.createTempDir(System.getProperty("java.io.tmpdir"), 
"input").toString();
+  }
+
+  @After
+  public void tearDown() {
+try {
+  Utils.deleteRecursively(new File(input));
+} finally {
+  spark.stop();
+  spark = null;
+}
+  }
+
+  @Test
+  public void testForeachBatchAPI() {
+StreamingQuery query = spark
+  .readStream()
+  .textFile(input)
+  .writeStream()
+  .foreachBatch(new VoidFunction2, Long>() {
+@Override
+public void call(Dataset v1, Long v2) throws Exception {}
+  })
+  .start();

spark git commit: [SPARK-25449][CORE] Heartbeat shouldn't include accumulators for zero metrics

2018-09-28 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/master a28146568 -> 9362c5cc2


[SPARK-25449][CORE] Heartbeat shouldn't include accumulators for zero metrics

## What changes were proposed in this pull request?

Heartbeat shouldn't include accumulators for zero metrics.

Heartbeats sent from executors to the driver every 10 seconds contain metrics 
and are generally on the order of a few KBs. However, for large jobs with lots 
of tasks, heartbeats can be on the order of tens of MBs, causing tasks to die 
with heartbeat failures. We can mitigate this by not sending zero metrics to 
the driver.

## How was this patch tested?

(Please explain how this patch was tested. E.g. unit tests, integration tests, 
manual tests)
(If this patch involves UI changes, please attach a screenshot; otherwise, 
remove this)

Please review http://spark.apache.org/contributing.html before opening a pull 
request.

Closes #22473 from mukulmurthy/25449-heartbeat.

Authored-by: Mukul Murthy 
Signed-off-by: Shixiong Zhu 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/9362c5cc
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/9362c5cc
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/9362c5cc

Branch: refs/heads/master
Commit: 9362c5cc273fdd09f9b3b512e2f6b64bcefc25ab
Parents: a281465
Author: Mukul Murthy 
Authored: Fri Sep 28 16:34:17 2018 -0700
Committer: Shixiong Zhu 
Committed: Fri Sep 28 16:34:17 2018 -0700

--
 .../main/scala/org/apache/spark/SparkConf.scala |  11 +-
 .../scala/org/apache/spark/SparkContext.scala   |   2 +-
 .../org/apache/spark/executor/Executor.scala|  40 +--
 .../apache/spark/internal/config/package.scala  |  14 +++
 .../apache/spark/executor/ExecutorSuite.scala   | 111 +--
 .../MesosCoarseGrainedSchedulerBackend.scala|   3 +-
 6 files changed, 154 insertions(+), 27 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/9362c5cc/core/src/main/scala/org/apache/spark/SparkConf.scala
--
diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala 
b/core/src/main/scala/org/apache/spark/SparkConf.scala
index e0f98f1..81aa31d 100644
--- a/core/src/main/scala/org/apache/spark/SparkConf.scala
+++ b/core/src/main/scala/org/apache/spark/SparkConf.scala
@@ -609,13 +609,14 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable 
with Logging with Seria
 require(!encryptionEnabled || get(NETWORK_AUTH_ENABLED),
   s"${NETWORK_AUTH_ENABLED.key} must be enabled when enabling encryption.")
 
-val executorTimeoutThreshold = getTimeAsSeconds("spark.network.timeout", 
"120s")
-val executorHeartbeatInterval = 
getTimeAsSeconds("spark.executor.heartbeatInterval", "10s")
+val executorTimeoutThresholdMs =
+  getTimeAsSeconds("spark.network.timeout", "120s") * 1000
+val executorHeartbeatIntervalMs = get(EXECUTOR_HEARTBEAT_INTERVAL)
 // If spark.executor.heartbeatInterval bigger than spark.network.timeout,
 // it will almost always cause ExecutorLostFailure. See SPARK-22754.
-require(executorTimeoutThreshold > executorHeartbeatInterval, "The value 
of " +
-  s"spark.network.timeout=${executorTimeoutThreshold}s must be no less 
than the value of " +
-  s"spark.executor.heartbeatInterval=${executorHeartbeatInterval}s.")
+require(executorTimeoutThresholdMs > executorHeartbeatIntervalMs, "The 
value of " +
+  s"spark.network.timeout=${executorTimeoutThresholdMs}ms must be no less 
than the value of " +
+  s"spark.executor.heartbeatInterval=${executorHeartbeatIntervalMs}ms.")
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/spark/blob/9362c5cc/core/src/main/scala/org/apache/spark/SparkContext.scala
--
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala 
b/core/src/main/scala/org/apache/spark/SparkContext.scala
index d943087..0a66dae 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -499,7 +499,7 @@ class SparkContext(config: SparkConf) extends Logging {
 
 // create and start the heartbeater for collecting memory metrics
 _heartbeater = new Heartbeater(env.memoryManager, reportHeartBeat, 
"driver-heartbeater",
-  conf.getTimeAsMs("spark.executor.heartbeatInterval", "10s"))
+  conf.get(EXECUTOR_HEARTBEAT_INTERVAL))
 _heartbeater.start()
 
 // start TaskScheduler after taskScheduler sets DAGScheduler reference in 
DAGScheduler's

http://git-wip-us.apache.org/repos/asf/spark/blob/9362c5cc/core/src/main/scala/org/apache/spark/executor/Executor.scala
--
diff --git 

spark git commit: [SPARK-25495][SS] FetchedData.reset should reset all fields

2018-09-25 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/branch-2.4 a709718da -> 544f86a69


[SPARK-25495][SS] FetchedData.reset should reset all fields

## What changes were proposed in this pull request?

`FetchedData.reset` should reset `_nextOffsetInFetchedData` and 
`_offsetAfterPoll`. Otherwise it will cause inconsistent cached data and may 
make Kafka connector return wrong results.

## How was this patch tested?

The new unit test.

Closes #22507 from zsxwing/fix-kafka-reset.

Lead-authored-by: Shixiong Zhu 
Co-authored-by: Shixiong Zhu 
Signed-off-by: Shixiong Zhu 
(cherry picked from commit 66d29870c09e6050dd846336e596faaa8b0d14ad)
Signed-off-by: Shixiong Zhu 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/544f86a6
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/544f86a6
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/544f86a6

Branch: refs/heads/branch-2.4
Commit: 544f86a69bba94dfcb241e41c799ed63ef4210fc
Parents: a709718
Author: Shixiong Zhu 
Authored: Tue Sep 25 11:42:27 2018 -0700
Committer: Shixiong Zhu 
Committed: Tue Sep 25 11:42:39 2018 -0700

--
 .../spark/sql/kafka010/KafkaDataConsumer.scala  |  5 +-
 .../kafka010/KafkaMicroBatchSourceSuite.scala   | 52 
 2 files changed, 56 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/544f86a6/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataConsumer.scala
--
diff --git 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataConsumer.scala
 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataConsumer.scala
index ceb9e31..7b1314b 100644
--- 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataConsumer.scala
+++ 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataConsumer.scala
@@ -134,6 +134,8 @@ private[kafka010] case class InternalKafkaConsumer(
 /** Reset the internal pre-fetched data. */
 def reset(): Unit = {
   _records = ju.Collections.emptyListIterator()
+  _nextOffsetInFetchedData = UNKNOWN_OFFSET
+  _offsetAfterPoll = UNKNOWN_OFFSET
 }
 
 /**
@@ -361,8 +363,9 @@ private[kafka010] case class InternalKafkaConsumer(
   if (offset < fetchedData.offsetAfterPoll) {
 // Offsets in [offset, fetchedData.offsetAfterPoll) are invisible. 
Return a record to ask
 // the next call to start from `fetchedData.offsetAfterPoll`.
+val nextOffsetToFetch = fetchedData.offsetAfterPoll
 fetchedData.reset()
-return fetchedRecord.withRecord(null, fetchedData.offsetAfterPoll)
+return fetchedRecord.withRecord(null, nextOffsetToFetch)
   } else {
 // Fetch records from Kafka and update `fetchedData`.
 fetchData(offset, pollTimeoutMs)

http://git-wip-us.apache.org/repos/asf/spark/blob/544f86a6/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
--
diff --git 
a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
 
b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
index 65615fd..e0b6d8c 100644
--- 
a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
+++ 
b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
@@ -853,6 +853,58 @@ abstract class KafkaMicroBatchSourceSuiteBase extends 
KafkaSourceSuiteBase {
   )
 }
   }
+
+  test("SPARK-25495: FetchedData.reset should reset all fields") {
+val topic = newTopic()
+val topicPartition = new TopicPartition(topic, 0)
+testUtils.createTopic(topic, partitions = 1)
+
+val ds = spark
+  .readStream
+  .format("kafka")
+  .option("kafka.bootstrap.servers", testUtils.brokerAddress)
+  .option("kafka.metadata.max.age.ms", "1")
+  .option("kafka.isolation.level", "read_committed")
+  .option("subscribe", topic)
+  .option("startingOffsets", "earliest")
+  .load()
+  .select($"value".as[String])
+
+testUtils.withTranscationalProducer { producer =>
+  producer.beginTransaction()
+  (0 to 3).foreach { i =>
+producer.send(new ProducerRecord[String, String](topic, 
i.toString)).get()
+  }
+  producer.commitTransaction()
+}
+testUtils.waitUntilOffsetAppears(topicPartition, 5)
+
+val q = ds.writeStream.foreachBatch { (ds, epochId) =&

spark git commit: [SPARK-25495][SS] FetchedData.reset should reset all fields

2018-09-25 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/master 04db03537 -> 66d29870c


[SPARK-25495][SS] FetchedData.reset should reset all fields

## What changes were proposed in this pull request?

`FetchedData.reset` should reset `_nextOffsetInFetchedData` and 
`_offsetAfterPoll`. Otherwise it will cause inconsistent cached data and may 
make Kafka connector return wrong results.

## How was this patch tested?

The new unit test.

Closes #22507 from zsxwing/fix-kafka-reset.

Lead-authored-by: Shixiong Zhu 
Co-authored-by: Shixiong Zhu 
Signed-off-by: Shixiong Zhu 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/66d29870
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/66d29870
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/66d29870

Branch: refs/heads/master
Commit: 66d29870c09e6050dd846336e596faaa8b0d14ad
Parents: 04db035
Author: Shixiong Zhu 
Authored: Tue Sep 25 11:42:27 2018 -0700
Committer: Shixiong Zhu 
Committed: Tue Sep 25 11:42:27 2018 -0700

--
 .../spark/sql/kafka010/KafkaDataConsumer.scala  |  5 +-
 .../kafka010/KafkaMicroBatchSourceSuite.scala   | 52 
 2 files changed, 56 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/66d29870/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataConsumer.scala
--
diff --git 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataConsumer.scala
 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataConsumer.scala
index ceb9e31..7b1314b 100644
--- 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataConsumer.scala
+++ 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataConsumer.scala
@@ -134,6 +134,8 @@ private[kafka010] case class InternalKafkaConsumer(
 /** Reset the internal pre-fetched data. */
 def reset(): Unit = {
   _records = ju.Collections.emptyListIterator()
+  _nextOffsetInFetchedData = UNKNOWN_OFFSET
+  _offsetAfterPoll = UNKNOWN_OFFSET
 }
 
 /**
@@ -361,8 +363,9 @@ private[kafka010] case class InternalKafkaConsumer(
   if (offset < fetchedData.offsetAfterPoll) {
 // Offsets in [offset, fetchedData.offsetAfterPoll) are invisible. 
Return a record to ask
 // the next call to start from `fetchedData.offsetAfterPoll`.
+val nextOffsetToFetch = fetchedData.offsetAfterPoll
 fetchedData.reset()
-return fetchedRecord.withRecord(null, fetchedData.offsetAfterPoll)
+return fetchedRecord.withRecord(null, nextOffsetToFetch)
   } else {
 // Fetch records from Kafka and update `fetchedData`.
 fetchData(offset, pollTimeoutMs)

http://git-wip-us.apache.org/repos/asf/spark/blob/66d29870/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
--
diff --git 
a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
 
b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
index e5f0088..39c2cde 100644
--- 
a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
+++ 
b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
@@ -874,6 +874,58 @@ abstract class KafkaMicroBatchSourceSuiteBase extends 
KafkaSourceSuiteBase {
   )
 }
   }
+
+  test("SPARK-25495: FetchedData.reset should reset all fields") {
+val topic = newTopic()
+val topicPartition = new TopicPartition(topic, 0)
+testUtils.createTopic(topic, partitions = 1)
+
+val ds = spark
+  .readStream
+  .format("kafka")
+  .option("kafka.bootstrap.servers", testUtils.brokerAddress)
+  .option("kafka.metadata.max.age.ms", "1")
+  .option("kafka.isolation.level", "read_committed")
+  .option("subscribe", topic)
+  .option("startingOffsets", "earliest")
+  .load()
+  .select($"value".as[String])
+
+testUtils.withTranscationalProducer { producer =>
+  producer.beginTransaction()
+  (0 to 3).foreach { i =>
+producer.send(new ProducerRecord[String, String](topic, 
i.toString)).get()
+  }
+  producer.commitTransaction()
+}
+testUtils.waitUntilOffsetAppears(topicPartition, 5)
+
+val q = ds.writeStream.foreachBatch { (ds, epochId) =>
+  if (epochId == 0) {
+// Send more message before the tasks of the current batch sta

spark git commit: [SPARK-25288][TESTS] Fix flaky Kafka transaction tests

2018-08-31 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/master f29c2b528 -> aa70a0a1a


[SPARK-25288][TESTS] Fix flaky Kafka transaction tests

## What changes were proposed in this pull request?

Here are the failures:

http://spark-tests.appspot.com/test-details?suite_name=org.apache.spark.sql.kafka010.KafkaRelationSuite_name=read+Kafka+transactional+messages%3A+read_committed
http://spark-tests.appspot.com/test-details?suite_name=org.apache.spark.sql.kafka010.KafkaMicroBatchV1SourceSuite_name=read+Kafka+transactional+messages%3A+read_committed
http://spark-tests.appspot.com/test-details?suite_name=org.apache.spark.sql.kafka010.KafkaMicroBatchV2SourceSuite_name=read+Kafka+transactional+messages%3A+read_committed

I found the Kafka consumer may not see the committed messages for a short time. 
This PR just adds a new method `waitUntilOffsetAppears` and uses it to make 
sure the consumer can see a specified offset before checking the result.

## How was this patch tested?

Jenkins

Closes #22293 from zsxwing/SPARK-25288.

Authored-by: Shixiong Zhu 
Signed-off-by: Shixiong Zhu 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/aa70a0a1
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/aa70a0a1
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/aa70a0a1

Branch: refs/heads/master
Commit: aa70a0a1a434e8a4b1d4dde00e20b865bb70b8dd
Parents: f29c2b5
Author: Shixiong Zhu 
Authored: Thu Aug 30 23:23:11 2018 -0700
Committer: Shixiong Zhu 
Committed: Thu Aug 30 23:23:11 2018 -0700

--
 .../kafka010/KafkaMicroBatchSourceSuite.scala   | 34 
 .../spark/sql/kafka010/KafkaRelationSuite.scala |  7 
 .../spark/sql/kafka010/KafkaTestUtils.scala | 10 ++
 3 files changed, 37 insertions(+), 14 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/aa70a0a1/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
--
diff --git 
a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
 
b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
index eb66cca..78249f7 100644
--- 
a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
+++ 
b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
@@ -160,14 +160,18 @@ abstract class KafkaSourceTest extends StreamTest with 
SharedSQLContext with Kaf
   }
 
   object WithOffsetSync {
-def apply(topic: String)(func: () => Unit): StreamAction = {
+/**
+ * Run `func` to write some Kafka messages and wait until the latest 
offset of the given
+ * `TopicPartition` is not less than `expectedOffset`.
+ */
+def apply(
+topicPartition: TopicPartition,
+expectedOffset: Long)(func: () => Unit): StreamAction = {
   Execute("Run Kafka Producer")(_ => {
 func()
 // This is a hack for the race condition that the committed message 
may be not visible to
 // consumer for a short time.
-// Looks like after the following call returns, the consumer can 
always read the committed
-// messages.
-testUtils.getLatestOffsets(Set(topic))
+testUtils.waitUntilOffsetAppears(topicPartition, expectedOffset)
   })
 }
   }
@@ -652,13 +656,14 @@ abstract class KafkaMicroBatchSourceSuiteBase extends 
KafkaSourceSuiteBase {
   }
 }
 
+val topicPartition = new TopicPartition(topic, 0)
 // The message values are the same as their offsets to make the test easy 
to follow
 testUtils.withTranscationalProducer { producer =>
   testStream(mapped)(
 StartStream(ProcessingTime(100), clock),
 waitUntilBatchProcessed,
 CheckAnswer(),
-WithOffsetSync(topic) { () =>
+WithOffsetSync(topicPartition, expectedOffset = 5) { () =>
   // Send 5 messages. They should be visible only after being 
committed.
   producer.beginTransaction()
   (0 to 4).foreach { i =>
@@ -669,7 +674,7 @@ abstract class KafkaMicroBatchSourceSuiteBase extends 
KafkaSourceSuiteBase {
 waitUntilBatchProcessed,
 // Should not see any uncommitted messages
 CheckNewAnswer(),
-WithOffsetSync(topic) { () =>
+WithOffsetSync(topicPartition, expectedOffset = 6) { () =>
   producer.commitTransaction()
 },
 AdvanceManualClock(100),
@@ -678,7 +683,7 @@ abstract class KafkaMicroBatchSourceSuiteBase extends 
KafkaSourceSuiteBase {
 AdvanceManualClock(100),
 waitUntilBatchProcessed,
 CheckNewAnswer(

spark git commit: [SPARK-25005][SS] Support non-consecutive offsets for Kafka

2018-08-28 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/master 592e3a42c -> 1149c4efb


[SPARK-25005][SS] Support non-consecutive offsets for Kafka

## What changes were proposed in this pull request?

As the user uses Kafka transactions to write data, the offsets in Kafka will be 
non-consecutive. It will contains some transaction (commit or abort) markers. 
In addition, if the consumer's `isolation.level` is `read_committed`, `poll` 
will not return aborted messages either. Hence, we will see non-consecutive 
offsets in the date returned by `poll`. However, as `seekToEnd` may move the 
offset point to these missing offsets, there are 4 possible corner cases we 
need to support:

- The whole batch contains no data messages
- The first offset in a batch is not a committed data message
- The last offset in a batch is not a committed data message
- There is a gap in the middle of a batch

They are all covered by the new unit tests.

## How was this patch tested?

The new unit tests.

Closes #22042 from zsxwing/kafka-transaction-read.

Authored-by: Shixiong Zhu 
Signed-off-by: Shixiong Zhu 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/1149c4ef
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/1149c4ef
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/1149c4ef

Branch: refs/heads/master
Commit: 1149c4efbc5ebe5b412d8f9c61558fef59179a9e
Parents: 592e3a4
Author: Shixiong Zhu 
Authored: Tue Aug 28 08:38:07 2018 -0700
Committer: Shixiong Zhu 
Committed: Tue Aug 28 08:38:07 2018 -0700

--
 .../kafka010/KafkaContinuousReadSupport.scala   |   2 +-
 .../spark/sql/kafka010/KafkaDataConsumer.scala  | 273 ++-
 .../kafka010/KafkaContinuousSourceSuite.scala   | 149 +-
 .../kafka010/KafkaMicroBatchSourceSuite.scala   | 255 -
 .../spark/sql/kafka010/KafkaRelationSuite.scala |  93 +++
 .../spark/sql/kafka010/KafkaTestUtils.scala |  22 +-
 6 files changed, 720 insertions(+), 74 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/1149c4ef/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReadSupport.scala
--
diff --git 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReadSupport.scala
 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReadSupport.scala
index 4a18839..1753a28 100644
--- 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReadSupport.scala
+++ 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReadSupport.scala
@@ -227,7 +227,7 @@ class KafkaContinuousPartitionReader(
 
 // This is a failOnDataLoss exception. Retry if nextKafkaOffset is 
within the data range,
 // or if it's the endpoint of the data range (i.e. the "true" next 
offset).
-case e: IllegalStateException  if 
e.getCause.isInstanceOf[OffsetOutOfRangeException] =>
+case e: IllegalStateException if 
e.getCause.isInstanceOf[OffsetOutOfRangeException] =>
   val range = consumer.getAvailableOffsetRange()
   if (range.latest >= nextKafkaOffset && range.earliest <= 
nextKafkaOffset) {
 // retry

http://git-wip-us.apache.org/repos/asf/spark/blob/1149c4ef/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataConsumer.scala
--
diff --git 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataConsumer.scala
 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataConsumer.scala
index 65046c1..ceb9e31 100644
--- 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataConsumer.scala
+++ 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataConsumer.scala
@@ -33,9 +33,19 @@ import org.apache.spark.util.UninterruptibleThread
 
 private[kafka010] sealed trait KafkaDataConsumer {
   /**
-   * Get the record for the given offset if available. Otherwise it will 
either throw error
-   * (if failOnDataLoss = true), or return the next available offset within 
[offset, untilOffset),
-   * or null.
+   * Get the record for the given offset if available.
+   *
+   * If the record is invisible (either a
+   * transaction message, or an aborted message when the consumer's 
`isolation.level` is
+   * `read_committed`), it will be skipped and this method will try to fetch 
next available record
+   * within [offset, untilOffset).
+   *
+   * This method also will try its best to detect data loss. If 
`failOnDataLoss` is `true`, it will
+   * throw an exception whe

spark git commit: [SPARK-25218][CORE] Fix potential resource leaks in TransportServer and SocketAuthHelper

2018-08-28 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/master 8198ea501 -> 592e3a42c


[SPARK-25218][CORE] Fix potential resource leaks in TransportServer and 
SocketAuthHelper

## What changes were proposed in this pull request?

Make sure TransportServer and SocketAuthHelper close the resources for all 
types of errors.

## How was this patch tested?

Jenkins

Closes #22210 from zsxwing/SPARK-25218.

Authored-by: Shixiong Zhu 
Signed-off-by: Shixiong Zhu 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/592e3a42
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/592e3a42
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/592e3a42

Branch: refs/heads/master
Commit: 592e3a42c20b72edd6e8b9dd07da367596f43da5
Parents: 8198ea5
Author: Shixiong Zhu 
Authored: Tue Aug 28 08:36:06 2018 -0700
Committer: Shixiong Zhu 
Committed: Tue Aug 28 08:36:06 2018 -0700

--
 .../buffer/FileSegmentManagedBuffer.java| 32 ++---
 .../spark/network/server/TransportServer.java   |  9 ++--
 .../spark/security/SocketAuthHelper.scala   | 50 +---
 3 files changed, 54 insertions(+), 37 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/592e3a42/common/network-common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java
--
diff --git 
a/common/network-common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java
 
b/common/network-common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java
index 8b8f989..45fee54 100644
--- 
a/common/network-common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java
+++ 
b/common/network-common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java
@@ -77,16 +77,16 @@ public final class FileSegmentManagedBuffer extends 
ManagedBuffer {
 return channel.map(FileChannel.MapMode.READ_ONLY, offset, length);
   }
 } catch (IOException e) {
+  String errorMessage = "Error in reading " + this;
   try {
 if (channel != null) {
   long size = channel.size();
-  throw new IOException("Error in reading " + this + " (actual file 
length " + size + ")",
-e);
+  errorMessage = "Error in reading " + this + " (actual file length " 
+ size + ")";
 }
   } catch (IOException ignored) {
 // ignore
   }
-  throw new IOException("Error in opening " + this, e);
+  throw new IOException(errorMessage, e);
 } finally {
   JavaUtils.closeQuietly(channel);
 }
@@ -95,26 +95,24 @@ public final class FileSegmentManagedBuffer extends 
ManagedBuffer {
   @Override
   public InputStream createInputStream() throws IOException {
 FileInputStream is = null;
+boolean shouldClose = true;
 try {
   is = new FileInputStream(file);
   ByteStreams.skipFully(is, offset);
-  return new LimitedInputStream(is, length);
+  InputStream r = new LimitedInputStream(is, length);
+  shouldClose = false;
+  return r;
 } catch (IOException e) {
-  try {
-if (is != null) {
-  long size = file.length();
-  throw new IOException("Error in reading " + this + " (actual file 
length " + size + ")",
-  e);
-}
-  } catch (IOException ignored) {
-// ignore
-  } finally {
+  String errorMessage = "Error in reading " + this;
+  if (is != null) {
+long size = file.length();
+errorMessage = "Error in reading " + this + " (actual file length " + 
size + ")";
+  }
+  throw new IOException(errorMessage, e);
+} finally {
+  if (shouldClose) {
 JavaUtils.closeQuietly(is);
   }
-  throw new IOException("Error in opening " + this, e);
-} catch (RuntimeException e) {
-  JavaUtils.closeQuietly(is);
-  throw e;
 }
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/592e3a42/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java
--
diff --git 
a/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java
 
b/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java
index d95ed22..9c85ab2 100644
--- 
a/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java
+++ 
b/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java
@@ -70,11 +70,14 @@ public class TransportServer

spark git commit: [SPARK-24882][FOLLOWUP] Fix flaky synchronization in Kafka tests.

2018-08-27 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/master 381a967a7 -> 810d59ce4


[SPARK-24882][FOLLOWUP] Fix flaky synchronization in Kafka tests.

## What changes were proposed in this pull request?

Fix flaky synchronization in Kafka tests - we need to use the scan config that 
was persisted rather than reconstructing it to identify the stream's current 
configuration.

We caught most instances of this in the original PR, but this one slipped 
through.

## How was this patch tested?

n/a

Closes #22245 from jose-torres/fixflake.

Authored-by: Jose Torres 
Signed-off-by: Shixiong Zhu 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/810d59ce
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/810d59ce
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/810d59ce

Branch: refs/heads/master
Commit: 810d59ce44e43f725d1b6d822166c2d97ff49929
Parents: 381a967
Author: Jose Torres 
Authored: Mon Aug 27 11:04:39 2018 -0700
Committer: Shixiong Zhu 
Committed: Mon Aug 27 11:04:39 2018 -0700

--
 .../spark/sql/kafka010/KafkaContinuousSourceSuite.scala   | 10 +-
 1 file changed, 5 insertions(+), 5 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/810d59ce/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala
--
diff --git 
a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala
 
b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala
index 3216650..5d68a14 100644
--- 
a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala
+++ 
b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala
@@ -18,7 +18,7 @@
 package org.apache.spark.sql.kafka010
 
 import org.apache.spark.sql.Dataset
-import 
org.apache.spark.sql.execution.datasources.v2.StreamingDataSourceV2Relation
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanExec
 import org.apache.spark.sql.streaming.Trigger
 
 // Run tests in KafkaSourceSuiteBase in continuous execution mode.
@@ -60,10 +60,10 @@ class KafkaContinuousSourceTopicDeletionSuite extends 
KafkaContinuousTest {
 testUtils.createTopic(topic2, partitions = 5)
 eventually(timeout(streamingTimeout)) {
   assert(
-query.lastExecution.logical.collectFirst {
-  case r: StreamingDataSourceV2Relation
-  if r.readSupport.isInstanceOf[KafkaContinuousReadSupport] =>
-
r.scanConfigBuilder.build().asInstanceOf[KafkaContinuousScanConfig]
+query.lastExecution.executedPlan.collectFirst {
+  case scan: DataSourceV2ScanExec
+if scan.readSupport.isInstanceOf[KafkaContinuousReadSupport] =>
+scan.scanConfig.asInstanceOf[KafkaContinuousScanConfig]
 }.exists { config =>
   // Ensure the new topic is present and the old topic is gone.
   config.knownPartitions.exists(_.topic == topic2)


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



spark git commit: [SPARK-25214][SS][FOLLOWUP] Fix the issue that Kafka v2 source may return duplicated records when `failOnDataLoss=false`

2018-08-25 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/master 6c66ab8b3 -> c17a8ff52


[SPARK-25214][SS][FOLLOWUP] Fix the issue that Kafka v2 source may return 
duplicated records when `failOnDataLoss=false`

## What changes were proposed in this pull request?

This is a follow up PR for #22207 to fix a potential flaky test. 
`processAllAvailable` doesn't work for continuous processing so we should not 
use it for a continuous query.

## How was this patch tested?

Jenkins.

Closes #22230 from zsxwing/SPARK-25214-2.

Authored-by: Shixiong Zhu 
Signed-off-by: Shixiong Zhu 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/c17a8ff5
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/c17a8ff5
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/c17a8ff5

Branch: refs/heads/master
Commit: c17a8ff52377871ab4ff96b648ebaf4112f0b5be
Parents: 6c66ab8
Author: Shixiong Zhu 
Authored: Sat Aug 25 09:17:40 2018 -0700
Committer: Shixiong Zhu 
Committed: Sat Aug 25 09:17:40 2018 -0700

--
 .../spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala| 8 ++--
 1 file changed, 6 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/c17a8ff5/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala
--
diff --git 
a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala
 
b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala
index 0ff341c..39c4e3f 100644
--- 
a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala
+++ 
b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala
@@ -80,7 +80,7 @@ trait KafkaMissingOffsetsTest extends SharedSQLContext {
   }
 }
 
-class KafkaDontFailOnDataLossSuite extends KafkaMissingOffsetsTest {
+class KafkaDontFailOnDataLossSuite extends StreamTest with 
KafkaMissingOffsetsTest {
 
   import testImplicits._
 
@@ -165,7 +165,11 @@ class KafkaDontFailOnDataLossSuite extends 
KafkaMissingOffsetsTest {
 .trigger(Trigger.Continuous(100))
 .start()
   try {
-query.processAllAvailable()
+// `processAllAvailable` doesn't work for continuous processing, so 
just wait until the last
+// record appears in the table.
+eventually(timeout(streamingTimeout)) {
+  assert(spark.table(table).as[String].collect().contains("49"))
+}
   } finally {
 query.stop()
   }


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



spark git commit: [SPARK-25214][SS] Fix the issue that Kafka v2 source may return duplicated records when `failOnDataLoss=false`

2018-08-24 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/master c20916a5d -> 8bb9414aa


[SPARK-25214][SS] Fix the issue that Kafka v2 source may return duplicated 
records when `failOnDataLoss=false`

## What changes were proposed in this pull request?

When there are missing offsets, Kafka v2 source may return duplicated records 
when `failOnDataLoss=false` because it doesn't skip missing offsets.

This PR fixes the issue and also adds regression tests for all Kafka readers.

## How was this patch tested?

New tests.

Closes #22207 from zsxwing/SPARK-25214.

Authored-by: Shixiong Zhu 
Signed-off-by: Shixiong Zhu 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/8bb9414a
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/8bb9414a
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/8bb9414a

Branch: refs/heads/master
Commit: 8bb9414aaff4a147db2d921dccdbd04c8eb4e5db
Parents: c20916a
Author: Shixiong Zhu 
Authored: Fri Aug 24 12:00:34 2018 -0700
Committer: Shixiong Zhu 
Committed: Fri Aug 24 12:00:34 2018 -0700

--
 .../kafka010/KafkaMicroBatchReadSupport.scala   |   2 +-
 .../spark/sql/kafka010/KafkaSourceRDD.scala |  38 ---
 .../kafka010/KafkaDontFailOnDataLossSuite.scala | 272 +++
 .../kafka010/KafkaMicroBatchSourceSuite.scala   | 139 +-
 4 files changed, 276 insertions(+), 175 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/8bb9414a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReadSupport.scala
--
diff --git 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReadSupport.scala
 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReadSupport.scala
index c31af60..70f37e3 100644
--- 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReadSupport.scala
+++ 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReadSupport.scala
@@ -341,6 +341,7 @@ private[kafka010] case class KafkaMicroBatchPartitionReader(
   val record = consumer.get(nextOffset, rangeToRead.untilOffset, 
pollTimeoutMs, failOnDataLoss)
   if (record != null) {
 nextRow = converter.toUnsafeRow(record)
+nextOffset = record.offset + 1
 true
   } else {
 false
@@ -352,7 +353,6 @@ private[kafka010] case class KafkaMicroBatchPartitionReader(
 
   override def get(): UnsafeRow = {
 assert(nextRow != null)
-nextOffset += 1
 nextRow
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/8bb9414a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala
--
diff --git 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala
 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala
index 8b4494d..f8b9005 100644
--- 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala
+++ 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala
@@ -77,44 +77,6 @@ private[kafka010] class KafkaSourceRDD(
 offsetRanges.zipWithIndex.map { case (o, i) => new 
KafkaSourceRDDPartition(i, o) }.toArray
   }
 
-  override def count(): Long = offsetRanges.map(_.size).sum
-
-  override def countApprox(timeout: Long, confidence: Double): 
PartialResult[BoundedDouble] = {
-val c = count
-new PartialResult(new BoundedDouble(c, 1.0, c, c), true)
-  }
-
-  override def isEmpty(): Boolean = count == 0L
-
-  override def take(num: Int): Array[ConsumerRecord[Array[Byte], Array[Byte]]] 
= {
-val nonEmptyPartitions =
-  
this.partitions.map(_.asInstanceOf[KafkaSourceRDDPartition]).filter(_.offsetRange.size
 > 0)
-
-if (num < 1 || nonEmptyPartitions.isEmpty) {
-  return new Array[ConsumerRecord[Array[Byte], Array[Byte]]](0)
-}
-
-// Determine in advance how many messages need to be taken from each 
partition
-val parts = nonEmptyPartitions.foldLeft(Map[Int, Int]()) { (result, part) 
=>
-  val remain = num - result.values.sum
-  if (remain > 0) {
-val taken = Math.min(remain, part.offsetRange.size)
-result + (part.index -> taken.toInt)
-  } else {
-result
-  }
-}
-
-val buf = new ArrayBuffer[ConsumerRecord[Array[Byte], Array[Byte]]]
-val res = context.runJob(
-  this,
-  (tc: TaskContext, it: Iterator[ConsumerRecord[Array[Byte], 
Array[Byte]]]) =>
-  it.take(parts(tc.partitionId)).toArray, parts.keys.toArray
-)
-res.foreach(buf ++= _)
-buf.toArra

spark git commit: [SPARK-25163][SQL] Fix flaky test: o.a.s.util.collection.ExternalAppendOnlyMapSuiteCheck

2018-08-22 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/master 310632498 -> 49a1993b1


[SPARK-25163][SQL] Fix flaky test: 
o.a.s.util.collection.ExternalAppendOnlyMapSuiteCheck

## What changes were proposed in this pull request?

`ExternalAppendOnlyMapSuiteCheck` test is flaky.

We use a `SparkListener` to collect spill metrics of completed stages. 
`withListener` runs the code that does spill. Spill status was checked after 
the code finishes but it was still in `withListener`. At that time it was 
possibly not all events to the listener bus are processed.

We should check spill status after all events are processed.

## How was this patch tested?

Locally ran unit tests.

Closes #22181 from viirya/SPARK-25163.

Authored-by: Liang-Chi Hsieh 
Signed-off-by: Shixiong Zhu 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/49a1993b
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/49a1993b
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/49a1993b

Branch: refs/heads/master
Commit: 49a1993b168accb6f188c682546f12ea568173c4
Parents: 3106324
Author: Liang-Chi Hsieh 
Authored: Wed Aug 22 14:17:05 2018 -0700
Committer: Shixiong Zhu 
Committed: Wed Aug 22 14:17:05 2018 -0700

--
 core/src/main/scala/org/apache/spark/TestUtils.scala | 10 ++
 1 file changed, 6 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/49a1993b/core/src/main/scala/org/apache/spark/TestUtils.scala
--
diff --git a/core/src/main/scala/org/apache/spark/TestUtils.scala 
b/core/src/main/scala/org/apache/spark/TestUtils.scala
index 6cc8fe1..c2ebd38 100644
--- a/core/src/main/scala/org/apache/spark/TestUtils.scala
+++ b/core/src/main/scala/org/apache/spark/TestUtils.scala
@@ -173,10 +173,11 @@ private[spark] object TestUtils {
* Run some code involving jobs submitted to the given context and assert 
that the jobs spilled.
*/
   def assertSpilled(sc: SparkContext, identifier: String)(body: => Unit): Unit 
= {
-withListener(sc, new SpillListener) { listener =>
+val listener = new SpillListener
+withListener(sc, listener) { _ =>
   body
-  assert(listener.numSpilledStages > 0, s"expected $identifier to spill, 
but did not")
 }
+assert(listener.numSpilledStages > 0, s"expected $identifier to spill, but 
did not")
   }
 
   /**
@@ -184,10 +185,11 @@ private[spark] object TestUtils {
* did not spill.
*/
   def assertNotSpilled(sc: SparkContext, identifier: String)(body: => Unit): 
Unit = {
-withListener(sc, new SpillListener) { listener =>
+val listener = new SpillListener
+withListener(sc, listener) { _ =>
   body
-  assert(listener.numSpilledStages == 0, s"expected $identifier to not 
spill, but did")
 }
+assert(listener.numSpilledStages == 0, s"expected $identifier to not 
spill, but did")
   }
 
   /**


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



spark git commit: [SPARK-25181][CORE] Limit Thread Pool size in BlockManager Master and Slave endpoints

2018-08-22 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/master 2381953ab -> 68ec4d641


[SPARK-25181][CORE] Limit Thread Pool size in BlockManager Master and Slave 
endpoints

## What changes were proposed in this pull request?

Limit Thread Pool size in BlockManager Master and Slave endpoints.

Currently, BlockManagerMasterEndpoint and BlockManagerSlaveEndpoint both have 
thread pools with nearly unbounded (Integer.MAX_VALUE) numbers of threads. In 
certain cases, this can lead to driver OOM errors. This change limits the 
thread pools to 100 threads; this should not break any existing behavior 
because any tasks beyond that number will get queued.

## How was this patch tested?

Manual testing

Please review http://spark.apache.org/contributing.html before opening a pull 
request.

Closes #22176 from mukulmurthy/25181-threads.

Authored-by: Mukul Murthy 
Signed-off-by: Shixiong Zhu 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/68ec4d64
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/68ec4d64
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/68ec4d64

Branch: refs/heads/master
Commit: 68ec4d641b87d2ab6a8cafc5d10c08253ae09e3d
Parents: 2381953
Author: Mukul Murthy 
Authored: Wed Aug 22 10:36:20 2018 -0700
Committer: Shixiong Zhu 
Committed: Wed Aug 22 10:36:20 2018 -0700

--
 .../org/apache/spark/storage/BlockManagerMasterEndpoint.scala | 3 ++-
 .../org/apache/spark/storage/BlockManagerSlaveEndpoint.scala  | 2 +-
 2 files changed, 3 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/68ec4d64/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala
--
diff --git 
a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala 
b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala
index 8e8f7d1..f984cf7 100644
--- 
a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala
+++ 
b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala
@@ -54,7 +54,8 @@ class BlockManagerMasterEndpoint(
   // Mapping from block id to the set of block managers that have the block.
   private val blockLocations = new JHashMap[BlockId, 
mutable.HashSet[BlockManagerId]]
 
-  private val askThreadPool = 
ThreadUtils.newDaemonCachedThreadPool("block-manager-ask-thread-pool")
+  private val askThreadPool =
+ThreadUtils.newDaemonCachedThreadPool("block-manager-ask-thread-pool", 100)
   private implicit val askExecutionContext = 
ExecutionContext.fromExecutorService(askThreadPool)
 
   private val topologyMapper = {

http://git-wip-us.apache.org/repos/asf/spark/blob/68ec4d64/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveEndpoint.scala
--
diff --git 
a/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveEndpoint.scala 
b/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveEndpoint.scala
index 742cf4f..67544b2 100644
--- 
a/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveEndpoint.scala
+++ 
b/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveEndpoint.scala
@@ -37,7 +37,7 @@ class BlockManagerSlaveEndpoint(
   extends ThreadSafeRpcEndpoint with Logging {
 
   private val asyncThreadPool =
-
ThreadUtils.newDaemonCachedThreadPool("block-manager-slave-async-thread-pool")
+
ThreadUtils.newDaemonCachedThreadPool("block-manager-slave-async-thread-pool", 
100)
   private implicit val asyncExecutionContext = 
ExecutionContext.fromExecutorService(asyncThreadPool)
 
   // Operations that involve removing blocks may be slow and should be done 
asynchronously


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



spark git commit: [SPARK-25116][TESTS] Fix the Kafka cluster leak and clean up cached producers

2018-08-17 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/master 8b0e94d89 -> da2dc6929


[SPARK-25116][TESTS] Fix the Kafka cluster leak and clean up cached producers

## What changes were proposed in this pull request?

KafkaContinuousSinkSuite leaks a Kafka cluster because both KafkaSourceTest and 
KafkaContinuousSinkSuite create a Kafka cluster but `afterAll` only shuts down 
one cluster. This leaks a Kafka cluster and causes that some Kafka thread crash 
and kill JVM when SBT is trying to clean up tests.

This PR fixes the leak and also adds a shut down hook to detect Kafka cluster 
leak.

In additions, it also fixes `AdminClient` leak and cleans up cached producers 
(When a record is writtn using a producer, the producer will keep refreshing 
the topic and I don't find an API to clear it except closing the producer) to 
eliminate the following annoying logs:
```
8/13 15:34:42.568 kafka-admin-client-thread | adminclient-4 WARN NetworkClient: 
[AdminClient clientId=adminclient-4] Connection to node 0 could not be 
established. Broker may not be available.
18/08/13 15:34:42.570 kafka-admin-client-thread | adminclient-6 WARN 
NetworkClient: [AdminClient clientId=adminclient-6] Connection to node 0 could 
not be established. Broker may not be available.
18/08/13 15:34:42.606 kafka-admin-client-thread | adminclient-8 WARN 
NetworkClient: [AdminClient clientId=adminclient-8] Connection to node -1 could 
not be established. Broker may not be available.
18/08/13 15:34:42.729 kafka-producer-network-thread | producer-797 WARN 
NetworkClient: [Producer clientId=producer-797] Connection to node -1 could not 
be established. Broker may not be available.
18/08/13 15:34:42.906 kafka-producer-network-thread | producer-1598 WARN 
NetworkClient: [Producer clientId=producer-1598] Connection to node 0 could not 
be established. Broker may not be available.
```

I also reverted 
https://github.com/apache/spark/pull/22097/commits/b5eb54244ed573c8046f5abf7bf087f5f08dba58
 introduced by #22097 since it doesn't help.

## How was this patch tested?

Jenkins

Closes #22106 from zsxwing/SPARK-25116.

Authored-by: Shixiong Zhu 
Signed-off-by: Shixiong Zhu 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/da2dc692
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/da2dc692
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/da2dc692

Branch: refs/heads/master
Commit: da2dc69291cda8c8e7bb6b4a15001f768a97f65e
Parents: 8b0e94d
Author: Shixiong Zhu 
Authored: Fri Aug 17 14:21:08 2018 -0700
Committer: Shixiong Zhu 
Committed: Fri Aug 17 14:21:08 2018 -0700

--
 .../sql/kafka010/CachedKafkaProducer.scala  |  8 +-
 .../sql/kafka010/KafkaContinuousReader.scala|  2 +-
 .../sql/kafka010/CachedKafkaProducerSuite.scala |  5 +-
 .../sql/kafka010/KafkaContinuousSinkSuite.scala |  7 +-
 .../kafka010/KafkaMicroBatchSourceSuite.scala   |  2 +-
 .../spark/sql/kafka010/KafkaRelationSuite.scala |  3 +-
 .../spark/sql/kafka010/KafkaSinkSuite.scala |  2 +-
 .../apache/spark/sql/kafka010/KafkaTest.scala   | 32 +++
 .../spark/sql/kafka010/KafkaTestUtils.scala | 91 ++--
 .../streaming/kafka010/KafkaTestUtils.scala | 89 +--
 10 files changed, 132 insertions(+), 109 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/da2dc692/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaProducer.scala
--
diff --git 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaProducer.scala
 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaProducer.scala
index 571140b..cd680ad 100644
--- 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaProducer.scala
+++ 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaProducer.scala
@@ -33,8 +33,12 @@ private[kafka010] object CachedKafkaProducer extends Logging 
{
 
   private type Producer = KafkaProducer[Array[Byte], Array[Byte]]
 
+  private val defaultCacheExpireTimeout = TimeUnit.MINUTES.toMillis(10)
+
   private lazy val cacheExpireTimeout: Long =
-SparkEnv.get.conf.getTimeAsMs("spark.kafka.producer.cache.timeout", "10m")
+Option(SparkEnv.get).map(_.conf.getTimeAsMs(
+  "spark.kafka.producer.cache.timeout",
+  s"${defaultCacheExpireTimeout}ms")).getOrElse(defaultCacheExpireTimeout)
 
   private val cacheLoader = new CacheLoader[Seq[(String, Object)], Producer] {
 override def load(config: Seq[(String, Object)]): Producer = {
@@ -102,7 +106,7 @@ private[kafka010] object CachedKafkaProducer extends 
Logging {
 }
   }
 
-  private def clear(): Unit = {
+  private[k

spark git commit: [SPARK-18057][FOLLOW-UP] Use 127.0.0.1 to avoid zookeeper picking up an ipv6 address

2018-08-14 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/master 42263fd0c -> 80784a1de


[SPARK-18057][FOLLOW-UP] Use 127.0.0.1 to avoid zookeeper picking up an ipv6 
address

## What changes were proposed in this pull request?

I'm still seeing the Kafka tests failed randomly due to 
`kafka.zookeeper.ZooKeeperClientTimeoutException: Timed out waiting for 
connection while in state: CONNECTING`. I checked the test output and saw 
zookeeper picked up an ipv6 address. Most details can be found in 
https://issues.apache.org/jira/browse/KAFKA-7193

This PR just uses `127.0.0.1` rather than `localhost` to make sure zookeeper 
will never use an ipv6 address.

## How was this patch tested?

Jenkins

Closes #22097 from zsxwing/fix-zookeeper-connect.

Authored-by: Shixiong Zhu 
Signed-off-by: Shixiong Zhu 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/80784a1d
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/80784a1d
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/80784a1d

Branch: refs/heads/master
Commit: 80784a1de8d02536a94f3fd08ef632777478ab14
Parents: 42263fd
Author: Shixiong Zhu 
Authored: Tue Aug 14 09:57:01 2018 -0700
Committer: Shixiong Zhu 
Committed: Tue Aug 14 09:57:01 2018 -0700

--
 .../spark/sql/kafka010/KafkaTestUtils.scala | 80 
 .../streaming/kafka010/KafkaTestUtils.scala | 79 +++
 2 files changed, 96 insertions(+), 63 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/80784a1d/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala
--
diff --git 
a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala
 
b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala
index d89cccd..e58d183 100644
--- 
a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala
+++ 
b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala
@@ -39,6 +39,7 @@ import org.apache.kafka.clients.producer._
 import org.apache.kafka.common.TopicPartition
 import org.apache.kafka.common.network.ListenerName
 import org.apache.kafka.common.serialization.{StringDeserializer, 
StringSerializer}
+import org.apache.kafka.common.utils.Exit
 import org.apache.zookeeper.server.{NIOServerCnxnFactory, ZooKeeperServer}
 import org.scalatest.concurrent.Eventually._
 import org.scalatest.time.SpanSugar._
@@ -56,7 +57,7 @@ import org.apache.spark.util.Utils
 class KafkaTestUtils(withBrokerProps: Map[String, Object] = Map.empty) extends 
Logging {
 
   // Zookeeper related configurations
-  private val zkHost = "localhost"
+  private val zkHost = "127.0.0.1"
   private var zkPort: Int = 0
   private val zkConnectionTimeout = 6
   private val zkSessionTimeout = 6000
@@ -67,7 +68,7 @@ class KafkaTestUtils(withBrokerProps: Map[String, Object] = 
Map.empty) extends L
   private var adminClient: AdminClient = null
 
   // Kafka broker related configurations
-  private val brokerHost = "localhost"
+  private val brokerHost = "127.0.0.1"
   private var brokerPort = 0
   private var brokerConf: KafkaConfig = _
 
@@ -138,40 +139,55 @@ class KafkaTestUtils(withBrokerProps: Map[String, Object] 
= Map.empty) extends L
 
   /** Teardown the whole servers, including Kafka broker and Zookeeper */
   def teardown(): Unit = {
-brokerReady = false
-zkReady = false
-
-if (producer != null) {
-  producer.close()
-  producer = null
+// There is a race condition that may kill JVM when terminating the Kafka 
cluster. We set
+// a custom Procedure here during the termination in order to keep JVM 
running and not fail the
+// tests.
+val logExitEvent = new Exit.Procedure {
+  override def execute(statusCode: Int, message: String): Unit = {
+logError(s"Prevent Kafka from killing JVM (statusCode: $statusCode 
message: $message)")
+  }
 }
+Exit.setExitProcedure(logExitEvent)
+Exit.setHaltProcedure(logExitEvent)
+try {
+  brokerReady = false
+  zkReady = false
 
-if (server != null) {
-  server.shutdown()
-  server.awaitShutdown()
-  server = null
-}
+  if (producer != null) {
+producer.close()
+producer = null
+  }
 
-// On Windows, `logDirs` is left open even after Kafka server above is 
completely shut down
-// in some cases. It leads to test failures on Windows if the directory 
deletion failure
-// throws an exception.
-brokerConf.logDirs.foreach { f =>
-  try {
-Utils.deleteRecursively(new File(f))
-  } catch {
-case e: IOException 

spark git commit: [SPARK-25081][CORE] Nested spill in ShuffleExternalSorter should not access released memory page

2018-08-10 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/branch-2.3 7306ac71d -> 04c652064


[SPARK-25081][CORE] Nested spill in ShuffleExternalSorter should not access 
released memory page

## What changes were proposed in this pull request?

This issue is pretty similar to 
[SPARK-21907](https://issues.apache.org/jira/browse/SPARK-21907).

"allocateArray" in 
[ShuffleInMemorySorter.reset](https://github.com/apache/spark/blob/9b8521e53e56a53b44c02366a99f8a8ee1307bbf/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java#L99)
 may trigger a spill and cause ShuffleInMemorySorter access the released 
`array`. Another task may get the same memory page from the pool. This will 
cause two tasks access the same memory page. When a task reads memory written 
by another task, many types of failures may happen. Here are some examples I  
have seen:

- JVM crash. (This is easy to reproduce in a unit test as we fill newly 
allocated and deallocated memory with 0xa5 and 0x5a bytes which usually points 
to an invalid memory address)
- java.lang.IllegalArgumentException: Comparison method violates its general 
contract!
- java.lang.NullPointerException at 
org.apache.spark.memory.TaskMemoryManager.getPage(TaskMemoryManager.java:384)
- java.lang.UnsupportedOperationException: Cannot grow BufferHolder by size 
-536870912 because the size after growing exceeds size limitation 2147483632

This PR resets states in `ShuffleInMemorySorter.reset` before calling 
`allocateArray` to fix the issue.

## How was this patch tested?

The new unit test will make JVM crash without the fix.

Closes #22062 from zsxwing/SPARK-25081.

Authored-by: Shixiong Zhu 
Signed-off-by: Shixiong Zhu 
(cherry picked from commit f5aba657396bd4e2e03dd06491a2d169a99592a7)
Signed-off-by: Shixiong Zhu 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/04c65206
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/04c65206
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/04c65206

Branch: refs/heads/branch-2.3
Commit: 04c652064861720d991675b7f5b53f2bbca9d14d
Parents: 7306ac7
Author: Shixiong Zhu 
Authored: Fri Aug 10 10:53:44 2018 -0700
Committer: Shixiong Zhu 
Committed: Fri Aug 10 10:54:03 2018 -0700

--
 .../shuffle/sort/ShuffleInMemorySorter.java |  12 +-
 .../sort/ShuffleExternalSorterSuite.scala   | 111 +++
 2 files changed, 121 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/04c65206/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java
--
diff --git 
a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java 
b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java
index dc36809..0d06912 100644
--- 
a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java
+++ 
b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java
@@ -66,7 +66,7 @@ final class ShuffleInMemorySorter {
*/
   private int usableCapacity = 0;
 
-  private int initialSize;
+  private final int initialSize;
 
   ShuffleInMemorySorter(MemoryConsumer consumer, int initialSize, boolean 
useRadixSort) {
 this.consumer = consumer;
@@ -95,12 +95,20 @@ final class ShuffleInMemorySorter {
   }
 
   public void reset() {
+// Reset `pos` here so that `spill` triggered by the below `allocateArray` 
will be no-op.
+pos = 0;
 if (consumer != null) {
   consumer.freeArray(array);
+  // As `array` has been released, we should set it to  `null` to avoid 
accessing it before
+  // `allocateArray` returns. `usableCapacity` is also set to `0` to avoid 
any codes writing
+  // data to `ShuffleInMemorySorter` when `array` is `null` (e.g., in
+  // ShuffleExternalSorter.growPointerArrayIfNecessary, we may try to 
access
+  // `ShuffleInMemorySorter` when `allocateArray` throws 
SparkOutOfMemoryError).
+  array = null;
+  usableCapacity = 0;
   array = consumer.allocateArray(initialSize);
   usableCapacity = getUsableCapacity();
 }
-pos = 0;
   }
 
   public void expandPointerArray(LongArray newArray) {

http://git-wip-us.apache.org/repos/asf/spark/blob/04c65206/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleExternalSorterSuite.scala
--
diff --git 
a/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleExternalSorterSuite.scala
 
b/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleExternalSorterSuite.scala
new file mode 100644
index 000..b9f0e87
--- /dev/null
+++ 
b/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleExternalSorterSuite.scala
@@ -0,0 +1,111 @@
+/*
+ * Licensed 

spark git commit: [SPARK-25081][CORE] Nested spill in ShuffleExternalSorter should not access released memory page

2018-08-10 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/master 91cdab51c -> f5aba6573


[SPARK-25081][CORE] Nested spill in ShuffleExternalSorter should not access 
released memory page

## What changes were proposed in this pull request?

This issue is pretty similar to 
[SPARK-21907](https://issues.apache.org/jira/browse/SPARK-21907).

"allocateArray" in 
[ShuffleInMemorySorter.reset](https://github.com/apache/spark/blob/9b8521e53e56a53b44c02366a99f8a8ee1307bbf/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java#L99)
 may trigger a spill and cause ShuffleInMemorySorter access the released 
`array`. Another task may get the same memory page from the pool. This will 
cause two tasks access the same memory page. When a task reads memory written 
by another task, many types of failures may happen. Here are some examples I  
have seen:

- JVM crash. (This is easy to reproduce in a unit test as we fill newly 
allocated and deallocated memory with 0xa5 and 0x5a bytes which usually points 
to an invalid memory address)
- java.lang.IllegalArgumentException: Comparison method violates its general 
contract!
- java.lang.NullPointerException at 
org.apache.spark.memory.TaskMemoryManager.getPage(TaskMemoryManager.java:384)
- java.lang.UnsupportedOperationException: Cannot grow BufferHolder by size 
-536870912 because the size after growing exceeds size limitation 2147483632

This PR resets states in `ShuffleInMemorySorter.reset` before calling 
`allocateArray` to fix the issue.

## How was this patch tested?

The new unit test will make JVM crash without the fix.

Closes #22062 from zsxwing/SPARK-25081.

Authored-by: Shixiong Zhu 
Signed-off-by: Shixiong Zhu 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/f5aba657
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/f5aba657
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/f5aba657

Branch: refs/heads/master
Commit: f5aba657396bd4e2e03dd06491a2d169a99592a7
Parents: 91cdab5
Author: Shixiong Zhu 
Authored: Fri Aug 10 10:53:44 2018 -0700
Committer: Shixiong Zhu 
Committed: Fri Aug 10 10:53:44 2018 -0700

--
 .../shuffle/sort/ShuffleInMemorySorter.java |  12 +-
 .../sort/ShuffleExternalSorterSuite.scala   | 111 +++
 2 files changed, 121 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/f5aba657/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java
--
diff --git 
a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java 
b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java
index 8f49859..4b48599 100644
--- 
a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java
+++ 
b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java
@@ -65,7 +65,7 @@ final class ShuffleInMemorySorter {
*/
   private int usableCapacity = 0;
 
-  private int initialSize;
+  private final int initialSize;
 
   ShuffleInMemorySorter(MemoryConsumer consumer, int initialSize, boolean 
useRadixSort) {
 this.consumer = consumer;
@@ -94,12 +94,20 @@ final class ShuffleInMemorySorter {
   }
 
   public void reset() {
+// Reset `pos` here so that `spill` triggered by the below `allocateArray` 
will be no-op.
+pos = 0;
 if (consumer != null) {
   consumer.freeArray(array);
+  // As `array` has been released, we should set it to  `null` to avoid 
accessing it before
+  // `allocateArray` returns. `usableCapacity` is also set to `0` to avoid 
any codes writing
+  // data to `ShuffleInMemorySorter` when `array` is `null` (e.g., in
+  // ShuffleExternalSorter.growPointerArrayIfNecessary, we may try to 
access
+  // `ShuffleInMemorySorter` when `allocateArray` throws 
SparkOutOfMemoryError).
+  array = null;
+  usableCapacity = 0;
   array = consumer.allocateArray(initialSize);
   usableCapacity = getUsableCapacity();
 }
-pos = 0;
   }
 
   public void expandPointerArray(LongArray newArray) {

http://git-wip-us.apache.org/repos/asf/spark/blob/f5aba657/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleExternalSorterSuite.scala
--
diff --git 
a/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleExternalSorterSuite.scala
 
b/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleExternalSorterSuite.scala
new file mode 100644
index 000..b9f0e87
--- /dev/null
+++ 
b/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleExternalSorterSuite.scala
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See th

spark git commit: [SPARK-24161][SS] Enable debug package feature on structured streaming

2018-08-06 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/master 3c96937c7 -> 87ca7396c


[SPARK-24161][SS] Enable debug package feature on structured streaming

## What changes were proposed in this pull request?

Currently, debug package has a implicit class "DebugQuery" which matches 
Dataset to provide debug features on Dataset class. It doesn't work with 
structured streaming: it requires query is already started, and the information 
can be retrieved from StreamingQuery, not Dataset. I guess that's why "explain" 
had to be placed to StreamingQuery whereas it already exists on Dataset.

This patch adds a new implicit class "DebugStreamQuery" which matches 
StreamingQuery to provide similar debug features on StreamingQuery class.

## How was this patch tested?

Added relevant unit tests.

Author: Jungtaek Lim 

Closes #21222 from HeartSaVioR/SPARK-24161.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/87ca7396
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/87ca7396
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/87ca7396

Branch: refs/heads/master
Commit: 87ca7396c7b21a87874d8ceb32e53119c609002c
Parents: 3c96937
Author: Jungtaek Lim 
Authored: Mon Aug 6 15:23:47 2018 -0700
Committer: Shixiong Zhu 
Committed: Mon Aug 6 15:23:47 2018 -0700

--
 .../spark/sql/execution/debug/package.scala |  59 +-
 .../spark/sql/streaming/StreamSuite.scala   | 116 +++
 2 files changed, 173 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/87ca7396/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala
index a717cbd..366e1fe 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala
@@ -29,6 +29,9 @@ import org.apache.spark.sql.catalyst.expressions.Attribute
 import org.apache.spark.sql.catalyst.expressions.codegen.{CodeFormatter, 
CodegenContext, ExprCode}
 import org.apache.spark.sql.catalyst.plans.physical.Partitioning
 import org.apache.spark.sql.catalyst.trees.TreeNodeRef
+import org.apache.spark.sql.execution.streaming.{StreamExecution, 
StreamingQueryWrapper}
+import 
org.apache.spark.sql.execution.streaming.continuous.WriteToContinuousDataSourceExec
+import org.apache.spark.sql.streaming.StreamingQuery
 import org.apache.spark.util.{AccumulatorV2, LongAccumulator}
 
 /**
@@ -40,6 +43,16 @@ import org.apache.spark.util.{AccumulatorV2, LongAccumulator}
  *   sql("SELECT 1").debug()
  *   sql("SELECT 1").debugCodegen()
  * }}}
+ *
+ * or for streaming case (structured streaming):
+ * {{{
+ *   import org.apache.spark.sql.execution.debug._
+ *   val query = df.writeStream.<...>.start()
+ *   query.debugCodegen()
+ * }}}
+ *
+ * Note that debug in structured streaming is not supported, because it 
doesn't make sense for
+ * streaming to execute batch once while main query is running concurrently.
  */
 package object debug {
 
@@ -89,13 +102,49 @@ package object debug {
   }
 
   /**
+   * Get WholeStageCodegenExec subtrees and the codegen in a query plan into 
one String
+   *
+   * @param query the streaming query for codegen
+   * @return single String containing all WholeStageCodegen subtrees and 
corresponding codegen
+   */
+  def codegenString(query: StreamingQuery): String = {
+val w = asStreamExecution(query)
+if (w.lastExecution != null) {
+  codegenString(w.lastExecution.executedPlan)
+} else {
+  "No physical plan. Waiting for data."
+}
+  }
+
+  /**
+   * Get WholeStageCodegenExec subtrees and the codegen in a query plan
+   *
+   * @param query the streaming query for codegen
+   * @return Sequence of WholeStageCodegen subtrees and corresponding codegen
+   */
+  def codegenStringSeq(query: StreamingQuery): Seq[(String, String)] = {
+val w = asStreamExecution(query)
+if (w.lastExecution != null) {
+  codegenStringSeq(w.lastExecution.executedPlan)
+} else {
+  Seq.empty
+}
+  }
+
+  private def asStreamExecution(query: StreamingQuery): StreamExecution = 
query match {
+case wrapper: StreamingQueryWrapper => wrapper.streamingQuery
+case q: StreamExecution => q
+case _ => throw new IllegalArgumentException("Parameter should be an 
instance of " +
+  "StreamExecution!")
+  }
+
+  /**
* Augments [[Dataset]]s with debug methods.
*/
   implicit class DebugQuery(query: Dataset[_]) extends Logging {
 def debug(): Unit = {
-  val plan = query.queryExecution.executedPlan
   val visited = new 

spark git commit: [SPARK-18057][FOLLOW-UP][SS] Update Kafka client version from 0.10.0.1 to 2.0.0

2018-08-03 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/master 8c14276c3 -> 4c27663cb


[SPARK-18057][FOLLOW-UP][SS] Update Kafka client version from 0.10.0.1 to 2.0.0

## What changes were proposed in this pull request?

Increase ZK timeout and harmonize configs across Kafka tests to resol…ve 
potentially flaky test failure

## How was this patch tested?

Existing tests

Author: Sean Owen 

Closes #21995 from srowen/SPARK-18057.3.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/4c27663c
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/4c27663c
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/4c27663c

Branch: refs/heads/master
Commit: 4c27663cb20f3cde7317ffcb2c9d42257a40057f
Parents: 8c14276
Author: Sean Owen 
Authored: Fri Aug 3 16:22:54 2018 -0700
Committer: Shixiong Zhu 
Committed: Fri Aug 3 16:22:54 2018 -0700

--
 .../scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala   | 1 +
 .../org/apache/spark/streaming/kafka010/KafkaTestUtils.scala   | 6 +-
 2 files changed, 6 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/4c27663c/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala
--
diff --git 
a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala
 
b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala
index 8229490..d89cccd 100644
--- 
a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala
+++ 
b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala
@@ -304,6 +304,7 @@ class KafkaTestUtils(withBrokerProps: Map[String, Object] = 
Map.empty) extends L
 props.put("port", brokerPort.toString)
 props.put("log.dir", Utils.createTempDir().getAbsolutePath)
 props.put("zookeeper.connect", zkAddress)
+props.put("zookeeper.connection.timeout.ms", "6")
 props.put("log.flush.interval.messages", "1")
 props.put("replica.socket.timeout.ms", "1500")
 props.put("delete.topic.enable", "true")

http://git-wip-us.apache.org/repos/asf/spark/blob/4c27663c/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaTestUtils.scala
--
diff --git 
a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaTestUtils.scala
 
b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaTestUtils.scala
index 2315baf..eef4c55 100644
--- 
a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaTestUtils.scala
+++ 
b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaTestUtils.scala
@@ -32,6 +32,7 @@ import kafka.api.Request
 import kafka.server.{KafkaConfig, KafkaServer}
 import kafka.utils.ZkUtils
 import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord}
+import org.apache.kafka.common.network.ListenerName
 import org.apache.kafka.common.serialization.StringSerializer
 import org.apache.zookeeper.server.{NIOServerCnxnFactory, ZooKeeperServer}
 
@@ -109,7 +110,7 @@ private[kafka010] class KafkaTestUtils extends Logging {
   brokerConf = new KafkaConfig(brokerConfiguration, doLog = false)
   server = new KafkaServer(brokerConf)
   server.startup()
-  brokerPort = server.boundPort(brokerConf.interBrokerListenerName)
+  brokerPort = server.boundPort(new ListenerName("PLAINTEXT"))
   (server, brokerPort)
 }, new SparkConf(), "KafkaBroker")
 
@@ -220,8 +221,11 @@ private[kafka010] class KafkaTestUtils extends Logging {
 props.put("port", brokerPort.toString)
 props.put("log.dir", brokerLogDir)
 props.put("zookeeper.connect", zkAddress)
+props.put("zookeeper.connection.timeout.ms", "6")
 props.put("log.flush.interval.messages", "1")
 props.put("replica.socket.timeout.ms", "1500")
+props.put("delete.topic.enable", "true")
+props.put("offsets.topic.num.partitions", "1")
 props.put("offsets.topic.replication.factor", "1")
 props.put("group.initial.rebalance.delay.ms", "10")
 props


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



spark git commit: [SPARK-24896][SQL] Uuid should produce different values for each execution in streaming query

2018-08-02 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/master efef55388 -> d0bc3ed67


[SPARK-24896][SQL] Uuid should produce different values for each execution in 
streaming query

## What changes were proposed in this pull request?

`Uuid`'s results depend on random seed given during analysis. Thus under 
streaming query, we will have the same uuids in each execution. This seems to 
be incorrect for streaming query execution.

## How was this patch tested?

Added test.

Author: Liang-Chi Hsieh 

Closes #21854 from viirya/uuid_in_streaming.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/d0bc3ed6
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/d0bc3ed6
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/d0bc3ed6

Branch: refs/heads/master
Commit: d0bc3ed6797e0c06f688b7b2ef6c26282a25b175
Parents: efef553
Author: Liang-Chi Hsieh 
Authored: Thu Aug 2 15:35:46 2018 -0700
Committer: Shixiong Zhu 
Committed: Thu Aug 2 15:35:46 2018 -0700

--
 .../streaming/IncrementalExecution.scala|  8 ++-
 .../sql/streaming/StreamingQuerySuite.scala | 22 +++-
 2 files changed, 28 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/d0bc3ed6/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala
index 6ae7f28..e9ffe12 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala
@@ -20,9 +20,11 @@ package org.apache.spark.sql.execution.streaming
 import java.util.UUID
 import java.util.concurrent.atomic.AtomicInteger
 
+import scala.util.Random
+
 import org.apache.spark.internal.Logging
 import org.apache.spark.sql.{AnalysisException, SparkSession, Strategy}
-import org.apache.spark.sql.catalyst.expressions.CurrentBatchTimestamp
+import org.apache.spark.sql.catalyst.expressions.{CurrentBatchTimestamp, Uuid}
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, 
ClusteredDistribution, HashPartitioning, SinglePartition}
 import org.apache.spark.sql.catalyst.rules.Rule
@@ -73,10 +75,14 @@ class IncrementalExecution(
* with the desired literal
*/
   override lazy val optimizedPlan: LogicalPlan = {
+val random = new Random()
+
 sparkSession.sessionState.optimizer.execute(withCachedData) 
transformAllExpressions {
   case ts @ CurrentBatchTimestamp(timestamp, _, _) =>
 logInfo(s"Current batch timestamp = $timestamp")
 ts.toLiteral
+  // SPARK-24896: Set the seed for random number generation in Uuid 
expressions.
+  case _: Uuid => Uuid(Some(random.nextLong()))
 }
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/d0bc3ed6/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
index 78199b0..f37f368 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
@@ -21,6 +21,8 @@ import java.{util => ju}
 import java.util.Optional
 import java.util.concurrent.CountDownLatch
 
+import scala.collection.mutable
+
 import org.apache.commons.lang3.RandomStringUtils
 import org.scalactic.TolerantNumerics
 import org.scalatest.BeforeAndAfter
@@ -29,8 +31,9 @@ import org.scalatest.mockito.MockitoSugar
 
 import org.apache.spark.SparkException
 import org.apache.spark.internal.Logging
-import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.{Column, DataFrame, Dataset, Row}
 import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Uuid
 import org.apache.spark.sql.execution.streaming._
 import org.apache.spark.sql.execution.streaming.sources.TestForeachWriter
 import org.apache.spark.sql.functions._
@@ -834,6 +837,23 @@ class StreamingQuerySuite extends StreamTest with 
BeforeAndAfter with Logging wi
   CheckLastBatch(("A", 1)))
   }
 
+  test("Uuid in streaming query should not produce same uuids in each 
execution") {
+val uuids = mutable.ArrayBuffer[String]()
+def collectUuid: Seq[Row] => Unit = { rows: Seq[Row] =>
+  rows.foreach(r 

spark git commit: [SPARK-18057][SS] Update Kafka client version from 0.10.0.1 to 2.0.0

2018-07-31 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/master 1223a201f -> e82784d13


[SPARK-18057][SS] Update Kafka client version from 0.10.0.1 to 2.0.0

## What changes were proposed in this pull request?

This PR upgrades to the Kafka 2.0.0 release where KIP-266 is integrated.

## How was this patch tested?

This PR uses existing Kafka related unit tests

(Please explain how this patch was tested. E.g. unit tests, integration tests, 
manual tests)
(If this patch involves UI changes, please attach a screenshot; otherwise, 
remove this)

Please review http://spark.apache.org/contributing.html before opening a pull 
request.

Author: tedyu 

Closes #21488 from tedyu/master.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/e82784d1
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/e82784d1
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/e82784d1

Branch: refs/heads/master
Commit: e82784d13fac7d45164dfadb00d3fa43e64e0bde
Parents: 1223a20
Author: tedyu 
Authored: Tue Jul 31 13:14:14 2018 -0700
Committer: zsxwing 
Committed: Tue Jul 31 13:14:14 2018 -0700

--
 external/kafka-0-10-sql/pom.xml | 24 +++--
 .../kafka010/KafkaContinuousSourceSuite.scala   |  1 +
 .../kafka010/KafkaMicroBatchSourceSuite.scala   |  7 +++-
 .../spark/sql/kafka010/KafkaTestUtils.scala | 36 +---
 4 files changed, 53 insertions(+), 15 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/e82784d1/external/kafka-0-10-sql/pom.xml
--
diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml
index 16bbc6d..9550003 100644
--- a/external/kafka-0-10-sql/pom.xml
+++ b/external/kafka-0-10-sql/pom.xml
@@ -29,10 +29,10 @@
   spark-sql-kafka-0-10_2.11
   
 sql-kafka-0-10
-0.10.0.1
+2.0.0
   
   jar
-  Kafka 0.10 Source for Structured Streaming
+  Kafka 0.10+ Source for Structured Streaming
   http://spark.apache.org/
 
   
@@ -73,6 +73,20 @@
   kafka_${scala.binary.version}
   ${kafka.version}
   test
+  
+
+  com.fasterxml.jackson.core
+  jackson-core
+
+
+  com.fasterxml.jackson.core
+  jackson-databind
+
+
+  com.fasterxml.jackson.core
+  jackson-annotations
+
+  
 
 
   net.sf.jopt-simple
@@ -80,6 +94,12 @@
   3.2
   test
 
+ 
+org.eclipse.jetty
+jetty-servlet
+${jetty.version}
+test
+  
 
   org.scalacheck
   scalacheck_${scala.binary.version}

http://git-wip-us.apache.org/repos/asf/spark/blob/e82784d1/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala
--
diff --git 
a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala
 
b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala
index aab8ec4..ea2a2a8 100644
--- 
a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala
+++ 
b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala
@@ -42,6 +42,7 @@ class KafkaContinuousSourceTopicDeletionSuite extends 
KafkaContinuousTest {
   .format("kafka")
   .option("kafka.bootstrap.servers", testUtils.brokerAddress)
   .option("kafka.metadata.max.age.ms", "1")
+  .option("kafka.default.api.timeout.ms", "3000")
   .option("subscribePattern", s"$topicPrefix-.*")
   .option("failOnDataLoss", "false")
 

http://git-wip-us.apache.org/repos/asf/spark/blob/e82784d1/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
--
diff --git 
a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
 
b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
index 5d5e573..aa89868 100644
--- 
a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
+++ 
b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
@@ -290,6 +290,7 @@ abstract class KafkaMicroBatchSourceSuiteBase extends 
KafkaSourceSuiteBase {
   .format("kafka")
   .option("kafka.bootstrap.servers", testUtils.brokerAddress)
   .option("kafka.metadata.max.age.ms", "1")
+ 

spark git commit: [SPARK-24566][CORE] Fix spark.storage.blockManagerSlaveTimeoutMs default config

2018-06-29 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/master f6e6899a8 -> f71e8da5e


[SPARK-24566][CORE] Fix spark.storage.blockManagerSlaveTimeoutMs default config

This PR use spark.network.timeout in place of 
spark.storage.blockManagerSlaveTimeoutMs when it is not configured, as 
configuration doc said

manual test

Author: xueyu <278006...@qq.com>

Closes #21575 from xueyumusic/slaveTimeOutConfig.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/f71e8da5
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/f71e8da5
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/f71e8da5

Branch: refs/heads/master
Commit: f71e8da5efde96aacc89e59c6e27b71fffcbc25f
Parents: f6e6899
Author: xueyu <278006...@qq.com>
Authored: Fri Jun 29 10:44:17 2018 -0700
Committer: Shixiong Zhu 
Committed: Fri Jun 29 10:44:49 2018 -0700

--
 core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala| 5 ++---
 .../cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala  | 2 +-
 2 files changed, 3 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/f71e8da5/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala
--
diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala 
b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala
index ff960b3..bcbc8df 100644
--- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala
+++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala
@@ -74,10 +74,9 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, 
clock: Clock)
 
   // "spark.network.timeout" uses "seconds", while 
`spark.storage.blockManagerSlaveTimeoutMs` uses
   // "milliseconds"
-  private val slaveTimeoutMs =
-sc.conf.getTimeAsMs("spark.storage.blockManagerSlaveTimeoutMs", "120s")
   private val executorTimeoutMs =
-sc.conf.getTimeAsSeconds("spark.network.timeout", s"${slaveTimeoutMs}ms") 
* 1000
+sc.conf.getTimeAsMs("spark.storage.blockManagerSlaveTimeoutMs",
+  s"${sc.conf.getTimeAsSeconds("spark.network.timeout", "120s")}s")
 
   // "spark.network.timeoutInterval" uses "seconds", while
   // "spark.storage.blockManagerTimeoutIntervalMs" uses "milliseconds"

http://git-wip-us.apache.org/repos/asf/spark/blob/f71e8da5/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala
--
diff --git 
a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala
 
b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala
index d35bea4..1ce2f81 100644
--- 
a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala
+++ 
b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala
@@ -634,7 +634,7 @@ private[spark] class MesosCoarseGrainedSchedulerBackend(
 slave.hostname,
 externalShufflePort,
 sc.conf.getTimeAsMs("spark.storage.blockManagerSlaveTimeoutMs",
-  s"${sc.conf.getTimeAsSeconds("spark.network.timeout", "120s") * 
1000L}ms"),
+  s"${sc.conf.getTimeAsSeconds("spark.network.timeout", 
"120s")}s"),
 sc.conf.getTimeAsMs("spark.executor.heartbeatInterval", "10s"))
 slave.shuffleRegistered = true
   }


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



spark git commit: [SPARK-24578][CORE] Cap sub-region's size of returned nio buffer

2018-06-20 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/branch-2.3 d687d97b1 -> 8928de3cd


[SPARK-24578][CORE] Cap sub-region's size of returned nio buffer

## What changes were proposed in this pull request?
This PR tries to fix the performance regression introduced by SPARK-21517.

In our production job, we performed many parallel computations, with high 
possibility, some task could be scheduled to a host-2 where it needs to read 
the cache block data from host-1. Often, this big transfer makes the cluster 
suffer time out issue (it will retry 3 times, each with 120s timeout, and then 
do recompute to put the cache block into the local MemoryStore).

The root cause is that we don't do `consolidateIfNeeded` anymore as we are using
```
Unpooled.wrappedBuffer(chunks.length, getChunks(): _*)
```
in ChunkedByteBuffer. If we have many small chunks, it could cause the 
`buf.notBuffer(...)` have very bad performance in the case that we have to call 
`copyByteBuf(...)` many times.

## How was this patch tested?
Existing unit tests and also test in production

Author: Wenbo Zhao 

Closes #21593 from WenboZhao/spark-24578.

(cherry picked from commit 3f4bda7289f1bfbbe8b9bc4b516007f569c44d2e)
Signed-off-by: Shixiong Zhu 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/8928de3c
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/8928de3c
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/8928de3c

Branch: refs/heads/branch-2.3
Commit: 8928de3cd448922d43f9ae80cea7138ecbea0d24
Parents: d687d97
Author: Wenbo Zhao 
Authored: Wed Jun 20 14:26:04 2018 -0700
Committer: Shixiong Zhu 
Committed: Wed Jun 20 14:26:32 2018 -0700

--
 .../network/protocol/MessageWithHeader.java | 25 
 1 file changed, 5 insertions(+), 20 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/8928de3c/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageWithHeader.java
--
diff --git 
a/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageWithHeader.java
 
b/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageWithHeader.java
index a533765..e7b66a6 100644
--- 
a/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageWithHeader.java
+++ 
b/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageWithHeader.java
@@ -137,30 +137,15 @@ class MessageWithHeader extends AbstractFileRegion {
   }
 
   private int copyByteBuf(ByteBuf buf, WritableByteChannel target) throws 
IOException {
-ByteBuffer buffer = buf.nioBuffer();
-int written = (buffer.remaining() <= NIO_BUFFER_LIMIT) ?
-  target.write(buffer) : writeNioBuffer(target, buffer);
+// SPARK-24578: cap the sub-region's size of returned nio buffer to 
improve the performance
+// for the case that the passed-in buffer has too many components.
+int length = Math.min(buf.readableBytes(), NIO_BUFFER_LIMIT);
+ByteBuffer buffer = buf.nioBuffer(buf.readerIndex(), length);
+int written = target.write(buffer);
 buf.skipBytes(written);
 return written;
   }
 
-  private int writeNioBuffer(
-  WritableByteChannel writeCh,
-  ByteBuffer buf) throws IOException {
-int originalLimit = buf.limit();
-int ret = 0;
-
-try {
-  int ioSize = Math.min(buf.remaining(), NIO_BUFFER_LIMIT);
-  buf.limit(buf.position() + ioSize);
-  ret = writeCh.write(buf);
-} finally {
-  buf.limit(originalLimit);
-}
-
-return ret;
-  }
-
   @Override
   public MessageWithHeader touch(Object o) {
 super.touch(o);


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



spark git commit: [SPARK-24578][CORE] Cap sub-region's size of returned nio buffer

2018-06-20 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/master c5a0d1132 -> 3f4bda728


[SPARK-24578][CORE] Cap sub-region's size of returned nio buffer

## What changes were proposed in this pull request?
This PR tries to fix the performance regression introduced by SPARK-21517.

In our production job, we performed many parallel computations, with high 
possibility, some task could be scheduled to a host-2 where it needs to read 
the cache block data from host-1. Often, this big transfer makes the cluster 
suffer time out issue (it will retry 3 times, each with 120s timeout, and then 
do recompute to put the cache block into the local MemoryStore).

The root cause is that we don't do `consolidateIfNeeded` anymore as we are using
```
Unpooled.wrappedBuffer(chunks.length, getChunks(): _*)
```
in ChunkedByteBuffer. If we have many small chunks, it could cause the 
`buf.notBuffer(...)` have very bad performance in the case that we have to call 
`copyByteBuf(...)` many times.

## How was this patch tested?
Existing unit tests and also test in production

Author: Wenbo Zhao 

Closes #21593 from WenboZhao/spark-24578.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/3f4bda72
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/3f4bda72
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/3f4bda72

Branch: refs/heads/master
Commit: 3f4bda7289f1bfbbe8b9bc4b516007f569c44d2e
Parents: c5a0d11
Author: Wenbo Zhao 
Authored: Wed Jun 20 14:26:04 2018 -0700
Committer: Shixiong Zhu 
Committed: Wed Jun 20 14:26:04 2018 -0700

--
 .../network/protocol/MessageWithHeader.java | 25 
 1 file changed, 5 insertions(+), 20 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/3f4bda72/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageWithHeader.java
--
diff --git 
a/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageWithHeader.java
 
b/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageWithHeader.java
index a533765..e7b66a6 100644
--- 
a/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageWithHeader.java
+++ 
b/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageWithHeader.java
@@ -137,30 +137,15 @@ class MessageWithHeader extends AbstractFileRegion {
   }
 
   private int copyByteBuf(ByteBuf buf, WritableByteChannel target) throws 
IOException {
-ByteBuffer buffer = buf.nioBuffer();
-int written = (buffer.remaining() <= NIO_BUFFER_LIMIT) ?
-  target.write(buffer) : writeNioBuffer(target, buffer);
+// SPARK-24578: cap the sub-region's size of returned nio buffer to 
improve the performance
+// for the case that the passed-in buffer has too many components.
+int length = Math.min(buf.readableBytes(), NIO_BUFFER_LIMIT);
+ByteBuffer buffer = buf.nioBuffer(buf.readerIndex(), length);
+int written = target.write(buffer);
 buf.skipBytes(written);
 return written;
   }
 
-  private int writeNioBuffer(
-  WritableByteChannel writeCh,
-  ByteBuffer buf) throws IOException {
-int originalLimit = buf.limit();
-int ret = 0;
-
-try {
-  int ioSize = Math.min(buf.remaining(), NIO_BUFFER_LIMIT);
-  buf.limit(buf.position() + ioSize);
-  ret = writeCh.write(buf);
-} finally {
-  buf.limit(originalLimit);
-}
-
-return ret;
-  }
-
   @Override
   public MessageWithHeader touch(Object o) {
 super.touch(o);


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



spark git commit: [SPARK-24565][SS] Add API for in Structured Streaming for exposing output rows of each microbatch as a DataFrame

2018-06-19 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/master 13092d733 -> 2cb976355


[SPARK-24565][SS] Add API for in Structured Streaming for exposing output rows 
of each microbatch as a DataFrame

## What changes were proposed in this pull request?

Currently, the micro-batches in the MicroBatchExecution is not exposed to the 
user through any public API. This was because we did not want to expose the 
micro-batches, so that all the APIs we expose, we can eventually support them 
in the Continuous engine. But now that we have better sense of buiding a 
ContinuousExecution, I am considering adding APIs which will run only the 
MicroBatchExecution. I have quite a few use cases where exposing the microbatch 
output as a dataframe is useful.
- Pass the output rows of each batch to a library that is designed only the 
batch jobs (example, uses many ML libraries need to collect() while learning).
- Reuse batch data sources for output whose streaming version does not exists 
(e.g. redshift data source).
- Writer the output rows to multiple places by writing twice for each batch. 
This is not the most elegant thing to do for multiple-output streaming queries 
but is likely to be better than running two streaming queries processing the 
same data twice.

The proposal is to add a method `foreachBatch(f: Dataset[T] => Unit)` to 
Scala/Java/Python `DataStreamWriter`.

## How was this patch tested?
New unit tests.

Author: Tathagata Das 

Closes #21571 from tdas/foreachBatch.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/2cb97635
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/2cb97635
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/2cb97635

Branch: refs/heads/master
Commit: 2cb976355c615eee4ebd0a86f3911fa9284fccf6
Parents: 13092d7
Author: Tathagata Das 
Authored: Tue Jun 19 13:56:51 2018 -0700
Committer: Shixiong Zhu 
Committed: Tue Jun 19 13:56:51 2018 -0700

--
 python/pyspark/java_gateway.py  |  25 +++-
 python/pyspark/sql/streaming.py |  33 -
 python/pyspark/sql/tests.py |  36 +
 python/pyspark/sql/utils.py |  23 +++
 python/pyspark/streaming/context.py |  18 +--
 .../streaming/sources/ForeachBatchSink.scala|  58 
 .../spark/sql/streaming/DataStreamWriter.scala  |  63 +++-
 .../sources/ForeachBatchSinkSuite.scala | 148 +++
 8 files changed, 383 insertions(+), 21 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/2cb97635/python/pyspark/java_gateway.py
--
diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py
index 0afbe9d..fa2d5e8 100644
--- a/python/pyspark/java_gateway.py
+++ b/python/pyspark/java_gateway.py
@@ -31,7 +31,7 @@ from subprocess import Popen, PIPE
 if sys.version >= '3':
 xrange = range
 
-from py4j.java_gateway import java_import, JavaGateway, GatewayParameters
+from py4j.java_gateway import java_import, JavaGateway, JavaObject, 
GatewayParameters
 from pyspark.find_spark_home import _find_spark_home
 from pyspark.serializers import read_int, write_with_length, UTF8Deserializer
 
@@ -145,3 +145,26 @@ def do_server_auth(conn, auth_secret):
 if reply != "ok":
 conn.close()
 raise Exception("Unexpected reply from iterator server.")
+
+
+def ensure_callback_server_started(gw):
+"""
+Start callback server if not already started. The callback server is 
needed if the Java
+driver process needs to callback into the Python driver process to execute 
Python code.
+"""
+
+# getattr will fallback to JVM, so we cannot test by hasattr()
+if "_callback_server" not in gw.__dict__ or gw._callback_server is None:
+gw.callback_server_parameters.eager_load = True
+gw.callback_server_parameters.daemonize = True
+gw.callback_server_parameters.daemonize_connections = True
+gw.callback_server_parameters.port = 0
+gw.start_callback_server(gw.callback_server_parameters)
+cbport = gw._callback_server.server_socket.getsockname()[1]
+gw._callback_server.port = cbport
+# gateway with real port
+gw._python_proxy_port = gw._callback_server.port
+# get the GatewayServer object in JVM by ID
+jgws = JavaObject("GATEWAY_SERVER", gw._gateway_client)
+# update the port of CallbackClient with real port
+jgws.resetCallbackClient(jgws.getCallbackClient().getAddress(), 
gw._python_proxy_port)

http://git-wip-us.apache.org/repos/asf/spark/blob/2cb97635/python/pyspark/sql/streaming.py
--
diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py

spark git commit: [SPARK-24235][SS] Implement continuous shuffle writer for single reader partition.

2018-06-13 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/master 299d297e2 -> 1b46f41c5


[SPARK-24235][SS] Implement continuous shuffle writer for single reader 
partition.

## What changes were proposed in this pull request?

https://docs.google.com/document/d/1IL4kJoKrZWeyIhklKUJqsW-yEN7V7aL05MmM65AYOfE/edit

Implement continuous shuffle write RDD for a single reader partition. (I don't 
believe any implementation changes are actually required for multiple reader 
partitions, but this PR is already very large, so I want to exclude those for 
now to keep the size down.)

## How was this patch tested?

new unit tests

Author: Jose Torres 

Closes #21428 from jose-torres/writerTask.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/1b46f41c
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/1b46f41c
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/1b46f41c

Branch: refs/heads/master
Commit: 1b46f41c55f5cd29956e17d7da95a95580cf273f
Parents: 299d297
Author: Jose Torres 
Authored: Wed Jun 13 13:13:01 2018 -0700
Committer: Shixiong Zhu 
Committed: Wed Jun 13 13:13:01 2018 -0700

--
 .../shuffle/ContinuousShuffleReadRDD.scala  |   6 +-
 .../shuffle/ContinuousShuffleWriter.scala   |  27 ++
 .../shuffle/RPCContinuousShuffleReader.scala| 138 ++
 .../shuffle/RPCContinuousShuffleWriter.scala|  60 +++
 .../continuous/shuffle/UnsafeRowReceiver.scala  | 138 --
 .../shuffle/ContinuousShuffleReadSuite.scala| 291 -
 .../shuffle/ContinuousShuffleSuite.scala| 416 +++
 7 files changed, 645 insertions(+), 431 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/1b46f41c/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/shuffle/ContinuousShuffleReadRDD.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/shuffle/ContinuousShuffleReadRDD.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/shuffle/ContinuousShuffleReadRDD.scala
index 801b28b..cf6572d 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/shuffle/ContinuousShuffleReadRDD.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/shuffle/ContinuousShuffleReadRDD.scala
@@ -34,8 +34,10 @@ case class ContinuousShuffleReadPartition(
   // Initialized only on the executor, and only once even as we call compute() 
multiple times.
   lazy val (reader: ContinuousShuffleReader, endpoint) = {
 val env = SparkEnv.get.rpcEnv
-val receiver = new UnsafeRowReceiver(queueSize, numShuffleWriters, 
epochIntervalMs, env)
-val endpoint = 
env.setupEndpoint(s"UnsafeRowReceiver-${UUID.randomUUID()}", receiver)
+val receiver = new RPCContinuousShuffleReader(
+  queueSize, numShuffleWriters, epochIntervalMs, env)
+val endpoint = 
env.setupEndpoint(s"RPCContinuousShuffleReader-${UUID.randomUUID()}", receiver)
+
 TaskContext.get().addTaskCompletionListener { ctx =>
   env.stop(endpoint)
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/1b46f41c/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/shuffle/ContinuousShuffleWriter.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/shuffle/ContinuousShuffleWriter.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/shuffle/ContinuousShuffleWriter.scala
new file mode 100644
index 000..47b1f78
--- /dev/null
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/shuffle/ContinuousShuffleWriter.scala
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.streaming.continuous.shuffle
+
+import org.apache.spark.sql.catalyst.expressions.UnsafeRow
+
+/**
+ * Trait for writing to a 

spark git commit: [SPARK-24351][SS] offsetLog/commitLog purge thresholdBatchId should be computed with current committed epoch but not currentBatchId in CP mode

2018-06-01 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/master 98909c398 -> 6039b1323


[SPARK-24351][SS] offsetLog/commitLog purge thresholdBatchId should be computed 
with current committed epoch but not currentBatchId in CP mode

## What changes were proposed in this pull request?
Compute the thresholdBatchId to purge metadata based on current committed epoch 
instead of currentBatchId in CP mode to avoid cleaning all the committed 
metadata in some case as described in the jira 
[SPARK-24351](https://issues.apache.org/jira/browse/SPARK-24351).

## How was this patch tested?
Add new unit test.

Author: Huang Tengfei 

Closes #21400 from ivoson/branch-cp-meta.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/6039b132
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/6039b132
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/6039b132

Branch: refs/heads/master
Commit: 6039b132304cc77ed39e4ca7813850507ae0b440
Parents: 98909c3
Author: Huang Tengfei 
Authored: Fri Jun 1 10:47:53 2018 -0700
Committer: Shixiong Zhu 
Committed: Fri Jun 1 10:47:53 2018 -0700

--
 .../continuous/ContinuousExecution.scala| 11 +++--
 .../streaming/continuous/ContinuousSuite.scala  | 46 
 2 files changed, 54 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/6039b132/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala
index d16b24c..e3d0cea 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala
@@ -318,9 +318,14 @@ class ContinuousExecution(
   }
 }
 
-if (minLogEntriesToMaintain < currentBatchId) {
-  offsetLog.purge(currentBatchId - minLogEntriesToMaintain)
-  commitLog.purge(currentBatchId - minLogEntriesToMaintain)
+// Since currentBatchId increases independently in cp mode, the current 
committed epoch may
+// be far behind currentBatchId. It is not safe to discard the metadata 
with thresholdBatchId
+// computed based on currentBatchId. As minLogEntriesToMaintain is used to 
keep the minimum
+// number of batches that must be retained and made recoverable, so we 
should keep the
+// specified number of metadata that have been committed.
+if (minLogEntriesToMaintain <= epoch) {
+  offsetLog.purge(epoch + 1 - minLogEntriesToMaintain)
+  commitLog.purge(epoch + 1 - minLogEntriesToMaintain)
 }
 
 awaitProgressLock.lock()

http://git-wip-us.apache.org/repos/asf/spark/blob/6039b132/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousSuite.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousSuite.scala
index cd1704a..4980b0c 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousSuite.scala
@@ -297,3 +297,49 @@ class ContinuousStressSuite extends ContinuousSuiteBase {
   CheckAnswerRowsContains(scala.Range(0, 25000).map(Row(_
   }
 }
+
+class ContinuousMetaSuite extends ContinuousSuiteBase {
+  import testImplicits._
+
+  // We need to specify spark.sql.streaming.minBatchesToRetain to do the 
following test.
+  override protected def createSparkSession = new TestSparkSession(
+new SparkContext(
+  "local[10]",
+  "continuous-stream-test-sql-context",
+  sparkConf.set("spark.sql.testkey", "true")
+.set("spark.sql.streaming.minBatchesToRetain", "2")))
+
+  test("SPARK-24351: check offsetLog/commitLog retained in the checkpoint 
directory") {
+withTempDir { checkpointDir =>
+  val input = ContinuousMemoryStream[Int]
+  val df = input.toDF().mapPartitions(iter => {
+// Sleep the task thread for 300 ms to make sure epoch processing time 
3 times
+// longer than epoch creating interval. So the gap between last 
committed
+// epoch and currentBatchId grows over time.
+Thread.sleep(300)
+iter.map(row => row.getInt(0) * 2)
+  })
+
+  testStream(df)(
+StartStream(trigger = Trigger.Continuous(100),
+  checkpointLocation = 

spark git commit: [SPARK-24332][SS][MESOS] Fix places reading 'spark.network.timeout' as milliseconds

2018-05-24 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/master 0d8994344 -> 53c06ddab


[SPARK-24332][SS][MESOS] Fix places reading 'spark.network.timeout' as 
milliseconds

## What changes were proposed in this pull request?

This PR replaces `getTimeAsMs` with `getTimeAsSeconds` to fix the issue that 
reading "spark.network.timeout" using a wrong time unit when the user doesn't 
specify a time out.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <zsxw...@gmail.com>

Closes #21382 from zsxwing/fix-network-timeout-conf.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/53c06dda
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/53c06dda
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/53c06dda

Branch: refs/heads/master
Commit: 53c06ddabbdf689f8823807445849ad63173676f
Parents: 0d89943
Author: Shixiong Zhu <zsxw...@gmail.com>
Authored: Thu May 24 13:00:24 2018 -0700
Committer: Shixiong Zhu <zsxw...@gmail.com>
Committed: Thu May 24 13:00:24 2018 -0700

--
 .../org/apache/spark/sql/kafka010/KafkaMicroBatchReader.scala| 2 +-
 .../main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala | 4 +++-
 .../main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala   | 2 +-
 .../scala/org/apache/spark/streaming/kafka010/KafkaRDD.scala | 2 +-
 .../cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala   | 2 +-
 5 files changed, 7 insertions(+), 5 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/53c06dda/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReader.scala
--
diff --git 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReader.scala
 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReader.scala
index 64ba987..737da2e 100644
--- 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReader.scala
+++ 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReader.scala
@@ -68,7 +68,7 @@ private[kafka010] class KafkaMicroBatchReader(
 
   private val pollTimeoutMs = options.getLong(
 "kafkaConsumer.pollTimeoutMs",
-SparkEnv.get.conf.getTimeAsMs("spark.network.timeout", "120s"))
+SparkEnv.get.conf.getTimeAsSeconds("spark.network.timeout", "120s") * 
1000L)
 
   private val maxOffsetsPerTrigger =
 Option(options.get("maxOffsetsPerTrigger").orElse(null)).map(_.toLong)

http://git-wip-us.apache.org/repos/asf/spark/blob/53c06dda/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala
--
diff --git 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala
 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala
index 7103709..c31e6ed 100644
--- 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala
+++ 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala
@@ -48,7 +48,9 @@ private[kafka010] class KafkaRelation(
 
   private val pollTimeoutMs = sourceOptions.getOrElse(
 "kafkaConsumer.pollTimeoutMs",
-sqlContext.sparkContext.conf.getTimeAsMs("spark.network.timeout", 
"120s").toString
+(sqlContext.sparkContext.conf.getTimeAsSeconds(
+  "spark.network.timeout",
+  "120s") * 1000L).toString
   ).toLong
 
   override def schema: StructType = KafkaOffsetReader.kafkaSchema

http://git-wip-us.apache.org/repos/asf/spark/blob/53c06dda/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
--
diff --git 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
index 1c7b3a2..101e649 100644
--- 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
+++ 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
@@ -84,7 +84,7 @@ private[kafka010] class KafkaSource(
 
   private val pollTimeoutMs = sourceOptions.getOrElse(
 "kafkaConsumer.pollTimeoutMs",
-sc.conf.getTimeAsMs("spark.network.timeout", "120s").toString
+(sc.conf.getTimeAsSeconds("spark.network.timeout", "120s") * 
1000L).toString
   ).toLong
 
   private val maxOffsetsPerTrigger =

http://git-wip-us.apache.org/repos/

spark git commit: [SPARK-24159][SS] Enable no-data micro batches for streaming mapGroupswithState

2018-05-18 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/master 7696b9de0 -> 807ba44cb


[SPARK-24159][SS] Enable no-data micro batches for streaming mapGroupswithState

## What changes were proposed in this pull request?

Enabled no-data batches in flatMapGroupsWithState in following two cases.
- When ProcessingTime timeout is used, then we always run a batch every trigger 
interval.
- When event-time watermark is defined, then the user may be doing arbitrary 
logic against the watermark value even if timeouts are not set. In such cases, 
it's best to run batches whenever the watermark has changed, irrespective of 
whether timeouts (i.e. event-time timeout) have been explicitly enabled.

## How was this patch tested?
updated tests

Author: Tathagata Das 

Closes #21345 from tdas/SPARK-24159.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/807ba44c
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/807ba44c
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/807ba44c

Branch: refs/heads/master
Commit: 807ba44cb742c5f7c22bdf6bfe2cf814be85398e
Parents: 7696b9d
Author: Tathagata Das 
Authored: Fri May 18 10:35:43 2018 -0700
Committer: Shixiong Zhu 
Committed: Fri May 18 10:35:43 2018 -0700

--
 .../streaming/FlatMapGroupsWithStateExec.scala  |  17 ++-
 .../streaming/FlatMapGroupsWithStateSuite.scala | 120 ++-
 2 files changed, 80 insertions(+), 57 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/807ba44c/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FlatMapGroupsWithStateExec.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FlatMapGroupsWithStateExec.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FlatMapGroupsWithStateExec.scala
index 80769d7..8e82ccc 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FlatMapGroupsWithStateExec.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FlatMapGroupsWithStateExec.scala
@@ -97,6 +97,18 @@ case class FlatMapGroupsWithStateExec(
 
   override def keyExpressions: Seq[Attribute] = groupingAttributes
 
+  override def shouldRunAnotherBatch(newMetadata: OffsetSeqMetadata): Boolean 
= {
+timeoutConf match {
+  case ProcessingTimeTimeout =>
+true  // Always run batches to process timeouts
+  case EventTimeTimeout =>
+// Process another non-data batch only if the watermark has changed in 
this executed plan
+eventTimeWatermark.isDefined && newMetadata.batchWatermarkMs > 
eventTimeWatermark.get
+  case _ =>
+false
+}
+  }
+
   override protected def doExecute(): RDD[InternalRow] = {
 metrics // force lazy init at driver
 
@@ -126,7 +138,6 @@ case class FlatMapGroupsWithStateExec(
   case _ =>
 iter
 }
-
 // Generate a iterator that returns the rows grouped by the grouping 
function
 // Note that this code ensures that the filtering for timeout occurs 
only after
 // all the data has been processed. This is to ensure that the timeout 
information of all
@@ -194,11 +205,11 @@ case class FlatMapGroupsWithStateExec(
 throw new IllegalStateException(
   s"Cannot filter timed out keys for $timeoutConf")
 }
-val timingOutKeys = store.getRange(None, None).filter { rowPair =>
+val timingOutPairs = store.getRange(None, None).filter { rowPair =>
   val timeoutTimestamp = getTimeoutTimestamp(rowPair.value)
   timeoutTimestamp != NO_TIMESTAMP && timeoutTimestamp < 
timeoutThreshold
 }
-timingOutKeys.flatMap { rowPair =>
+timingOutPairs.flatMap { rowPair =>
   callFunctionAndUpdateState(rowPair.key, Iterator.empty, 
rowPair.value, hasTimedOut = true)
 }
   } else Iterator.empty

http://git-wip-us.apache.org/repos/asf/spark/blob/807ba44c/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala
index b1416bf..988c8e6 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala
@@ -615,20 +615,20 @@ class FlatMapGroupsWithStateSuite extends 
StateStoreMetricsTest
 
 testStream(result, Update)(
   AddData(inputData, "a"),
-

spark git commit: [SPARK-20538][SQL] Wrap Dataset.reduce with withNewRddExecutionId.

2018-05-18 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/master 0cf59fcbe -> 7696b9de0


[SPARK-20538][SQL] Wrap Dataset.reduce with withNewRddExecutionId.

## What changes were proposed in this pull request?

Wrap Dataset.reduce with `withNewExecutionId`.

Author: Soham Aurangabadkar 

Closes #21316 from sohama4/dataset_reduce_withexecutionid.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/7696b9de
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/7696b9de
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/7696b9de

Branch: refs/heads/master
Commit: 7696b9de0df6e9eb85a74bdb404409da693cf65e
Parents: 0cf59fc
Author: Soham Aurangabadkar 
Authored: Fri May 18 10:29:34 2018 -0700
Committer: Shixiong Zhu 
Committed: Fri May 18 10:29:34 2018 -0700

--
 sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/7696b9de/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
index f001f16..32267eb 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
@@ -1617,7 +1617,9 @@ class Dataset[T] private[sql](
*/
   @Experimental
   @InterfaceStability.Evolving
-  def reduce(func: (T, T) => T): T = rdd.reduce(func)
+  def reduce(func: (T, T) => T): T = withNewRDDExecutionId {
+rdd.reduce(func)
+  }
 
   /**
* :: Experimental ::


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



spark git commit: [SPARK-24040][SS] Support single partition aggregates in continuous processing.

2018-05-15 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/master d610d2a3f -> 3fabbc576


[SPARK-24040][SS] Support single partition aggregates in continuous processing.

## What changes were proposed in this pull request?

Support aggregates with exactly 1 partition in continuous processing.

A few small tweaks are needed to make this work:

* Replace currentEpoch tracking with an ThreadLocal. This means that current 
epoch is scoped to a task rather than a node, but I think that's sustainable 
even once we add shuffle.
* Add a new testing-only flag to disable the UnsupportedOperationChecker 
whitelist of allowed continuous processing nodes. I think this is preferable to 
writing a pile of custom logic to enforce that there is in fact only 1 
partition; we plan to support multi-partition aggregates before the next Spark 
release, so we'd just have to tear that logic back out.
* Restart continuous processing queries from the first available uncommitted 
epoch, rather than one that's guaranteed to be unused. This is required for 
stateful operators to overwrite partial state from the previous attempt at the 
epoch, and there was no specific motivation for the original strategy. In 
another PR before stabilizing the StreamWriter API, we'll need to narrow down 
and document more precise semantic guarantees for the epoch IDs.
* We need a single-partition ContinuousMemoryStream. The way MemoryStream is 
constructed means it can't be a text option like it is for rate source, 
unfortunately.

## How was this patch tested?

new unit tests

Author: Jose Torres 

Closes #21239 from jose-torres/withAggr.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/3fabbc57
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/3fabbc57
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/3fabbc57

Branch: refs/heads/master
Commit: 3fabbc576203c7fd63808a259adafc5c3cea1838
Parents: d610d2a
Author: Jose Torres 
Authored: Tue May 15 10:25:29 2018 -0700
Committer: Shixiong Zhu 
Committed: Tue May 15 10:25:29 2018 -0700

--
 .../analysis/UnsupportedOperationChecker.scala  |  1 +
 .../continuous/ContinuousExecution.scala| 11 +--
 .../continuous/ContinuousQueuedDataReader.scala |  7 +-
 .../continuous/ContinuousWriteRDD.scala | 18 +++--
 .../streaming/continuous/EpochTracker.scala | 58 
 .../sources/ContinuousMemoryStream.scala| 14 ++--
 .../streaming/state/StateStoreRDD.scala | 10 ++-
 .../sql/streaming/StreamingQueryManager.scala   |  4 +-
 .../continuous/ContinuousAggregationSuite.scala | 72 
 .../ContinuousQueuedDataReaderSuite.scala   |  1 +
 10 files changed, 167 insertions(+), 29 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/3fabbc57/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala
index d3d6c63..2bed416 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala
@@ -24,6 +24,7 @@ import 
org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys
 import org.apache.spark.sql.catalyst.plans._
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.streaming.InternalOutputModes
+import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.streaming.OutputMode
 
 /**

http://git-wip-us.apache.org/repos/asf/spark/blob/3fabbc57/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala
index f58146a..0e7d101 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala
@@ -122,16 +122,7 @@ class ContinuousExecution(
 s"Batch $latestEpochId was committed without end epoch offsets!")
 }
 committedOffsets = nextOffsets.toStreamProgress(sources)
-
-// Get to an epoch ID that has 

spark git commit: [SPARK-24214][SS] Fix toJSON for StreamingRelationV2/StreamingExecutionRelation/ContinuousExecutionRelation

2018-05-09 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/branch-2.3 aba52f449 -> 8889d7864


[SPARK-24214][SS] Fix toJSON for 
StreamingRelationV2/StreamingExecutionRelation/ContinuousExecutionRelation

## What changes were proposed in this pull request?

We should overwrite "otherCopyArgs" to provide the SparkSession parameter 
otherwise TreeNode.toJSON cannot get the full constructor parameter list.

## How was this patch tested?

The new unit test.

Author: Shixiong Zhu <zsxw...@gmail.com>

Closes #21275 from zsxwing/SPARK-24214.

(cherry picked from commit fd1179c17273283d32f275d5cd5f97aaa2aca1f7)
Signed-off-by: Shixiong Zhu <zsxw...@gmail.com>


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/8889d786
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/8889d786
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/8889d786

Branch: refs/heads/branch-2.3
Commit: 8889d78643154a0eb5ce81363ba471a80a1e64f1
Parents: aba52f4
Author: Shixiong Zhu <zsxw...@gmail.com>
Authored: Wed May 9 11:32:17 2018 -0700
Committer: Shixiong Zhu <zsxw...@gmail.com>
Committed: Wed May 9 11:32:27 2018 -0700

--
 .../sql/execution/streaming/StreamingRelation.scala  |  3 +++
 .../spark/sql/streaming/StreamingQuerySuite.scala| 15 +++
 2 files changed, 18 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/8889d786/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala
index f02d3a2..24195b5 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala
@@ -66,6 +66,7 @@ case class StreamingExecutionRelation(
 output: Seq[Attribute])(session: SparkSession)
   extends LeafNode with MultiInstanceRelation {
 
+  override def otherCopyArgs: Seq[AnyRef] = session :: Nil
   override def isStreaming: Boolean = true
   override def toString: String = source.toString
 
@@ -97,6 +98,7 @@ case class StreamingRelationV2(
 output: Seq[Attribute],
 v1Relation: Option[StreamingRelation])(session: SparkSession)
   extends LeafNode with MultiInstanceRelation {
+  override def otherCopyArgs: Seq[AnyRef] = session :: Nil
   override def isStreaming: Boolean = true
   override def toString: String = sourceName
 
@@ -116,6 +118,7 @@ case class ContinuousExecutionRelation(
 output: Seq[Attribute])(session: SparkSession)
   extends LeafNode with MultiInstanceRelation {
 
+  override def otherCopyArgs: Seq[AnyRef] = session :: Nil
   override def isStreaming: Boolean = true
   override def toString: String = source.toString
 

http://git-wip-us.apache.org/repos/asf/spark/blob/8889d786/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
index 2b0ab33..e3429b5 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
@@ -687,6 +687,21 @@ class StreamingQuerySuite extends StreamTest with 
BeforeAndAfter with Logging wi
   CheckLastBatch(("A", 1)))
   }
 
+  
test("StreamingRelationV2/StreamingExecutionRelation/ContinuousExecutionRelation.toJSON
 " +
+"should not fail") {
+val df = spark.readStream.format("rate").load()
+assert(df.logicalPlan.toJSON.contains("StreamingRelationV2"))
+
+testStream(df)(
+  
AssertOnQuery(_.logicalPlan.toJSON.contains("StreamingExecutionRelation"))
+)
+
+testStream(df, useV2Sink = true)(
+  StartStream(trigger = Trigger.Continuous(100)),
+  
AssertOnQuery(_.logicalPlan.toJSON.contains("ContinuousExecutionRelation"))
+)
+  }
+
   /** Create a streaming DF that only execute one batch in which it returns 
the given static DF */
   private def createSingleTriggerStreamingDF(triggerDF: DataFrame): DataFrame 
= {
 require(!triggerDF.isStreaming)


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



spark git commit: [SPARK-24214][SS] Fix toJSON for StreamingRelationV2/StreamingExecutionRelation/ContinuousExecutionRelation

2018-05-09 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/master 7aaa148f5 -> fd1179c17


[SPARK-24214][SS] Fix toJSON for 
StreamingRelationV2/StreamingExecutionRelation/ContinuousExecutionRelation

## What changes were proposed in this pull request?

We should overwrite "otherCopyArgs" to provide the SparkSession parameter 
otherwise TreeNode.toJSON cannot get the full constructor parameter list.

## How was this patch tested?

The new unit test.

Author: Shixiong Zhu <zsxw...@gmail.com>

Closes #21275 from zsxwing/SPARK-24214.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/fd1179c1
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/fd1179c1
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/fd1179c1

Branch: refs/heads/master
Commit: fd1179c17273283d32f275d5cd5f97aaa2aca1f7
Parents: 7aaa148
Author: Shixiong Zhu <zsxw...@gmail.com>
Authored: Wed May 9 11:32:17 2018 -0700
Committer: Shixiong Zhu <zsxw...@gmail.com>
Committed: Wed May 9 11:32:17 2018 -0700

--
 .../sql/execution/streaming/StreamingRelation.scala  |  3 +++
 .../spark/sql/streaming/StreamingQuerySuite.scala| 15 +++
 2 files changed, 18 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/fd1179c1/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala
index f02d3a2..24195b5 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala
@@ -66,6 +66,7 @@ case class StreamingExecutionRelation(
 output: Seq[Attribute])(session: SparkSession)
   extends LeafNode with MultiInstanceRelation {
 
+  override def otherCopyArgs: Seq[AnyRef] = session :: Nil
   override def isStreaming: Boolean = true
   override def toString: String = source.toString
 
@@ -97,6 +98,7 @@ case class StreamingRelationV2(
 output: Seq[Attribute],
 v1Relation: Option[StreamingRelation])(session: SparkSession)
   extends LeafNode with MultiInstanceRelation {
+  override def otherCopyArgs: Seq[AnyRef] = session :: Nil
   override def isStreaming: Boolean = true
   override def toString: String = sourceName
 
@@ -116,6 +118,7 @@ case class ContinuousExecutionRelation(
 output: Seq[Attribute])(session: SparkSession)
   extends LeafNode with MultiInstanceRelation {
 
+  override def otherCopyArgs: Seq[AnyRef] = session :: Nil
   override def isStreaming: Boolean = true
   override def toString: String = source.toString
 

http://git-wip-us.apache.org/repos/asf/spark/blob/fd1179c1/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
index 0cb2375..5798699 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
@@ -831,6 +831,21 @@ class StreamingQuerySuite extends StreamTest with 
BeforeAndAfter with Logging wi
   CheckLastBatch(("A", 1)))
   }
 
+  
test("StreamingRelationV2/StreamingExecutionRelation/ContinuousExecutionRelation.toJSON
 " +
+"should not fail") {
+val df = spark.readStream.format("rate").load()
+assert(df.logicalPlan.toJSON.contains("StreamingRelationV2"))
+
+testStream(df)(
+  
AssertOnQuery(_.logicalPlan.toJSON.contains("StreamingExecutionRelation"))
+)
+
+testStream(df, useV2Sink = true)(
+  StartStream(trigger = Trigger.Continuous(100)),
+  
AssertOnQuery(_.logicalPlan.toJSON.contains("ContinuousExecutionRelation"))
+)
+  }
+
   /** Create a streaming DF that only execute one batch in which it returns 
the given static DF */
   private def createSingleTriggerStreamingDF(triggerDF: DataFrame): DataFrame 
= {
 require(!triggerDF.isStreaming)


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



spark git commit: [SPARK-24061][SS] Add TypedFilter support for continuous processing

2018-05-01 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/master b857fb549 -> 7bbec0dce


[SPARK-24061][SS] Add TypedFilter support for continuous processing

## What changes were proposed in this pull request?

Add TypedFilter support for continuous processing application.

## How was this patch tested?

unit tests

Author: wangyanlin01 

Closes #21136 from yanlin-Lynn/SPARK-24061.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/7bbec0dc
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/7bbec0dc
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/7bbec0dc

Branch: refs/heads/master
Commit: 7bbec0dced35aeed79c1a24b6f7a1e0a3508b0fb
Parents: b857fb5
Author: wangyanlin01 
Authored: Tue May 1 16:22:52 2018 +0800
Committer: Shixiong Zhu 
Committed: Tue May 1 16:22:52 2018 +0800

--
 .../analysis/UnsupportedOperationChecker.scala  |  3 ++-
 .../analysis/UnsupportedOperationsSuite.scala   | 23 
 2 files changed, 25 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/7bbec0dc/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala
index ff9d6d7..d3d6c63 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala
@@ -345,7 +345,8 @@ object UnsupportedOperationChecker {
 plan.foreachUp { implicit subPlan =>
   subPlan match {
 case (_: Project | _: Filter | _: MapElements | _: MapPartitions |
-  _: DeserializeToObject | _: SerializeFromObject | _: 
SubqueryAlias) =>
+  _: DeserializeToObject | _: SerializeFromObject | _: 
SubqueryAlias |
+  _: TypedFilter) =>
 case node if node.nodeName == "StreamingRelationV2" =>
 case node =>
   throwError(s"Continuous processing does not support ${node.nodeName} 
operations.")

http://git-wip-us.apache.org/repos/asf/spark/blob/7bbec0dc/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala
--
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala
index 60d1351..cb487c8 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala
@@ -621,6 +621,13 @@ class UnsupportedOperationsSuite extends SparkFunSuite {
 outputMode = Append,
 expectedMsgs = Seq("monotonically_increasing_id"))
 
+  assertSupportedForContinuousProcessing(
+"TypedFilter", TypedFilter(
+  null,
+  null,
+  null,
+  null,
+  new TestStreamingRelationV2(attribute)), OutputMode.Append())
 
   /*
 
===
@@ -771,6 +778,16 @@ class UnsupportedOperationsSuite extends SparkFunSuite {
 }
   }
 
+  /** Assert that the logical plan is supported for continuous procsssing mode 
*/
+  def assertSupportedForContinuousProcessing(
+name: String,
+plan: LogicalPlan,
+outputMode: OutputMode): Unit = {
+test(s"continuous processing - $name: supported") {
+  UnsupportedOperationChecker.checkForContinuous(plan, outputMode)
+}
+  }
+
   /**
* Assert that the logical plan is not supported inside a streaming plan.
*
@@ -840,4 +857,10 @@ class UnsupportedOperationsSuite extends SparkFunSuite {
 def this(attribute: Attribute) = this(Seq(attribute))
 override def isStreaming: Boolean = true
   }
+
+  case class TestStreamingRelationV2(output: Seq[Attribute]) extends LeafNode {
+def this(attribute: Attribute) = this(Seq(attribute))
+override def isStreaming: Boolean = true
+override def nodeName: String = "StreamingRelationV2"
+  }
 }


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



spark git commit: [SPARK-23565][SS] New error message for structured streaming sources assertion

2018-04-27 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/master 109935fc5 -> 2824f12b8


[SPARK-23565][SS] New error message for structured streaming sources assertion

## What changes were proposed in this pull request?

A more informative message to tell you why a structured streaming query cannot 
continue if you have added more sources, than there are in the existing 
checkpoint offsets.

## How was this patch tested?

I added a Unit Test.

Author: Patrick McGloin 

Closes #20946 from patrickmcgloin/master.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/2824f12b
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/2824f12b
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/2824f12b

Branch: refs/heads/master
Commit: 2824f12b8bac5d86a82339d4dfb4d2625e978a15
Parents: 109935f
Author: Patrick McGloin 
Authored: Fri Apr 27 23:04:14 2018 +0800
Committer: Shixiong Zhu 
Committed: Fri Apr 27 23:04:14 2018 +0800

--
 .../org/apache/spark/sql/execution/streaming/OffsetSeq.scala | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/2824f12b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeq.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeq.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeq.scala
index 73945b3..7871744 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeq.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeq.scala
@@ -39,7 +39,9 @@ case class OffsetSeq(offsets: Seq[Option[Offset]], metadata: 
Option[OffsetSeqMet
* cannot be serialized).
*/
   def toStreamProgress(sources: Seq[BaseStreamingSource]): StreamProgress = {
-assert(sources.size == offsets.size)
+assert(sources.size == offsets.size, s"There are [${offsets.size}] sources 
in the " +
+  s"checkpoint offsets and now there are [${sources.size}] sources 
requested by the query. " +
+  s"Cannot continue.")
 new StreamProgress ++ sources.zip(offsets).collect { case (s, Some(o)) => 
(s, o) }
   }
 


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



spark git commit: [SPARK-23788][SS] Fix race in StreamingQuerySuite

2018-03-24 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/branch-2.2 85ab72b59 -> 6b5f9c374


[SPARK-23788][SS] Fix race in StreamingQuerySuite

## What changes were proposed in this pull request?

The serializability test uses the same MemoryStream instance for 3 different 
queries. If any of those queries ask it to commit before the others have run, 
the rest will see empty dataframes. This can fail the test if q3 is affected.

We should use one instance per query instead.

## How was this patch tested?

Existing unit test. If I move q2.processAllAvailable() before starting q3, the 
test always fails without the fix.

Author: Jose Torres 

Closes #20896 from jose-torres/fixrace.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/6b5f9c37
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/6b5f9c37
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/6b5f9c37

Branch: refs/heads/branch-2.2
Commit: 6b5f9c3745a1005519261fc80825a99377906451
Parents: 85ab72b
Author: Jose Torres 
Authored: Sat Mar 24 18:21:01 2018 -0700
Committer: Shixiong Zhu 
Committed: Sat Mar 24 18:22:15 2018 -0700

--
 .../apache/spark/sql/streaming/StreamingQuerySuite.scala  | 10 +-
 1 file changed, 5 insertions(+), 5 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/6b5f9c37/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
index 01c34b1..9e65aa8 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
@@ -533,22 +533,22 @@ class StreamingQuerySuite extends StreamTest with 
BeforeAndAfter with Logging wi
 .start()
 }
 
-val input = MemoryStream[Int]
-val q1 = startQuery(input.toDS, "stream_serializable_test_1")
-val q2 = startQuery(input.toDS.map { i =>
+val input = MemoryStream[Int] :: MemoryStream[Int] :: MemoryStream[Int] :: 
Nil
+val q1 = startQuery(input(0).toDS, "stream_serializable_test_1")
+val q2 = startQuery(input(1).toDS.map { i =>
   // Emulate that `StreamingQuery` get captured with normal usage 
unintentionally.
   // It should not fail the query.
   q1
   i
 }, "stream_serializable_test_2")
-val q3 = startQuery(input.toDS.map { i =>
+val q3 = startQuery(input(2).toDS.map { i =>
   // Emulate that `StreamingQuery` is used in executors. We should fail 
the query with a clear
   // error message.
   q1.explain()
   i
 }, "stream_serializable_test_3")
 try {
-  input.addData(1)
+  input.foreach(_.addData(1))
 
   // q2 should not fail since it doesn't use `q1` in the closure
   q2.processAllAvailable()


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



spark git commit: [SPARK-23788][SS] Fix race in StreamingQuerySuite

2018-03-24 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/branch-2.3 ea44783ad -> 523fcafc5


[SPARK-23788][SS] Fix race in StreamingQuerySuite

## What changes were proposed in this pull request?

The serializability test uses the same MemoryStream instance for 3 different 
queries. If any of those queries ask it to commit before the others have run, 
the rest will see empty dataframes. This can fail the test if q3 is affected.

We should use one instance per query instead.

## How was this patch tested?

Existing unit test. If I move q2.processAllAvailable() before starting q3, the 
test always fails without the fix.

Author: Jose Torres 

Closes #20896 from jose-torres/fixrace.

(cherry picked from commit 816a5496ba4caac438f70400f72bb10bfcc02418)
Signed-off-by: Shixiong Zhu 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/523fcafc
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/523fcafc
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/523fcafc

Branch: refs/heads/branch-2.3
Commit: 523fcafc5c4a79cf3455f3ceab6d886679399495
Parents: ea44783
Author: Jose Torres 
Authored: Sat Mar 24 18:21:01 2018 -0700
Committer: Shixiong Zhu 
Committed: Sat Mar 24 18:21:14 2018 -0700

--
 .../apache/spark/sql/streaming/StreamingQuerySuite.scala  | 10 +-
 1 file changed, 5 insertions(+), 5 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/523fcafc/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
index 76201c6..2b0ab33 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
@@ -532,22 +532,22 @@ class StreamingQuerySuite extends StreamTest with 
BeforeAndAfter with Logging wi
 .start()
 }
 
-val input = MemoryStream[Int]
-val q1 = startQuery(input.toDS, "stream_serializable_test_1")
-val q2 = startQuery(input.toDS.map { i =>
+val input = MemoryStream[Int] :: MemoryStream[Int] :: MemoryStream[Int] :: 
Nil
+val q1 = startQuery(input(0).toDS, "stream_serializable_test_1")
+val q2 = startQuery(input(1).toDS.map { i =>
   // Emulate that `StreamingQuery` get captured with normal usage 
unintentionally.
   // It should not fail the query.
   q1
   i
 }, "stream_serializable_test_2")
-val q3 = startQuery(input.toDS.map { i =>
+val q3 = startQuery(input(2).toDS.map { i =>
   // Emulate that `StreamingQuery` is used in executors. We should fail 
the query with a clear
   // error message.
   q1.explain()
   i
 }, "stream_serializable_test_3")
 try {
-  input.addData(1)
+  input.foreach(_.addData(1))
 
   // q2 should not fail since it doesn't use `q1` in the closure
   q2.processAllAvailable()


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



spark git commit: [SPARK-23788][SS] Fix race in StreamingQuerySuite

2018-03-24 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/master a33655348 -> 816a5496b


[SPARK-23788][SS] Fix race in StreamingQuerySuite

## What changes were proposed in this pull request?

The serializability test uses the same MemoryStream instance for 3 different 
queries. If any of those queries ask it to commit before the others have run, 
the rest will see empty dataframes. This can fail the test if q3 is affected.

We should use one instance per query instead.

## How was this patch tested?

Existing unit test. If I move q2.processAllAvailable() before starting q3, the 
test always fails without the fix.

Author: Jose Torres 

Closes #20896 from jose-torres/fixrace.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/816a5496
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/816a5496
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/816a5496

Branch: refs/heads/master
Commit: 816a5496ba4caac438f70400f72bb10bfcc02418
Parents: a336553
Author: Jose Torres 
Authored: Sat Mar 24 18:21:01 2018 -0700
Committer: Shixiong Zhu 
Committed: Sat Mar 24 18:21:01 2018 -0700

--
 .../apache/spark/sql/streaming/StreamingQuerySuite.scala  | 10 +-
 1 file changed, 5 insertions(+), 5 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/816a5496/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
index ebc9a87..08749b4 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
@@ -550,22 +550,22 @@ class StreamingQuerySuite extends StreamTest with 
BeforeAndAfter with Logging wi
 .start()
 }
 
-val input = MemoryStream[Int]
-val q1 = startQuery(input.toDS, "stream_serializable_test_1")
-val q2 = startQuery(input.toDS.map { i =>
+val input = MemoryStream[Int] :: MemoryStream[Int] :: MemoryStream[Int] :: 
Nil
+val q1 = startQuery(input(0).toDS, "stream_serializable_test_1")
+val q2 = startQuery(input(1).toDS.map { i =>
   // Emulate that `StreamingQuery` get captured with normal usage 
unintentionally.
   // It should not fail the query.
   q1
   i
 }, "stream_serializable_test_2")
-val q3 = startQuery(input.toDS.map { i =>
+val q3 = startQuery(input(2).toDS.map { i =>
   // Emulate that `StreamingQuery` is used in executors. We should fail 
the query with a clear
   // error message.
   q1.explain()
   i
 }, "stream_serializable_test_3")
 try {
-  input.addData(1)
+  input.foreach(_.addData(1))
 
   // q2 should not fail since it doesn't use `q1` in the closure
   q2.processAllAvailable()


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



spark git commit: [SPARK-23623][SS] Avoid concurrent use of cached consumers in CachedKafkaConsumer (branch-2.3)

2018-03-17 Thread zsxwing
Repository: spark
Updated Branches:
  refs/heads/branch-2.3 21b6de459 -> 6937571ab


[SPARK-23623][SS] Avoid concurrent use of cached consumers in 
CachedKafkaConsumer (branch-2.3)

This is a backport of #20767 to branch 2.3

## What changes were proposed in this pull request?
CacheKafkaConsumer in the project `kafka-0-10-sql` is designed to maintain a 
pool of KafkaConsumers that can be reused. However, it was built with the 
assumption there will be only one task using trying to read the same Kafka 
TopicPartition at the same time. Hence, the cache was keyed by the 
TopicPartition a consumer is supposed to read. And any cases where this 
assumption may not be true, we have SparkPlan flag to disable the use of a 
cache. So it was up to the planner to correctly identify when it was not safe 
to use the cache and set the flag accordingly.

Fundamentally, this is the wrong way to approach the problem. It is HARD for a 
high-level planner to reason about the low-level execution model, whether there 
will be multiple tasks in the same query trying to read the same partition. 
Case in point, 2.3.0 introduced stream-stream joins, and you can build a 
streaming self-join query on Kafka. It's pretty non-trivial to figure out how 
this leads to two tasks reading the same partition twice, possibly 
concurrently. And due to the non-triviality, it is hard to figure this out in 
the planner and set the flag to avoid the cache / consumer pool. And this can 
inadvertently lead to ConcurrentModificationException ,or worse, silent reading 
of incorrect data.

Here is a better way to design this. The planner shouldnt have to understand 
these low-level optimizations. Rather the consumer pool should be smart enough 
avoid concurrent use of a cached consumer. Currently, it tries to do so but 
incorrectly (the flag inuse is not checked when returning a cached consumer, 
see 
[this](https://github.com/apache/spark/blob/master/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala#L403)).
 If there is another request for the same partition as a currently in-use 
consumer, the pool should automatically return a fresh consumer that should be 
closed when the task is done. Then the planner does not have to have a flag to 
avoid reuses.

This PR is a step towards that goal. It does the following.
- There are effectively two kinds of consumer that may be generated
  - Cached consumer - this should be returned to the pool at task end
  - Non-cached consumer - this should be closed at task end
- A trait called KafkaConsumer is introduced to hide this difference from the 
users of the consumer so that the client code does not have to reason about 
whether to stop and release. They simply called `val consumer = 
KafkaConsumer.acquire` and then `consumer.release()`.
- If there is request for a consumer that is in-use, then a new consumer is 
generated.
- If there is a concurrent attempt of the same task, then a new consumer is 
generated, and the existing cached consumer is marked for close upon release.
- In addition, I renamed the classes because CachedKafkaConsumer is a misnomer 
given that what it returns may or may not be cached.

This PR does not remove the planner flag to avoid reuse to make this patch safe 
enough for merging in branch-2.3. This can be done later in master-only.

## How was this patch tested?
A new stress test that verifies it is safe to concurrently get consumers for 
the same partition from the consumer pool.

Author: Tathagata Das 

Closes #20848 from tdas/SPARK-23623-2.3.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/6937571a
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/6937571a
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/6937571a

Branch: refs/heads/branch-2.3
Commit: 6937571ab8818a62ec2457a373eb3f6f618985e1
Parents: 21b6de4
Author: Tathagata Das 
Authored: Sat Mar 17 16:24:51 2018 -0700
Committer: Shixiong Zhu 
Committed: Sat Mar 17 16:24:51 2018 -0700

--
 .../sql/kafka010/CachedKafkaConsumer.scala  | 438 
 .../sql/kafka010/KafkaContinuousReader.scala|   4 +-
 .../spark/sql/kafka010/KafkaDataConsumer.scala  | 516 +++
 .../spark/sql/kafka010/KafkaSourceRDD.scala |  23 +-
 .../sql/kafka010/CachedKafkaConsumerSuite.scala |  34 --
 .../sql/kafka010/KafkaDataConsumerSuite.scala   | 124 +
 6 files changed, 648 insertions(+), 491 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/6937571a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala
--
diff --git 

  1   2   3   4   5   6   7   8   >