[spark] branch branch-3.2 updated: [SPARK-36132][SS][SQL] Support initial state for batch mode of flatMapGroupsWithState

2021-07-20 Thread tdas
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new 0d60cb5  [SPARK-36132][SS][SQL] Support initial state for batch mode 
of flatMapGroupsWithState
0d60cb5 is described below

commit 0d60cb51c01c13b0febe2ff7601db7303bfff56d
Author: Rahul Mahadev 
AuthorDate: Wed Jul 21 01:48:58 2021 -0400

[SPARK-36132][SS][SQL] Support initial state for batch mode of 
flatMapGroupsWithState

### What changes were proposed in this pull request?
Adding support for accepting an initial state with flatMapGroupsWithState 
in batch mode.

### Why are the changes needed?
SPARK-35897  added support for accepting an initial state for streaming 
queries using flatMapGroupsWithState. the code flow is separate for batch and 
streaming and required a different PR.

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

Yes as discussed above flatMapGroupsWithState in batch mode can accept an 
initialState, previously this would throw an UnsupportedOperationException

### How was this patch tested?

Added relevant unit tests in FlatMapGroupsWithStateSuite and modified the  
tests `JavaDatasetSuite`

Closes #6 from rahulsmahadev/flatMapGroupsWithStateBatch.

Authored-by: Rahul Mahadev 
Signed-off-by: Tathagata Das 
(cherry picked from commit efcce23b913ce0de961ac261050e3d6dbf261f6e)
Signed-off-by: Tathagata Das 
---
 .../analysis/UnsupportedOperationChecker.scala |  6 --
 .../spark/sql/execution/SparkStrategies.scala  | 11 +++-
 .../streaming/FlatMapGroupsWithStateExec.scala | 71 +-
 .../org/apache/spark/sql/JavaDatasetSuite.java | 18 +-
 .../streaming/FlatMapGroupsWithStateSuite.scala| 52 
 5 files changed, 130 insertions(+), 28 deletions(-)

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 13c7f75..321725d 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
@@ -37,12 +37,6 @@ object UnsupportedOperationChecker extends Logging {
   case p if p.isStreaming =>
 throwError("Queries with streaming sources must be executed with 
writeStream.start()")(p)
 
-  case f: FlatMapGroupsWithState =>
-if (f.hasInitialState) {
-  throwError("Initial state is not supported in 
[flatMap|map]GroupsWithState" +
-" operation on a batch DataFrame/Dataset")(f)
-}
-
   case _ =>
 }
   }
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 6d10fa8..7624b15 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
@@ -690,9 +690,14 @@ abstract class SparkStrategies extends 
QueryPlanner[SparkPlan] {
   case logical.MapGroups(f, key, value, grouping, data, objAttr, child) =>
 execution.MapGroupsExec(f, key, value, grouping, data, objAttr, 
planLater(child)) :: Nil
   case logical.FlatMapGroupsWithState(
-  f, key, value, grouping, data, output, _, _, _, timeout, _, _, _, _, 
_, child) =>
-execution.MapGroupsExec(
-  f, key, value, grouping, data, output, timeout, planLater(child)) :: 
Nil
+  f, keyDeserializer, valueDeserializer, grouping, data, output, 
stateEncoder, outputMode,
+  isFlatMapGroupsWithState, timeout, hasInitialState, 
initialStateGroupAttrs,
+  initialStateDataAttrs, initialStateDeserializer, initialState, 
child) =>
+FlatMapGroupsWithStateExec.generateSparkPlanForBatchQueries(
+  f, keyDeserializer, valueDeserializer, initialStateDeserializer, 
grouping,
+  initialStateGroupAttrs, data, initialStateDataAttrs, output, timeout,
+  hasInitialState, planLater(initialState), planLater(child)
+) :: Nil
   case logical.CoGroup(f, key, lObj, rObj, lGroup, rGroup, lAttr, rAttr, 
oAttr, left, right) =>
 execution.CoGroupExec(
   f, key, lObj, rObj, lGroup, rGroup, lAttr, rAttr, oAttr,
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 03694d4..a00a622 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/F

[spark] branch master updated: [SPARK-36132][SS][SQL] Support initial state for batch mode of flatMapGroupsWithState

2021-07-20 Thread tdas
This is an automated email from the ASF dual-hosted git repository.

tdas 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 efcce23  [SPARK-36132][SS][SQL] Support initial state for batch mode 
of flatMapGroupsWithState
efcce23 is described below

commit efcce23b913ce0de961ac261050e3d6dbf261f6e
Author: Rahul Mahadev 
AuthorDate: Wed Jul 21 01:48:58 2021 -0400

[SPARK-36132][SS][SQL] Support initial state for batch mode of 
flatMapGroupsWithState

### What changes were proposed in this pull request?
Adding support for accepting an initial state with flatMapGroupsWithState 
in batch mode.

### Why are the changes needed?
SPARK-35897  added support for accepting an initial state for streaming 
queries using flatMapGroupsWithState. the code flow is separate for batch and 
streaming and required a different PR.

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

Yes as discussed above flatMapGroupsWithState in batch mode can accept an 
initialState, previously this would throw an UnsupportedOperationException

### How was this patch tested?

Added relevant unit tests in FlatMapGroupsWithStateSuite and modified the  
tests `JavaDatasetSuite`

Closes #6 from rahulsmahadev/flatMapGroupsWithStateBatch.

Authored-by: Rahul Mahadev 
Signed-off-by: Tathagata Das 
---
 .../analysis/UnsupportedOperationChecker.scala |  6 --
 .../spark/sql/execution/SparkStrategies.scala  | 11 +++-
 .../streaming/FlatMapGroupsWithStateExec.scala | 71 +-
 .../org/apache/spark/sql/JavaDatasetSuite.java | 18 +-
 .../streaming/FlatMapGroupsWithStateSuite.scala| 52 
 5 files changed, 130 insertions(+), 28 deletions(-)

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 13c7f75..321725d 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
@@ -37,12 +37,6 @@ object UnsupportedOperationChecker extends Logging {
   case p if p.isStreaming =>
 throwError("Queries with streaming sources must be executed with 
writeStream.start()")(p)
 
-  case f: FlatMapGroupsWithState =>
-if (f.hasInitialState) {
-  throwError("Initial state is not supported in 
[flatMap|map]GroupsWithState" +
-" operation on a batch DataFrame/Dataset")(f)
-}
-
   case _ =>
 }
   }
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 6d10fa8..7624b15 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
@@ -690,9 +690,14 @@ abstract class SparkStrategies extends 
QueryPlanner[SparkPlan] {
   case logical.MapGroups(f, key, value, grouping, data, objAttr, child) =>
 execution.MapGroupsExec(f, key, value, grouping, data, objAttr, 
planLater(child)) :: Nil
   case logical.FlatMapGroupsWithState(
-  f, key, value, grouping, data, output, _, _, _, timeout, _, _, _, _, 
_, child) =>
-execution.MapGroupsExec(
-  f, key, value, grouping, data, output, timeout, planLater(child)) :: 
Nil
+  f, keyDeserializer, valueDeserializer, grouping, data, output, 
stateEncoder, outputMode,
+  isFlatMapGroupsWithState, timeout, hasInitialState, 
initialStateGroupAttrs,
+  initialStateDataAttrs, initialStateDeserializer, initialState, 
child) =>
+FlatMapGroupsWithStateExec.generateSparkPlanForBatchQueries(
+  f, keyDeserializer, valueDeserializer, initialStateDeserializer, 
grouping,
+  initialStateGroupAttrs, data, initialStateDataAttrs, output, timeout,
+  hasInitialState, planLater(initialState), planLater(child)
+) :: Nil
   case logical.CoGroup(f, key, lObj, rObj, lGroup, rGroup, lAttr, rAttr, 
oAttr, left, right) =>
 execution.CoGroupExec(
   f, key, lObj, rObj, lGroup, rGroup, lAttr, rAttr, oAttr,
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 03694d4..a00a622 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/FlatMa

[spark] branch master updated: [SPARK-35800][SS] Improving GroupState testability by introducing TestGroupState

2021-06-22 Thread tdas
This is an automated email from the ASF dual-hosted git repository.

tdas 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 dfd7b02  [SPARK-35800][SS] Improving GroupState testability by 
introducing TestGroupState
dfd7b02 is described below

commit dfd7b026dc7c3c38bef9afab82852aff902a25d2
Author: Li Zhang 
AuthorDate: Tue Jun 22 15:04:01 2021 -0400

[SPARK-35800][SS] Improving GroupState testability by introducing 
TestGroupState

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

1. Introducing the `TestGroupState` interface which is inherited from 
`GroupState` so that testing related getters can be exposed in a controlled 
manner
2. Changing `GroupStateImpl` to inherit from `TestGroupState` interface, 
instead of directly from `GroupState`
3. Implementing `TestGroupState` object with `create()` method to forward 
inputs to the private `GroupStateImpl` constructor
4. User input validations have been added into `GroupStateImpl`'s 
`createForStreaming()` method to prevent users from creating invalid GroupState 
objects.
5. Replacing existing `GroupStateImpl` usages in sql pkg internal unit 
tests with the newly added `TestGroupState` to give user best practice about 
`TestGroupState` usage.

With the changes in this PR, the class hierarchy is changed from 
`GroupStateImpl` -> `GroupState` to `GroupStateImpl` -> `TestGroupState` -> 
`GroupState` (-> means inherits from)

### Why are the changes needed?
The internal `GroupStateImpl` implementation for the `GroupState` interface 
has no public constructors accessible outside of the sql pkg. However, the 
user-provided state transition function for `[map|flatMap]GroupsWithState` 
requires a `GroupState` object as the prevState input.

Currently, users are calling the Structured Streaming engine in their unit 
tests in order to instantiate such `GroupState` instances, which makes UTs 
cumbersome.

The proposed `TestGroupState` interface is to give users controlled access 
to the `GroupStateImpl` internal implementation to largely improve testability 
of Structured Streaming state transition functions.

**Usage Example**
```
import org.apache.spark.sql.streaming.TestGroupState

test(“Structured Streaming state update function”) {
  var prevState = TestGroupState.create[UserStatus](
optionalState = Optional.empty[UserStatus],
timeoutConf = EventTimeTimeout,
batchProcessingTimeMs = 1L,
eventTimeWatermarkMs = Optional.of(1L),
hasTimedOut = false)

  val userId: String = ...
  val actions: Iterator[UserAction] = ...

  assert(!prevState.hasUpdated)

  updateState(userId, actions, prevState)

  assert(prevState.hasUpdated)
}
```

### Does this PR introduce _any_ user-facing change?
Yes, the `TestGroupState` interface and its corresponding `create()` 
factory function in its companion object are introduced in this pull request 
for users to use in unit tests.

### How was this patch tested?
- New unit tests are added
- Existing GroupState unit tests are updated

Closes #32938 from lizhangdatabricks/improve-group-state-testability.

Authored-by: Li Zhang 
Signed-off-by: Tathagata Das 
---
 .../streaming/FlatMapGroupsWithStateExec.scala |   8 +-
 .../sql/execution/streaming/GroupStateImpl.scala   |  36 ++-
 .../state/FlatMapGroupsWithStateExecHelper.scala   |   5 +-
 .../spark/sql/streaming/TestGroupState.scala   | 173 ++
 .../org/apache/spark/sql/JavaDatasetSuite.java |  92 
 .../streaming/FlatMapGroupsWithStateSuite.scala| 255 +++--
 6 files changed, 475 insertions(+), 94 deletions(-)

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 e626fc1..981586e 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
@@ -60,8 +60,8 @@ case class FlatMapGroupsWithStateExec(
 child: SparkPlan
   ) extends UnaryExecNode with ObjectProducerExec with StateStoreWriter with 
WatermarkSupport {
 
-  import GroupStateImpl._
   import FlatMapGroupsWithStateExecHelper._
+  import GroupStateImpl._
 
   private val isTimeoutEnabled = timeoutConf != NoTimeout
   private val watermarkPresent = child.output.exists {
@@ -229,13 +229,13 @@ case class FlatMapGroupsWithStateExec(
 
   // When the iterator is consumed, then write changes to state
   def onIteratorComp

[spark] branch master updated: [SPARK-32585][SQL] Support scala enumeration in ScalaReflection

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

tdas 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 e62d247  [SPARK-32585][SQL] Support scala enumeration in 
ScalaReflection
e62d247 is described below

commit e62d24717eb774f1c7adfd0fbe39640b96bc661d
Author: ulysses 
AuthorDate: Thu Oct 1 15:58:01 2020 -0400

[SPARK-32585][SQL] Support scala enumeration in ScalaReflection

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

Add code in `ScalaReflection` to support scala enumeration and make 
enumeration type as string type in Spark.

### Why are the changes needed?

We support java enum but failed with scala enum, it's better to keep the 
same behavior.

Here is a example.

```
package test

object TestEnum extends Enumeration {
  type TestEnum = Value
  val E1, E2, E3 = Value
}
import TestEnum._
case class TestClass(i: Int,  e: TestEnum) {
}

import test._
Seq(TestClass(1, TestEnum.E1)).toDS
```

Before this PR
```
Exception in thread "main" java.lang.UnsupportedOperationException: No 
Encoder found for test.TestEnum.TestEnum
- field (class: "scala.Enumeration.Value", name: "e")
- root class: "test.TestClass"
  at 
org.apache.spark.sql.catalyst.ScalaReflection$.$anonfun$serializerFor$1(ScalaReflection.scala:567)
  at 
scala.reflect.internal.tpe.TypeConstraints$UndoLog.undo(TypeConstraints.scala:69)
  at 
org.apache.spark.sql.catalyst.ScalaReflection.cleanUpReflectionObjects(ScalaReflection.scala:882)
  at 
org.apache.spark.sql.catalyst.ScalaReflection.cleanUpReflectionObjects$(ScalaReflection.scala:881)
```

After this PR
`org.apache.spark.sql.Dataset[test.TestClass] = [i: int, e: string]`

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

Yes, user can make case class which include scala enumeration field as 
dataset.

### How was this patch tested?

Add test.

Closes #29403 from ulysses-you/SPARK-32585.

Authored-by: ulysses 
Signed-off-by: Tathagata Das 
---
 .../spark/sql/catalyst/ScalaReflection.scala   | 28 ++
 .../spark/sql/catalyst/ScalaReflectionSuite.scala  | 15 
 .../catalyst/encoders/ExpressionEncoderSuite.scala | 10 +++-
 .../scala/org/apache/spark/sql/DatasetSuite.scala  | 15 +++-
 4 files changed, 66 insertions(+), 2 deletions(-)

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 a9c8b0b..c65e181 100644
--- 
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
@@ -30,6 +30,7 @@ import org.apache.spark.sql.catalyst.expressions.objects._
 import org.apache.spark.sql.catalyst.util.{ArrayData, MapData}
 import org.apache.spark.sql.types._
 import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
+import org.apache.spark.util.Utils
 
 
 /**
@@ -377,6 +378,23 @@ object ScalaReflection extends ScalaReflection {
   expressions.Literal.create(null, ObjectType(cls)),
   newInstance
 )
+
+  case t if isSubtype(t, localTypeOf[Enumeration#Value]) =>
+// package example
+// object Foo extends Enumeration {
+//  type Foo = Value
+//  val E1, E2 = Value
+// }
+// the fullName of tpe is example.Foo.Foo, but we need example.Foo so 
that
+// we can call example.Foo.withName to deserialize string to 
enumeration.
+val parent = t.asInstanceOf[TypeRef].pre.typeSymbol.asClass
+val cls = mirror.runtimeClass(parent)
+StaticInvoke(
+  cls,
+  ObjectType(getClassFromType(t)),
+  "withName",
+  createDeserializerForString(path, false) :: Nil,
+  returnNullable = false)
 }
   }
 
@@ -561,6 +579,14 @@ object ScalaReflection extends ScalaReflection {
 }
 createSerializerForObject(inputObject, fields)
 
+  case t if isSubtype(t, localTypeOf[Enumeration#Value]) =>
+createSerializerForString(
+  Invoke(
+inputObject,
+"toString",
+ObjectType(classOf[java.lang.String]),
+returnNullable = false))
+
   case _ =>
 throw new UnsupportedOperationException(
   s"No Encoder found for $tpe\n" + walkedTypePath)
@@ -738,6 +764,8 @@ object ScalaReflection extends ScalaReflection {
 val Schema(dataType, nullable) = schemaFor(fieldType)
 StructField(fieldName, dataType, nullable)
   }), nullable = true)
+  case 

[spark] branch branch-2.4 updated: [SPARK-32794][SS] Fixed rare corner case error in micro-batch engine with some stateful queries + no-data-batches + V1 sources

2020-09-11 Thread tdas
This is an automated email from the ASF dual-hosted git repository.

tdas 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 c82b6e4  [SPARK-32794][SS] Fixed rare corner case error in micro-batch 
engine with some stateful queries + no-data-batches + V1 sources
c82b6e4 is described below

commit c82b6e4bd31bd4c0447a80caffadebff98baa63e
Author: Tathagata Das 
AuthorDate: Fri Sep 11 03:05:39 2020 -0400

[SPARK-32794][SS] Fixed rare corner case error in micro-batch engine with 
some stateful queries + no-data-batches + V1 sources

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

Make MicroBatchExecution explicitly call `getBatch` when the start and end 
offsets are the same.

### Why are the changes needed?

Structured Streaming micro-batch engine has the contract with V1 data 
sources that, after a restart, it will call `source.getBatch()` on the last 
batch attempted before the restart. However, a very rare combination of 
sequences violates this contract. It occurs only when
- The streaming query has specific types of stateful operations with 
watermarks (e.g., aggregation in append, mapGroupsWithState with timeouts).
- These queries can execute a batch even without new data when the 
previous updates the watermark and the stateful ops are such that the new 
watermark can cause new output/cleanup. Such batches are called no-data-batches.
- The last batch before termination was an incomplete no-data-batch. Upon 
restart, the micro-batch engine fails to call `source.getBatch` when attempting 
to re-execute the incomplete no-data-batch.

This occurs because no-data-batches has the same and end offsets, and when 
a batch is executed, if the start and end offset is same then calling 
`source.getBatch` is skipped as it is assumed the generated plan will be empty. 
This only affects V1 data sources which rely on this invariant to detect in the 
source whether the query is being started from scratch or restarted.
### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

New unit test with a mock v1 source that fails without the fix.

Closes #29700 from tdas/SPARK-32794-2.4.

Authored-by: Tathagata Das 
Signed-off-by: Tathagata Das 
---
 .../execution/streaming/MicroBatchExecution.scala  |  11 ++
 .../streaming/MicroBatchExecutionSuite.scala   | 123 -
 .../apache/spark/sql/streaming/StreamTest.scala|   8 ++
 3 files changed, 141 insertions(+), 1 deletion(-)

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 3bcc26b..0834491 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
@@ -291,6 +291,17 @@ class MicroBatchExecution(
   committedOffsets ++= availableOffsets
   watermarkTracker.setWatermark(
 math.max(watermarkTracker.currentWatermark, 
commitMetadata.nextBatchWatermarkMs))
+} else if (latestCommittedBatchId == latestBatchId - 1) {
+  availableOffsets.foreach {
+case (source: Source, end: Offset) =>
+  val start = 
committedOffsets.get(source).map(_.asInstanceOf[Offset])
+  if (start.map(_ == end).getOrElse(true)) {
+source.getBatch(start, end)
+  }
+case nonV1Tuple =>
+  // The V2 API does not have the same edge case requiring 
getBatch to be called
+  // here, so we do nothing here.
+  }
 } else if (latestCommittedBatchId < latestBatchId - 1) {
   logWarning(s"Batch completion log latest batch id is " +
 s"${latestCommittedBatchId}, which is not trailing " +
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecutionSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecutionSuite.scala
index c228740..fc84175 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecutionSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecutionSuite.scala
@@ -19,8 +19,11 @@ package org.apache.spark.sql.execution.streaming
 
 import org.scalatest.BeforeAndAfter
 
+import org.apache.spark.sql.{DataFrame, Dataset, SparkSession}
+import org.apache.spark.sql.catalyst.plans.logical.Range
 import org.apache.spark.sql.functions.{count, window}
-import org.apache.spark.sql.streaming.StreamTest
+import o

[spark] branch branch-3.0 updated: [SPARK-32794][SS] Fixed rare corner case error in micro-batch engine with some stateful queries + no-data-batches + V1 sources

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

tdas 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 e632e7c  [SPARK-32794][SS] Fixed rare corner case error in micro-batch 
engine with some stateful queries + no-data-batches + V1 sources
e632e7c is described below

commit e632e7c87498de6b1f1ecd31a568ac0f4505a6a5
Author: Tathagata Das 
AuthorDate: Wed Sep 9 18:31:50 2020 -0400

[SPARK-32794][SS] Fixed rare corner case error in micro-batch engine with 
some stateful queries + no-data-batches + V1 sources

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

Make MicroBatchExecution explicitly call `getBatch` when the start and end 
offsets are the same.

### Why are the changes needed?

Structured Streaming micro-batch engine has the contract with V1 data 
sources that, after a restart, it will call `source.getBatch()` on the last 
batch attempted before the restart. However, a very rare combination of 
sequences violates this contract. It occurs only when
- The streaming query has specific types of stateful operations with 
watermarks (e.g., aggregation in append, mapGroupsWithState with timeouts).
- These queries can execute a batch even without new data when the 
previous updates the watermark and the stateful ops are such that the new 
watermark can cause new output/cleanup. Such batches are called no-data-batches.
- The last batch before termination was an incomplete no-data-batch. Upon 
restart, the micro-batch engine fails to call `source.getBatch` when attempting 
to re-execute the incomplete no-data-batch.

This occurs because no-data-batches has the same and end offsets, and when 
a batch is executed, if the start and end offset is same then calling 
`source.getBatch` is skipped as it is assumed the generated plan will be empty. 
This only affects V1 data sources which rely on this invariant to detect in the 
source whether the query is being started from scratch or restarted.

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

No

### How was this patch tested?

New unit test with a mock v1 source that fails without the fix.

Closes #29696 from tdas/SPARK-32794-3.0.

Authored-by: Tathagata Das 
Signed-off-by: Tathagata Das 
---
 .../execution/streaming/MicroBatchExecution.scala  |  11 ++
 .../streaming/MicroBatchExecutionSuite.scala   | 123 +
 .../apache/spark/sql/streaming/StreamTest.scala|   8 ++
 3 files changed, 142 insertions(+)

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 e0731db..f85cfbc 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
@@ -318,6 +318,17 @@ class MicroBatchExecution(
   committedOffsets ++= availableOffsets
   watermarkTracker.setWatermark(
 math.max(watermarkTracker.currentWatermark, 
commitMetadata.nextBatchWatermarkMs))
+} else if (latestCommittedBatchId == latestBatchId - 1) {
+  availableOffsets.foreach {
+case (source: Source, end: Offset) =>
+  val start = 
committedOffsets.get(source).map(_.asInstanceOf[Offset])
+  if (start.map(_ == end).getOrElse(true)) {
+source.getBatch(start, end)
+  }
+case nonV1Tuple =>
+  // The V2 API does not have the same edge case requiring 
getBatch to be called
+  // here, so we do nothing here.
+  }
 } else if (latestCommittedBatchId < latestBatchId - 1) {
   logWarning(s"Batch completion log latest batch id is " +
 s"${latestCommittedBatchId}, which is not trailing " +
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecutionSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecutionSuite.scala
index c228740..60b8c96 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecutionSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecutionSuite.scala
@@ -19,8 +19,13 @@ package org.apache.spark.sql.execution.streaming
 
 import org.scalatest.BeforeAndAfter
 
+import org.apache.spark.sql.{DataFrame, Dataset, SparkSession}
+import org.apache.spark.sql.catalyst.plans.logical.Range
+import org.apache.spark.sql.connector.read.streaming
+import org.apache.spark.sql.connector.read.stre

[spark] branch master updated: [SPARK-32794][SS] Fixed rare corner case error in micro-batch engine with some stateful queries + no-data-batches + V1 sources

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

tdas 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 e4237bb  [SPARK-32794][SS] Fixed rare corner case error in micro-batch 
engine with some stateful queries + no-data-batches + V1 sources
e4237bb is described below

commit e4237bbda68c23a3367fab56fd8cdb521f8a1ae2
Author: Tathagata Das 
AuthorDate: Wed Sep 9 13:35:51 2020 -0400

[SPARK-32794][SS] Fixed rare corner case error in micro-batch engine with 
some stateful queries + no-data-batches + V1 sources

### What changes were proposed in this pull request?
Make MicroBatchExecution explicitly call `getBatch` when the start and end 
offsets are the same.

### Why are the changes needed?

Structured Streaming micro-batch engine has the contract with V1 data 
sources that, after a restart, it will call `source.getBatch()` on the last 
batch attempted before the restart. However, a very rare combination of 
sequences violates this contract. It occurs only when
- The streaming query has specific types of stateful operations with 
watermarks (e.g., aggregation in append, mapGroupsWithState with timeouts).
- These queries can execute a batch even without new data when the 
previous updates the watermark and the stateful ops are such that the new 
watermark can cause new output/cleanup. Such batches are called no-data-batches.
- The last batch before termination was an incomplete no-data-batch. Upon 
restart, the micro-batch engine fails to call `source.getBatch` when attempting 
to re-execute the incomplete no-data-batch.

This occurs because no-data-batches has the same and end offsets, and when 
a batch is executed, if the start and end offset is same then calling 
`source.getBatch` is skipped as it is assumed the generated plan will be empty. 
This only affects V1 data sources like Delta and Autoloader which rely on this 
invariant to detect in the source whether the query is being started from 
scratch or restarted.

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

### How was this patch tested?

New unit test with a mock v1 source that fails without the fix.

Closes #29651 from tdas/SPARK-32794.

Authored-by: Tathagata Das 
Signed-off-by: Tathagata Das 
---
 .../execution/streaming/MicroBatchExecution.scala  |  11 ++
 .../streaming/MicroBatchExecutionSuite.scala   | 123 +
 .../apache/spark/sql/streaming/StreamTest.scala|   8 ++
 3 files changed, 142 insertions(+)

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 abbfea8..468a8c9 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
@@ -321,6 +321,17 @@ class MicroBatchExecution(
   committedOffsets ++= availableOffsets
   watermarkTracker.setWatermark(
 math.max(watermarkTracker.currentWatermark, 
commitMetadata.nextBatchWatermarkMs))
+} else if (latestCommittedBatchId == latestBatchId - 1) {
+  availableOffsets.foreach {
+case (source: Source, end: Offset) =>
+  val start = 
committedOffsets.get(source).map(_.asInstanceOf[Offset])
+  if (start.map(_ == end).getOrElse(true)) {
+source.getBatch(start, end)
+  }
+case nonV1Tuple =>
+  // The V2 API does not have the same edge case requiring 
getBatch to be called
+  // here, so we do nothing here.
+  }
 } else if (latestCommittedBatchId < latestBatchId - 1) {
   logWarning(s"Batch completion log latest batch id is " +
 s"${latestCommittedBatchId}, which is not trailing " +
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecutionSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecutionSuite.scala
index c0f25e3..a508f92 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecutionSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecutionSuite.scala
@@ -19,8 +19,13 @@ package org.apache.spark.sql.execution.streaming
 
 import org.scalatest.BeforeAndAfter
 
+import org.apache.spark.sql.{DataFrame, Dataset, SparkSession}
+import org.apache.spark.sql.catalyst.plans.logical.Range
+import org.apache.spark.sql.connector.read.streaming
+import org.apache.spark.sql.connector.read.stre

[spark] branch master updated: [SPARK-29438][SS] Use partition ID of StateStoreAwareZipPartitionsRDD for determining partition ID of state store in stream-stream join

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

tdas 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 cbb714f  [SPARK-29438][SS] Use partition ID of 
StateStoreAwareZipPartitionsRDD for determining partition ID of state store in 
stream-stream join
cbb714f is described below

commit cbb714f67e96dfd9678c60586af48febfba878ca
Author: Jungtaek Lim (HeartSaVioR) 
AuthorDate: Thu Jan 30 20:21:43 2020 -0800

[SPARK-29438][SS] Use partition ID of StateStoreAwareZipPartitionsRDD for 
determining partition ID of state store in stream-stream join

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

Credit to uncleGen for discovering the problem and providing simple 
reproducer as UT. New UT in this patch is borrowed from #26156 and I'm 
retaining a commit from #26156 (except unnecessary part on this path) to 
properly give a credit.

This patch fixes the issue that partition ID could be mis-assigned when the 
query contains UNION and stream-stream join is placed on the right side. We 
assume the range of partition IDs as `(0 ~ number of shuffle partitions - 1)` 
for stateful operators, but when we use stream-stream join on the right side of 
UNION, the range of partition ID of task goes to `(number of partitions in left 
side, number of partitions in left side + number of shuffle partitions - 1)`, 
which `number of part [...]

The root reason of bug is that stream-stream join picks the partition ID 
from TaskContext, which wouldn't be same as partition ID from source if union 
is being used. Hopefully we can pick the right partition ID from source in 
StateStoreAwareZipPartitionsRDD - this patch leverages that partition ID.

### Why are the changes needed?

This patch will fix the broken of assumption of partition range on stateful 
operator, as well as fix the issue reported in JIRA issue SPARK-29438.

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

Yes, if their query is using UNION and stream-stream join is placed on the 
right side. They may encounter the problem to read state from checkpoint and 
may need to discard checkpoint to continue.

### How was this patch tested?

Added UT which fails on current master branch, and passes with this patch.

Closes #26162 from HeartSaVioR/SPARK-29438.

Lead-authored-by: Jungtaek Lim (HeartSaVioR) 
Co-authored-by: uncleGen 
Signed-off-by: Tathagata Das 
---
 .../streaming/StreamingSymmetricHashJoinExec.scala | 11 ++--
 .../StreamingSymmetricHashJoinHelper.scala | 36 +
 .../spark/sql/execution/streaming/memory.scala | 32 ++--
 .../state/SymmetricHashJoinStateManager.scala  |  5 +-
 .../state/SymmetricHashJoinStateManagerSuite.scala |  2 +-
 .../apache/spark/sql/streaming/StreamTest.scala|  6 ++-
 .../spark/sql/streaming/StreamingJoinSuite.scala   | 61 +-
 7 files changed, 129 insertions(+), 24 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala
index 3c45f22..198e17d 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala
@@ -215,6 +215,7 @@ case class StreamingSymmetricHashJoinExec(
   }
 
   private def processPartitions(
+  partitionId: Int,
   leftInputIter: Iterator[InternalRow],
   rightInputIter: Iterator[InternalRow]): Iterator[InternalRow] = {
 if (stateInfo.isEmpty) {
@@ -238,10 +239,10 @@ case class StreamingSymmetricHashJoinExec(
   Predicate.create(condition.bothSides.getOrElse(Literal(true)), 
inputSchema).eval _
 val leftSideJoiner = new OneSideHashJoiner(
   LeftSide, left.output, leftKeys, leftInputIter,
-  condition.leftSideOnly, postJoinFilter, stateWatermarkPredicates.left)
+  condition.leftSideOnly, postJoinFilter, stateWatermarkPredicates.left, 
partitionId)
 val rightSideJoiner = new OneSideHashJoiner(
   RightSide, right.output, rightKeys, rightInputIter,
-  condition.rightSideOnly, postJoinFilter, stateWatermarkPredicates.right)
+  condition.rightSideOnly, postJoinFilter, stateWatermarkPredicates.right, 
partitionId)
 
 //  Join one side input using the other side's buffered/state rows. Here 
is how it is done.
 //
@@ -406,6 +407,7 @@ case class StreamingSymmetricHashJoinExec(
* @param stateWatermarkPredicate The state watermark predicate. See
*[[StreamingSymmetricHashJoinExec]] for 
further description of
*state watermarks.
+   * @param partitionId

[spark] branch master updated: [SPARK-30609] Allow default merge command resolution to be bypassed by DSv2 tables

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

tdas 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 d2bca8f  [SPARK-30609] Allow default merge command resolution to be 
bypassed by DSv2 tables
d2bca8f is described below

commit d2bca8ff70e6c82e915f633bb9f2f8a4582f7026
Author: Tathagata Das 
AuthorDate: Wed Jan 22 19:20:25 2020 -0800

[SPARK-30609] Allow default merge command resolution to be bypassed by DSv2 
tables

### What changes were proposed in this pull request?
Skip resolving the merge expressions if the target is a DSv2 table with 
ACCEPT_ANY_SCHEMA capability.

### Why are the changes needed?
Some DSv2 sources may want to customize the merge resolution logic. For 
example, a table that can accept any schema (TableCapability.ACCEPT_ANY_SCHEMA) 
may want to allow certain merge queries that are blocked (that is, throws 
AnalysisError) by the default resolution logic. So there should be a way to 
completely bypass the merge resolution logic in the Analyzer.

### Does this PR introduce any user-facing change?
No, since merge itself is an unreleased feature

### How was this patch tested?
added unit test to specifically test the skipping.

Closes #27326 from tdas/SPARK-30609.

Authored-by: Tathagata Das 
Signed-off-by: Tathagata Das 
---
 .../spark/sql/catalyst/analysis/Analyzer.scala | 62 +-
 .../execution/command/PlanResolutionSuite.scala| 52 +-
 2 files changed, 86 insertions(+), 28 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index 7e9f85b..503dab1 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -1326,33 +1326,43 @@ class Analyzer(
 
   case m @ MergeIntoTable(targetTable, sourceTable, _, _, _)
 if !m.resolved && targetTable.resolved && sourceTable.resolved =>
-val newMatchedActions = m.matchedActions.map {
-  case DeleteAction(deleteCondition) =>
-val resolvedDeleteCondition = 
deleteCondition.map(resolveExpressionTopDown(_, m))
-DeleteAction(resolvedDeleteCondition)
-  case UpdateAction(updateCondition, assignments) =>
-val resolvedUpdateCondition = 
updateCondition.map(resolveExpressionTopDown(_, m))
-// The update value can access columns from both target and source 
tables.
-UpdateAction(
-  resolvedUpdateCondition,
-  resolveAssignments(assignments, m, resolveValuesWithSourceOnly = 
false))
-  case o => o
-}
-val newNotMatchedActions = m.notMatchedActions.map {
-  case InsertAction(insertCondition, assignments) =>
-// The insert action is used when not matched, so its condition 
and value can only
-// access columns from the source table.
-val resolvedInsertCondition =
-  insertCondition.map(resolveExpressionTopDown(_, Project(Nil, 
m.sourceTable)))
-InsertAction(
-  resolvedInsertCondition,
-  resolveAssignments(assignments, m, resolveValuesWithSourceOnly = 
true))
-  case o => o
+
+EliminateSubqueryAliases(targetTable) match {
+  case r: NamedRelation if r.skipSchemaResolution =>
+// Do not resolve the expression if the target table accepts any 
schema.
+// This allows data sources to customize their own resolution 
logic using
+// custom resolution rules.
+m
+
+  case _ =>
+val newMatchedActions = m.matchedActions.map {
+  case DeleteAction(deleteCondition) =>
+val resolvedDeleteCondition = 
deleteCondition.map(resolveExpressionTopDown(_, m))
+DeleteAction(resolvedDeleteCondition)
+  case UpdateAction(updateCondition, assignments) =>
+val resolvedUpdateCondition = 
updateCondition.map(resolveExpressionTopDown(_, m))
+// The update value can access columns from both target and 
source tables.
+UpdateAction(
+  resolvedUpdateCondition,
+  resolveAssignments(assignments, m, 
resolveValuesWithSourceOnly = false))
+  case o => o
+}
+val newNotMatchedActions = m.notMatchedActions.map {
+  case InsertAction(insertCondition, assignments) =>
+// The insert action is used when not matched, so its 
condition and value can only
+// access columns from the source

[spark] branch branch-2.4 updated: [SPARK-27453] Pass partitionBy as options in DataFrameWriter

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

tdas 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 df9a506  [SPARK-27453] Pass partitionBy as options in DataFrameWriter
df9a506 is described below

commit df9a50637e2622a15e9af7d837986a0e868878b1
Author: liwensun 
AuthorDate: Tue Apr 16 15:03:16 2019 -0700

[SPARK-27453] Pass partitionBy as options in DataFrameWriter

Pass partitionBy columns as options and feature-flag this behavior.

A new unit test.

Closes #24365 from liwensun/partitionby.

Authored-by: liwensun 
Signed-off-by: Tathagata Das 
(cherry picked from commit 26ed65f4150db1fa37f8bfab24ac0873d2e42936)
Signed-off-by: Tathagata Das 
---
 .../org/apache/spark/sql/internal/SQLConf.scala  |  9 +
 .../scala/org/apache/spark/sql/DataFrameWriter.scala | 11 ++-
 .../sql/execution/datasources/DataSourceUtils.scala  | 20 
 .../spark/sql/test/DataFrameReaderWriterSuite.scala  | 19 +++
 4 files changed, 58 insertions(+), 1 deletion(-)

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 29bd356..c9ee60e 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
@@ -1550,6 +1550,15 @@ object SQLConf {
 "WHERE, which does not follow SQL standard.")
   .booleanConf
   .createWithDefault(false)
+
+  val LEGACY_PASS_PARTITION_BY_AS_OPTIONS =
+buildConf("spark.sql.legacy.sources.write.passPartitionByAsOptions")
+  .internal()
+  .doc("Whether to pass the partitionBy columns as options in 
DataFrameWriter. " +
+"Data source V1 now silently drops partitionBy columns for 
non-file-format sources; " +
+"turning the flag on provides a way for these sources to see these 
partitionBy columns.")
+  .booleanConf
+  .createWithDefault(false)
 }
 
 /**
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala
index a2586cc..f90d353 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala
@@ -28,8 +28,9 @@ import org.apache.spark.sql.catalyst.catalog._
 import org.apache.spark.sql.catalyst.plans.logical.{AppendData, 
InsertIntoTable, LogicalPlan}
 import org.apache.spark.sql.execution.SQLExecution
 import org.apache.spark.sql.execution.command.DDLUtils
-import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource, 
LogicalRelation}
+import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource, 
DataSourceUtils, LogicalRelation}
 import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, 
DataSourceV2Utils, WriteToDataSourceV2}
+import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.sources.BaseRelation
 import org.apache.spark.sql.sources.v2._
 import org.apache.spark.sql.types.StructType
@@ -272,6 +273,14 @@ final class DataFrameWriter[T] private[sql](ds: 
Dataset[T]) {
   }
 
   private def saveToV1Source(): Unit = {
+if (SparkSession.active.sessionState.conf.getConf(
+  SQLConf.LEGACY_PASS_PARTITION_BY_AS_OPTIONS)) {
+  partitioningColumns.foreach { columns =>
+extraOptions += (DataSourceUtils.PARTITIONING_COLUMNS_KEY ->
+  DataSourceUtils.encodePartitioningColumns(columns))
+  }
+}
+
 // Code path for data source v1.
 runCommand(df.sparkSession, "save") {
   DataSource(
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceUtils.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceUtils.scala
index 90cec5e..1cb69d7 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceUtils.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceUtils.scala
@@ -18,6 +18,8 @@
 package org.apache.spark.sql.execution.datasources
 
 import org.apache.hadoop.fs.Path
+import org.json4s.NoTypeHints
+import org.json4s.jackson.Serialization
 
 import org.apache.spark.sql.AnalysisException
 import org.apache.spark.sql.types._
@@ -40,6 +42,24 @@ object DataSourceUtils {
   }
 
   /**
+   * The key to use for storing partitionBy columns as options.
+   */
+  val PARTITIONING_COLUMNS_KEY = "__partition_columns"
+
+  /**
+   * Utility methods for converting partitionBy columns to options and back.
+   */
+  private implicit val formats = Serialization.formats(NoTypeHints)
+
+  def encodePartitioningColumns(colum

[spark] branch master updated: [SPARK-27453] Pass partitionBy as options in DataFrameWriter

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

tdas 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 26ed65f  [SPARK-27453] Pass partitionBy as options in DataFrameWriter
26ed65f is described below

commit 26ed65f4150db1fa37f8bfab24ac0873d2e42936
Author: liwensun 
AuthorDate: Tue Apr 16 15:03:16 2019 -0700

[SPARK-27453] Pass partitionBy as options in DataFrameWriter

## What changes were proposed in this pull request?

Pass partitionBy columns as options and feature-flag this behavior.

## How was this patch tested?

A new unit test.

Closes #24365 from liwensun/partitionby.

Authored-by: liwensun 
Signed-off-by: Tathagata Das 
---
 .../org/apache/spark/sql/internal/SQLConf.scala  |  9 +
 .../scala/org/apache/spark/sql/DataFrameWriter.scala | 11 ++-
 .../sql/execution/datasources/DataSourceUtils.scala  | 20 
 .../spark/sql/test/DataFrameReaderWriterSuite.scala  | 19 +++
 4 files changed, 58 insertions(+), 1 deletion(-)

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 3f59fa1..b223a48 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
@@ -1687,6 +1687,15 @@ object SQLConf {
   .booleanConf
   .createWithDefault(false)
 
+  val LEGACY_PASS_PARTITION_BY_AS_OPTIONS =
+buildConf("spark.sql.legacy.sources.write.passPartitionByAsOptions")
+  .internal()
+  .doc("Whether to pass the partitionBy columns as options in 
DataFrameWriter. " +
+"Data source V1 now silently drops partitionBy columns for 
non-file-format sources; " +
+"turning the flag on provides a way for these sources to see these 
partitionBy columns.")
+  .booleanConf
+  .createWithDefault(false)
+
   val NAME_NON_STRUCT_GROUPING_KEY_AS_VALUE =
 buildConf("spark.sql.legacy.dataset.nameNonStructGroupingKeyAsValue")
   .internal()
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala
index 9371936..3b84151 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala
@@ -29,8 +29,9 @@ import org.apache.spark.sql.catalyst.expressions.Literal
 import org.apache.spark.sql.catalyst.plans.logical.{AppendData, 
InsertIntoTable, LogicalPlan, OverwriteByExpression}
 import org.apache.spark.sql.execution.SQLExecution
 import org.apache.spark.sql.execution.command.DDLUtils
-import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource, 
LogicalRelation}
+import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource, 
DataSourceUtils, LogicalRelation}
 import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, 
DataSourceV2Utils, FileDataSourceV2, WriteToDataSourceV2}
+import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.sources.BaseRelation
 import org.apache.spark.sql.sources.v2._
 import org.apache.spark.sql.sources.v2.TableCapability._
@@ -313,6 +314,14 @@ final class DataFrameWriter[T] private[sql](ds: 
Dataset[T]) {
   }
 
   private def saveToV1Source(): Unit = {
+if (SparkSession.active.sessionState.conf.getConf(
+  SQLConf.LEGACY_PASS_PARTITION_BY_AS_OPTIONS)) {
+  partitioningColumns.foreach { columns =>
+extraOptions += (DataSourceUtils.PARTITIONING_COLUMNS_KEY ->
+  DataSourceUtils.encodePartitioningColumns(columns))
+  }
+}
+
 // Code path for data source v1.
 runCommand(df.sparkSession, "save") {
   DataSource(
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceUtils.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceUtils.scala
index 74eae94..0ad914e 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceUtils.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceUtils.scala
@@ -18,6 +18,8 @@
 package org.apache.spark.sql.execution.datasources
 
 import org.apache.hadoop.fs.Path
+import org.json4s.NoTypeHints
+import org.json4s.jackson.Serialization
 
 import org.apache.spark.sql.AnalysisException
 import org.apache.spark.sql.types._
@@ -25,6 +27,24 @@ import org.apache.spark.sql.types._
 
 object DataSourceUtils {
   /**
+   * The key to use for storing partitionBy columns as options.
+   */
+  val PARTITIONING_COLUMNS_KEY = "__partition_columns"
+
+  /**
+   * Utility methods for converting par

spark git commit: [SPARK-25639][DOCS] Added docs for foreachBatch, python foreach and multiple watermarks

2018-10-08 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/branch-2.4 193ce77fc -> 4baa4d42a


[SPARK-25639][DOCS] Added docs for foreachBatch, python foreach and multiple 
watermarks

## What changes were proposed in this pull request?

Added
- Python foreach
- Scala, Java and Python foreachBatch
- Multiple watermark policy
- The semantics of what changes are allowed to the streaming between restarts.

## How was this patch tested?
No tests

Closes #22627 from tdas/SPARK-25639.

Authored-by: Tathagata Das 
Signed-off-by: Tathagata Das 
(cherry picked from commit f9935a3f85f46deef2cb7b213c1c02c8ff627a8c)
Signed-off-by: Tathagata Das 


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

Branch: refs/heads/branch-2.4
Commit: 4baa4d42acf2400537462bddff811a7644bb49a3
Parents: 193ce77
Author: Tathagata Das 
Authored: Mon Oct 8 14:32:04 2018 -0700
Committer: Tathagata Das 
Committed: Mon Oct 8 14:32:18 2018 -0700

--
 docs/structured-streaming-programming-guide.md | 323 +++-
 1 file changed, 312 insertions(+), 11 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/4baa4d42/docs/structured-streaming-programming-guide.md
--
diff --git a/docs/structured-streaming-programming-guide.md 
b/docs/structured-streaming-programming-guide.md
index 73de189..b6e4277 100644
--- a/docs/structured-streaming-programming-guide.md
+++ b/docs/structured-streaming-programming-guide.md
@@ -1560,6 +1560,35 @@ streamingDf <- dropDuplicates(streamingDf, "guid", 
"eventTime")
 
 
 
+### Policy for handling multiple watermarks
+A streaming query can have multiple input streams that are unioned or joined 
together.
+Each of the input streams can have a different threshold of late data that 
needs to
+be tolerated for stateful operations. You specify these thresholds using
+``withWatermarks("eventTime", delay)`` on each of the input streams. For 
example, consider
+a query with stream-stream joins between `inputStream1` and `inputStream2`.
+
+  inputStream1.withWatermark("eventTime1", "1 hour")
+.join(
+  inputStream2.withWatermark("eventTime2", "2 hours"),
+  joinCondition)
+
+While executing the query, Structured Streaming individually tracks the maximum
+event time seen in each input stream, calculates watermarks based on the 
corresponding delay,
+and chooses a single global watermark with them to be used for stateful 
operations. By default,
+the minimum is chosen as the global watermark because it ensures that no data 
is
+accidentally dropped as too late if one of the streams falls behind the others
+(for example, one of the streams stop receiving data due to upstream 
failures). In other words,
+the global watermark will safely move at the pace of the slowest stream and 
the query output will
+be delayed accordingly.
+
+However, in some cases, you may want to get faster results even if it means 
dropping data from the
+slowest stream. Since Spark 2.4, you can set the multiple watermark policy to 
choose
+the maximum value as the global watermark by setting the SQL configuration
+``spark.sql.streaming.multipleWatermarkPolicy`` to ``max`` (default is 
``min``). 
+This lets the global watermark move at the pace of the fastest stream.
+However, as a side effect, data from the slower streams will be aggressively 
dropped. Hence, use
+this configuration judiciously.
+
 ### Arbitrary Stateful Operations
 Many usecases require more advanced stateful operations than aggregations. For 
example, in many usecases, you have to track sessions from data streams of 
events. For doing such sessionization, you will have to save arbitrary types of 
data as state, and perform arbitrary operations on the state using the data 
stream events in every trigger. Since Spark 2.2, this can be done using the 
operation `mapGroupsWithState` and the more powerful operation 
`flatMapGroupsWithState`. Both operations allow you to apply user-defined code 
on grouped Datasets to update user-defined state. For more concrete details, 
take a look at the API documentation 
([Scala](api/scala/index.html#org.apache.spark.sql.streaming.GroupState)/[Java](api/java/org/apache/spark/sql/streaming/GroupState.html))
 and the examples 
([Scala]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredSessionization.scala)/[Java]({{site.SPARK_GITHUB_URL}}/b
 
lob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredSessionization.jav

spark git commit: [SPARK-25639][DOCS] Added docs for foreachBatch, python foreach and multiple watermarks

2018-10-08 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master 6a60fb0aa -> f9935a3f8


[SPARK-25639][DOCS] Added docs for foreachBatch, python foreach and multiple 
watermarks

## What changes were proposed in this pull request?

Added
- Python foreach
- Scala, Java and Python foreachBatch
- Multiple watermark policy
- The semantics of what changes are allowed to the streaming between restarts.

## How was this patch tested?
No tests

Closes #22627 from tdas/SPARK-25639.

Authored-by: Tathagata Das 
Signed-off-by: Tathagata Das 


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

Branch: refs/heads/master
Commit: f9935a3f85f46deef2cb7b213c1c02c8ff627a8c
Parents: 6a60fb0
Author: Tathagata Das 
Authored: Mon Oct 8 14:32:04 2018 -0700
Committer: Tathagata Das 
Committed: Mon Oct 8 14:32:04 2018 -0700

--
 docs/structured-streaming-programming-guide.md | 323 +++-
 1 file changed, 312 insertions(+), 11 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/f9935a3f/docs/structured-streaming-programming-guide.md
--
diff --git a/docs/structured-streaming-programming-guide.md 
b/docs/structured-streaming-programming-guide.md
index 73de189..b6e4277 100644
--- a/docs/structured-streaming-programming-guide.md
+++ b/docs/structured-streaming-programming-guide.md
@@ -1560,6 +1560,35 @@ streamingDf <- dropDuplicates(streamingDf, "guid", 
"eventTime")
 
 
 
+### Policy for handling multiple watermarks
+A streaming query can have multiple input streams that are unioned or joined 
together.
+Each of the input streams can have a different threshold of late data that 
needs to
+be tolerated for stateful operations. You specify these thresholds using
+``withWatermarks("eventTime", delay)`` on each of the input streams. For 
example, consider
+a query with stream-stream joins between `inputStream1` and `inputStream2`.
+
+  inputStream1.withWatermark("eventTime1", "1 hour")
+.join(
+  inputStream2.withWatermark("eventTime2", "2 hours"),
+  joinCondition)
+
+While executing the query, Structured Streaming individually tracks the maximum
+event time seen in each input stream, calculates watermarks based on the 
corresponding delay,
+and chooses a single global watermark with them to be used for stateful 
operations. By default,
+the minimum is chosen as the global watermark because it ensures that no data 
is
+accidentally dropped as too late if one of the streams falls behind the others
+(for example, one of the streams stop receiving data due to upstream 
failures). In other words,
+the global watermark will safely move at the pace of the slowest stream and 
the query output will
+be delayed accordingly.
+
+However, in some cases, you may want to get faster results even if it means 
dropping data from the
+slowest stream. Since Spark 2.4, you can set the multiple watermark policy to 
choose
+the maximum value as the global watermark by setting the SQL configuration
+``spark.sql.streaming.multipleWatermarkPolicy`` to ``max`` (default is 
``min``). 
+This lets the global watermark move at the pace of the fastest stream.
+However, as a side effect, data from the slower streams will be aggressively 
dropped. Hence, use
+this configuration judiciously.
+
 ### Arbitrary Stateful Operations
 Many usecases require more advanced stateful operations than aggregations. For 
example, in many usecases, you have to track sessions from data streams of 
events. For doing such sessionization, you will have to save arbitrary types of 
data as state, and perform arbitrary operations on the state using the data 
stream events in every trigger. Since Spark 2.2, this can be done using the 
operation `mapGroupsWithState` and the more powerful operation 
`flatMapGroupsWithState`. Both operations allow you to apply user-defined code 
on grouped Datasets to update user-defined state. For more concrete details, 
take a look at the API documentation 
([Scala](api/scala/index.html#org.apache.spark.sql.streaming.GroupState)/[Java](api/java/org/apache/spark/sql/streaming/GroupState.html))
 and the examples 
([Scala]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredSessionization.scala)/[Java]({{site.SPARK_GITHUB_URL}}/b
 
lob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredSessionization.java)).
 
@@ -1799,9 +1828,17 @@ Here are the details of all the sinks in Spark.
 Append, Update, Compl

spark git commit: [SPARK-25399][SS] Continuous processing state should not affect microbatch execution jobs

2018-09-11 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/branch-2.4 3a6ef8b7e -> 0dbf1450f


[SPARK-25399][SS] Continuous processing state should not affect microbatch 
execution jobs

## What changes were proposed in this pull request?

The leftover state from running a continuous processing streaming job should 
not affect later microbatch execution jobs. If a continuous processing job runs 
and the same thread gets reused for a microbatch execution job in the same 
environment, the microbatch job could get wrong answers because it can attempt 
to load the wrong version of the state.

## How was this patch tested?

New and existing unit tests

Closes #22386 from mukulmurthy/25399-streamthread.

Authored-by: Mukul Murthy 
Signed-off-by: Tathagata Das 
(cherry picked from commit 9f5c5b4cca7d4eaa30a3f8adb4cb1eebe3f77c7a)
Signed-off-by: Tathagata Das 


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

Branch: refs/heads/branch-2.4
Commit: 0dbf1450f7965c27ce9329c7dad351ff8b8072dc
Parents: 3a6ef8b
Author: Mukul Murthy 
Authored: Tue Sep 11 15:53:15 2018 -0700
Committer: Tathagata Das 
Committed: Tue Sep 11 15:53:25 2018 -0700

--
 .../streaming/MicroBatchExecution.scala |  2 ++
 .../execution/streaming/StreamExecution.scala   |  1 +
 .../continuous/ContinuousExecution.scala|  2 ++
 .../streaming/state/StateStoreRDD.scala | 12 +--
 .../spark/sql/streaming/StreamSuite.scala   | 33 ++--
 5 files changed, 45 insertions(+), 5 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/0dbf1450/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala
--
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 b1cafd6..2cac865 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
@@ -511,6 +511,8 @@ class MicroBatchExecution(
 
 sparkSessionToRunBatch.sparkContext.setLocalProperty(
   MicroBatchExecution.BATCH_ID_KEY, currentBatchId.toString)
+sparkSessionToRunBatch.sparkContext.setLocalProperty(
+  StreamExecution.IS_CONTINUOUS_PROCESSING, false.toString)
 
 reportTimeTaken("queryPlanning") {
   lastExecution = new IncrementalExecution(

http://git-wip-us.apache.org/repos/asf/spark/blob/0dbf1450/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 a39bb71..f6c60c1 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
@@ -529,6 +529,7 @@ abstract class StreamExecution(
 
 object StreamExecution {
   val QUERY_ID_KEY = "sql.streaming.queryId"
+  val IS_CONTINUOUS_PROCESSING = "__is_continuous_processing"
 
   def isInterruptionException(e: Throwable): Boolean = e match {
 // InterruptedIOException - thrown when an I/O operation is interrupted

http://git-wip-us.apache.org/repos/asf/spark/blob/0dbf1450/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 4ddebb3..ccca726 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
@@ -210,6 +210,8 @@ class ContinuousExecution(
 }.head
 
 sparkSessionForQuery.sparkContext.setLocalProperty(
+  StreamExecution.IS_CONTINUOUS_PROCESSING, true.toString)
+sparkSessionForQuery.sparkContext.setLocalProperty(
   ContinuousExecution.START_EPOCH_KEY, currentBatchId.toString)
 // Add another random ID on top of the run ID, to distinguish epoch 
coordinators across
 // reconfigurations.


spark git commit: [SPARK-25399][SS] Continuous processing state should not affect microbatch execution jobs

2018-09-11 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master 97d4afaa1 -> 9f5c5b4cc


[SPARK-25399][SS] Continuous processing state should not affect microbatch 
execution jobs

## What changes were proposed in this pull request?

The leftover state from running a continuous processing streaming job should 
not affect later microbatch execution jobs. If a continuous processing job runs 
and the same thread gets reused for a microbatch execution job in the same 
environment, the microbatch job could get wrong answers because it can attempt 
to load the wrong version of the state.

## How was this patch tested?

New and existing unit tests

Closes #22386 from mukulmurthy/25399-streamthread.

Authored-by: Mukul Murthy 
Signed-off-by: Tathagata Das 


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

Branch: refs/heads/master
Commit: 9f5c5b4cca7d4eaa30a3f8adb4cb1eebe3f77c7a
Parents: 97d4afa
Author: Mukul Murthy 
Authored: Tue Sep 11 15:53:15 2018 -0700
Committer: Tathagata Das 
Committed: Tue Sep 11 15:53:15 2018 -0700

--
 .../streaming/MicroBatchExecution.scala |  2 ++
 .../execution/streaming/StreamExecution.scala   |  1 +
 .../continuous/ContinuousExecution.scala|  2 ++
 .../streaming/state/StateStoreRDD.scala | 12 +--
 .../spark/sql/streaming/StreamSuite.scala   | 33 ++--
 5 files changed, 45 insertions(+), 5 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/9f5c5b4c/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala
--
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 b1cafd6..2cac865 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
@@ -511,6 +511,8 @@ class MicroBatchExecution(
 
 sparkSessionToRunBatch.sparkContext.setLocalProperty(
   MicroBatchExecution.BATCH_ID_KEY, currentBatchId.toString)
+sparkSessionToRunBatch.sparkContext.setLocalProperty(
+  StreamExecution.IS_CONTINUOUS_PROCESSING, false.toString)
 
 reportTimeTaken("queryPlanning") {
   lastExecution = new IncrementalExecution(

http://git-wip-us.apache.org/repos/asf/spark/blob/9f5c5b4c/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 a39bb71..f6c60c1 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
@@ -529,6 +529,7 @@ abstract class StreamExecution(
 
 object StreamExecution {
   val QUERY_ID_KEY = "sql.streaming.queryId"
+  val IS_CONTINUOUS_PROCESSING = "__is_continuous_processing"
 
   def isInterruptionException(e: Throwable): Boolean = e match {
 // InterruptedIOException - thrown when an I/O operation is interrupted

http://git-wip-us.apache.org/repos/asf/spark/blob/9f5c5b4c/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 4ddebb3..ccca726 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
@@ -210,6 +210,8 @@ class ContinuousExecution(
 }.head
 
 sparkSessionForQuery.sparkContext.setLocalProperty(
+  StreamExecution.IS_CONTINUOUS_PROCESSING, true.toString)
+sparkSessionForQuery.sparkContext.setLocalProperty(
   ContinuousExecution.START_EPOCH_KEY, currentBatchId.toString)
 // Add another random ID on top of the run ID, to distinguish epoch 
coordinators across
 // reconfigurations.


spark git commit: [SPARK-25204][SS] Fix race in rate source test.

2018-08-23 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master a9aacdf1c -> 8ed044928


[SPARK-25204][SS] Fix race in rate source test.

## What changes were proposed in this pull request?

Fix a race in the rate source tests. We need a better way of testing restart 
behavior.

## How was this patch tested?

unit test

Closes #22191 from jose-torres/racetest.

Authored-by: Jose Torres 
Signed-off-by: Tathagata Das 


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

Branch: refs/heads/master
Commit: 8ed0449285507459bbd00752338ed3242427a14f
Parents: a9aacdf
Author: Jose Torres 
Authored: Thu Aug 23 12:14:27 2018 -0700
Committer: Tathagata Das 
Committed: Thu Aug 23 12:14:27 2018 -0700

--
 .../sources/RateStreamProviderSuite.scala   | 40 ++--
 .../apache/spark/sql/streaming/StreamTest.scala |  5 ++-
 2 files changed, 40 insertions(+), 5 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/8ed04492/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProviderSuite.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProviderSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProviderSuite.scala
index 9c1756d..dd74af8 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProviderSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProviderSuite.scala
@@ -22,6 +22,7 @@ import java.util.concurrent.TimeUnit
 import scala.collection.JavaConverters._
 import scala.collection.mutable.ArrayBuffer
 
+import org.apache.spark.sql.Row
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.execution.datasources.DataSource
 import org.apache.spark.sql.execution.streaming._
@@ -81,12 +82,43 @@ class RateSourceSuite extends StreamTest {
   .load()
 testStream(input)(
   AdvanceRateManualClock(seconds = 1),
-  CheckLastBatch((0 until 10).map(v => new java.sql.Timestamp(v * 100L) -> 
v): _*),
+  CheckLastBatch((0 until 10).map(v => new java.sql.Timestamp(v * 100L) -> 
v): _*)
+)
+  }
+
+  test("microbatch - restart") {
+val input = spark.readStream
+  .format("rate")
+  .option("rowsPerSecond", "10")
+  .load()
+  .select('value)
+
+var streamDuration = 0
+
+// Microbatch rate stream offsets contain the number of seconds since the 
beginning of
+// the stream.
+def updateStreamDurationFromOffset(s: StreamExecution, expectedMin: Int): 
Unit = {
+  streamDuration = s.lastProgress.sources(0).endOffset.toInt
+  assert(streamDuration >= expectedMin)
+}
+
+// We have to use the lambda version of CheckAnswer because we don't know 
the right range
+// until we see the last offset.
+def expectedResultsFromDuration(rows: Seq[Row]): Unit = {
+  assert(rows.map(_.getLong(0)).sorted == (0 until (streamDuration * 10)))
+}
+
+testStream(input)(
+  StartStream(),
+  Execute(_.awaitOffset(0, LongOffset(2), streamingTimeout.toMillis)),
   StopStream,
+  Execute(updateStreamDurationFromOffset(_, 2)),
+  CheckAnswer(expectedResultsFromDuration _),
   StartStream(),
-  // Advance 2 seconds because creating a new RateSource will also create 
a new ManualClock
-  AdvanceRateManualClock(seconds = 2),
-  CheckLastBatch((10 until 20).map(v => new java.sql.Timestamp(v * 100L) 
-> v): _*)
+  Execute(_.awaitOffset(0, LongOffset(4), streamingTimeout.toMillis)),
+  StopStream,
+  Execute(updateStreamDurationFromOffset(_, 4)),
+  CheckAnswer(expectedResultsFromDuration _)
 )
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/8ed04492/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala
index cd9b892..491dc34 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala
@@ -735,7 +735,10 @@ trait StreamTest extends QueryTest with SharedSQLContext 
with TimeLimits with Be
   }
 
 case CheckAnswerRowsByFunc(globalCheckFunction, lastOnly) =>
-  val sparkAnswer = fetchStreamAnswer(currentStream, lastOnly)
+  val sparkAnswer = currentStream match {
+   

spark git commit: [SPARK-25184][SS] Fixed race condition in StreamExecution that caused flaky test in FlatMapGroupsWithState

2018-08-22 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master 68ec4d641 -> 310632498


[SPARK-25184][SS] Fixed race condition in StreamExecution that caused flaky 
test in FlatMapGroupsWithState

## What changes were proposed in this pull request?

The race condition that caused test failure is between 2 threads.
- The MicrobatchExecution thread that processes inputs to produce answers and 
then generates progress events.
- The test thread that generates some input data, checked the answer and then 
verified the query generated progress event.

The synchronization structure between these threads is as follows
1. MicrobatchExecution thread, in every batch, does the following in order.
   a. Processes batch input to generate answer.
   b. Signals `awaitProgressLockCondition` to wake up threads waiting for 
progress using `awaitOffset`
   c. Generates progress event

2. Test execution thread
   a. Calls `awaitOffset` to wait for progress, which waits on 
`awaitProgressLockCondition`.
   b. As soon as `awaitProgressLockCondition` is signaled, it would move on the 
in the test to check answer.
  c. Finally, it would verify the last generated progress event.

What can happen is the following sequence of events: 2a -> 1a -> 1b -> 2b -> 2c 
-> 1c.
In other words, the progress event may be generated after the test tries to 
verify it.

The solution has two steps.
1. Signal the waiting thread after the progress event has been generated, that 
is, after `finishTrigger()`.
2. Increase the timeout of `awaitProgressLockCondition.await(100 ms)` to a 
large value.

This latter is to ensure that test thread for keeps waiting on 
`awaitProgressLockCondition`until the MicroBatchExecution thread explicitly 
signals it. With the existing small timeout of 100ms the following sequence can 
occur.
 - MicroBatchExecution thread updates committed offsets
 - Test thread waiting on `awaitProgressLockCondition` accidentally times out 
after 100 ms, finds that the committed offsets have been updated, therefore 
returns from `awaitOffset` and moves on to the progress event tests.
 - MicroBatchExecution thread then generates progress event and signals. But 
the test thread has already attempted to verify the event and failed.

By increasing the timeout to large (e.g., `streamingTimeoutMs = 60 seconds`, 
similar to `awaitInitialization`), this above type of race condition is also 
avoided.

## How was this patch tested?
Ran locally many times.

Closes #22182 from tdas/SPARK-25184.

Authored-by: Tathagata Das 
Signed-off-by: Tathagata Das 


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

Branch: refs/heads/master
Commit: 3106324986612800240bc8c945be90c4cb368d79
Parents: 68ec4d6
Author: Tathagata Das 
Authored: Wed Aug 22 12:22:53 2018 -0700
Committer: Tathagata Das 
Committed: Wed Aug 22 12:22:53 2018 -0700

--
 .../kafka010/KafkaMicroBatchSourceSuite.scala   |  3 +-
 .../streaming/MicroBatchExecution.scala |  5 ++-
 .../execution/streaming/StreamExecution.scala   |  4 +-
 .../sql/streaming/StateStoreMetricsTest.scala   | 44 +++-
 .../apache/spark/sql/streaming/StreamTest.scala |  2 +-
 5 files changed, 33 insertions(+), 25 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/31063249/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 946b636..c9c5250 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
@@ -970,7 +970,8 @@ abstract class KafkaSourceSuiteBase extends KafkaSourceTest 
{
   makeSureGetOffsetCalled,
   Execute { q =>
 // wait to reach the last offset in every partition
-q.awaitOffset(0, KafkaSourceOffset(partitionOffsets.mapValues(_ => 
3L)))
+q.awaitOffset(
+  0, KafkaSourceOffset(partitionOffsets.mapValues(_ => 3L)), 
streamingTimeout.toMillis)
   },
   CheckAnswer(-20, -21, -22, 0, 1, 2, 11, 12, 22),
   StopStream,

http://git-wip-us.apache.org/repos/asf/spark/blob/31063249/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala
--
diff --git 
a/sql/core/src/

spark git commit: [MINOR] Added import to fix compilation

2018-08-21 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master ad45299d0 -> a998e9d82


[MINOR] Added import to fix compilation

## What changes were proposed in this pull request?

Two back to PRs implicitly conflicted by one PR removing an existing import 
that the other PR needed. This did not cause explicit conflict as the import 
already existed, but not used.

https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Compile/job/spark-master-compile-maven-hadoop-2.7/8226/consoleFull

```
[info] Compiling 342 Scala sources and 97 Java sources to 
/home/jenkins/workspace/spark-master-compile-maven-hadoop-2.7/sql/core/target/scala-2.11/classes...
[warn] 
/home/jenkins/workspace/spark-master-compile-maven-hadoop-2.7/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala:128:
 value ENABLE_JOB_SUMMARY in object ParquetOutputFormat is deprecated: see 
corresponding Javadoc for more information.
[warn]   && conf.get(ParquetOutputFormat.ENABLE_JOB_SUMMARY) == null) {
[warn]   ^
[error] 
/home/jenkins/workspace/spark-master-compile-maven-hadoop-2.7/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala:95:
 value asJava is not a member of scala.collection.immutable.Map[String,Long]
[error]   new java.util.HashMap(customMetrics.mapValues(long2Long).asJava)
[error]^
[warn] one warning found
[error] one error found
[error] Compile failed at Aug 21, 2018 4:04:35 PM [12.827s]
```

## How was this patch tested?
It compiles!

Closes #22175 from tdas/fix-build.

Authored-by: Tathagata Das 
Signed-off-by: Tathagata Das 


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

Branch: refs/heads/master
Commit: a998e9d829bd499dd7c65f973ea4389e0401b001
Parents: ad45299
Author: Tathagata Das 
Authored: Tue Aug 21 17:08:15 2018 -0700
Committer: Tathagata Das 
Committed: Tue Aug 21 17:08:15 2018 -0700

--
 .../apache/spark/sql/execution/streaming/statefulOperators.scala   | 2 ++
 1 file changed, 2 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/a998e9d8/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala
index 7351db8..c11af34 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala
@@ -20,6 +20,8 @@ package org.apache.spark.sql.execution.streaming
 import java.util.UUID
 import java.util.concurrent.TimeUnit._
 
+import scala.collection.JavaConverters._
+
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.errors._


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



spark git commit: [SPARK-24441][SS] Expose total estimated size of states in HDFSBackedStateStoreProvider

2018-08-21 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master ac0174e55 -> 42035a4fe


[SPARK-24441][SS] Expose total estimated size of states in 
HDFSBackedStateStoreProvider

## What changes were proposed in this pull request?

This patch exposes the estimation of size of cache (loadedMaps) in 
HDFSBackedStateStoreProvider as a custom metric of StateStore.

The rationalize of the patch is that state backed by 
HDFSBackedStateStoreProvider will consume more memory than the number what we 
can get from query status due to caching multiple versions of states. The 
memory footprint to be much larger than query status reports in situations 
where the state store is getting a lot of updates: while shallow-copying map 
incurs additional small memory usages due to the size of map entities and 
references, but row objects will still be shared across the versions. If 
there're lots of updates between batches, less row objects will be shared and 
more row objects will exist in memory consuming much memory then what we expect.

While HDFSBackedStateStore refers loadedMaps in HDFSBackedStateStoreProvider 
directly, there would be only one `StateStoreWriter` which refers a 
StateStoreProvider, so the value is not exposed as well as being aggregated 
multiple times. Current state metrics are safe to aggregate for the same reason.

## How was this patch tested?

Tested manually. Below is the snapshot of UI page which is reflected by the 
patch:

https://user-images.githubusercontent.com/1317309/40978481-b46ad324-690e-11e8-9b0f-e80528612a62.png;>

Please refer "estimated size of states cache in provider total" as well as 
"count of versions in state cache in provider".

Closes #21469 from HeartSaVioR/SPARK-24441.

Authored-by: Jungtaek Lim 
Signed-off-by: Tathagata Das 


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

Branch: refs/heads/master
Commit: 42035a4fec6eb216427486b5067a45fceb65cc2d
Parents: ac0174e
Author: Jungtaek Lim 
Authored: Tue Aug 21 15:28:31 2018 -0700
Committer: Tathagata Das 
Committed: Tue Aug 21 15:28:31 2018 -0700

--
 .../state/HDFSBackedStateStoreProvider.scala|  39 +++-
 .../execution/streaming/state/StateStore.scala  |   2 +
 .../state/SymmetricHashJoinStateManager.scala   |   2 +
 .../execution/streaming/statefulOperators.scala |  12 ++-
 .../apache/spark/sql/streaming/progress.scala   |  15 ++-
 .../streaming/state/StateStoreSuite.scala   | 100 +++
 .../streaming/StreamingQueryListenerSuite.scala |   2 +-
 .../StreamingQueryStatusAndProgressSuite.scala  |  13 ++-
 8 files changed, 176 insertions(+), 9 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/42035a4f/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala
index 523acef..92a2480 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala
@@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.streaming.state
 import java.io._
 import java.util
 import java.util.Locale
+import java.util.concurrent.atomic.LongAdder
 
 import scala.collection.JavaConverters._
 import scala.collection.mutable
@@ -165,7 +166,16 @@ private[state] class HDFSBackedStateStoreProvider extends 
StateStoreProvider wit
 }
 
 override def metrics: StateStoreMetrics = {
-  StateStoreMetrics(mapToUpdate.size(), 
SizeEstimator.estimate(mapToUpdate), Map.empty)
+  // NOTE: we provide estimation of cache size as "memoryUsedBytes", and 
size of state for
+  // current version as "stateOnCurrentVersionSizeBytes"
+  val metricsFromProvider: Map[String, Long] = getMetricsForProvider()
+
+  val customMetrics = metricsFromProvider.flatMap { case (name, value) =>
+// just allow searching from list cause the list is small enough
+supportedCustomMetrics.find(_.name == name).map(_ -> value)
+  } + (metricStateOnCurrentVersionSizeBytes -> 
SizeEstimator.estimate(mapToUpdate))
+
+  StateStoreMetrics(mapToUpdate.size(), 
metricsFromProvider("memoryUsedBytes"), customMetrics)
 }
 
 /**
@@ -180,6 +190,12 @@ private[state] class HDFSBackedStateStoreProvider extends 
StateStoreProvider wit
 }
   }
 
+  def 

spark git commit: [SPARK-24763][SS] Remove redundant key data from value in streaming aggregation

2018-08-21 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master 72ecfd095 -> 6c5cb8585


[SPARK-24763][SS] Remove redundant key data from value in streaming aggregation

## What changes were proposed in this pull request?

This patch proposes a new flag option for stateful aggregation: remove 
redundant key data from value.
Enabling new option runs similar with current, and uses less memory for state 
according to key/value fields of state operator.

Please refer below link to see detailed perf. test result:
https://issues.apache.org/jira/browse/SPARK-24763?focusedCommentId=16536539=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-16536539

Since the state between enabling the option and disabling the option is not 
compatible, the option is set to 'disable' by default (to ensure backward 
compatibility), and OffsetSeqMetadata would prevent modifying the option after 
executing query.

## How was this patch tested?

Modify unit tests to cover both disabling option and enabling option.
Also did manual tests to see whether propose patch improves state memory usage.

Closes #21733 from HeartSaVioR/SPARK-24763.

Authored-by: Jungtaek Lim 
Signed-off-by: Tathagata Das 


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

Branch: refs/heads/master
Commit: 6c5cb85856235efd464b109558896f81ae2c4c75
Parents: 72ecfd0
Author: Jungtaek Lim 
Authored: Tue Aug 21 15:22:42 2018 -0700
Committer: Tathagata Das 
Committed: Tue Aug 21 15:22:42 2018 -0700

--
 .../org/apache/spark/sql/internal/SQLConf.scala |  10 +
 .../spark/sql/execution/SparkStrategies.scala   |   3 +
 .../sql/execution/aggregate/AggUtils.scala  |   5 +-
 .../streaming/IncrementalExecution.scala|   6 +-
 .../sql/execution/streaming/OffsetSeq.scala |   8 +-
 .../StreamingAggregationStateManager.scala  | 205 +++
 .../execution/streaming/statefulOperators.scala |  61 +++---
 .../commits/0   |   2 +
 .../commits/1   |   2 +
 .../metadata|   1 +
 .../offsets/0   |   3 +
 .../offsets/1   |   3 +
 .../state/0/0/1.delta   | Bin 0 -> 46 bytes
 .../state/0/0/2.delta   | Bin 0 -> 46 bytes
 .../state/0/1/1.delta   | Bin 0 -> 77 bytes
 .../state/0/1/2.delta   | Bin 0 -> 77 bytes
 .../state/0/2/1.delta   | Bin 0 -> 46 bytes
 .../state/0/2/2.delta   | Bin 0 -> 46 bytes
 .../state/0/3/1.delta   | Bin 0 -> 46 bytes
 .../state/0/3/2.delta   | Bin 0 -> 46 bytes
 .../state/0/4/1.delta   | Bin 0 -> 46 bytes
 .../state/0/4/2.delta   | Bin 0 -> 77 bytes
 .../streaming/state/MemoryStateStore.scala  |  49 +
 .../StreamingAggregationStateManagerSuite.scala | 126 
 .../streaming/FlatMapGroupsWithStateSuite.scala |  24 +--
 .../streaming/StreamingAggregationSuite.scala   | 150 +++---
 26 files changed, 573 insertions(+), 85 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/6c5cb858/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
--
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 bffdddc..b44bfe7 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
@@ -888,6 +888,16 @@ object SQLConf {
 .intConf
 .createWithDefault(2)
 
+  val STREAMING_AGGREGATION_STATE_FORMAT_VERSION =
+buildConf("spark.sql.streaming.aggregation.stateFormatVersion")
+  .internal()
+  .doc("State format version used by streaming aggregation operations in a 
streaming query. " +
+"State between versions are tend to be incompatible, so state format 
version shouldn't " +
+"be modified after running.")
+  .intConf
+  .checkValue(v => Set(1, 2).contains(v), "Valid versions are 1 and 2")
+  .createWithDefault(2)
+
   val UNSUPPORTED_OPERATION_CHECK_ENABLED =
 buildConf("spark.sql.streaming.unsupportedOperationCheck")
   .internal()

http://git-wip-us.apache.org/repos/asf/spark/blob/6c5cb858/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala

spark git commit: [SPARK-24699][SS] Make watermarks work with Trigger.Once by saving updated watermark to commit log

2018-07-23 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master 2edf17eff -> 61f0ca4f1


[SPARK-24699][SS] Make watermarks work with Trigger.Once by saving updated 
watermark to commit log

## What changes were proposed in this pull request?

Streaming queries with watermarks do not work with Trigger.Once because of the 
following.
- Watermark is updated in the driver memory after a batch completes, but it is 
persisted to checkpoint (in the offset log) only when the next batch is planned
- In trigger.once, the query terminated as soon as one batch has completed. 
Hence, the updated watermark is never persisted anywhere.

The simple solution is to persist the updated watermark value in the commit log 
when a batch is marked as completed. Then the next batch, in the next 
trigger.once run can pick it up from the commit log.

## How was this patch tested?
new unit tests

Co-authored-by: Tathagata Das 
Co-authored-by: c-horn 

Author: Tathagata Das 

Closes #21746 from tdas/SPARK-24699.


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

Branch: refs/heads/master
Commit: 61f0ca4f1c4f1498c0b6ad02370839619871d6c5
Parents: 2edf17e
Author: Tathagata Das 
Authored: Mon Jul 23 13:03:32 2018 -0700
Committer: Tathagata Das 
Committed: Mon Jul 23 13:03:32 2018 -0700

--
 .../sql/execution/streaming/CommitLog.scala |  33 ++--
 .../streaming/MicroBatchExecution.scala |   9 +-
 .../continuous/ContinuousExecution.scala|   2 +-
 .../commits/0   |   2 +
 .../commits/1   |   2 +
 .../metadata|   1 +
 .../offsets/0   |   3 +
 .../offsets/1   |   3 +
 .../state/0/0/1.delta   | Bin 0 -> 46 bytes
 .../state/0/0/2.delta   | Bin 0 -> 46 bytes
 .../state/0/1/1.delta   | Bin 0 -> 46 bytes
 .../state/0/1/2.delta   | Bin 0 -> 46 bytes
 .../state/0/2/1.delta   | Bin 0 -> 103 bytes
 .../state/0/2/2.delta   | Bin 0 -> 46 bytes
 .../state/0/3/1.delta   | Bin 0 -> 46 bytes
 .../state/0/3/2.delta   | Bin 0 -> 46 bytes
 .../state/0/4/1.delta   | Bin 0 -> 46 bytes
 .../state/0/4/2.delta   | Bin 0 -> 103 bytes
 .../sql/streaming/EventTimeWatermarkSuite.scala | 156 ---
 .../apache/spark/sql/streaming/StreamTest.scala |   8 +-
 20 files changed, 177 insertions(+), 42 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/61f0ca4f/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CommitLog.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CommitLog.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CommitLog.scala
index 5b11424..0063318 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CommitLog.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CommitLog.scala
@@ -22,6 +22,9 @@ import java.nio.charset.StandardCharsets._
 
 import scala.io.{Source => IOSource}
 
+import org.json4s.NoTypeHints
+import org.json4s.jackson.Serialization
+
 import org.apache.spark.sql.SparkSession
 
 /**
@@ -43,36 +46,28 @@ import org.apache.spark.sql.SparkSession
  * line 2: metadata (optional json string)
  */
 class CommitLog(sparkSession: SparkSession, path: String)
-  extends HDFSMetadataLog[String](sparkSession, path) {
+  extends HDFSMetadataLog[CommitMetadata](sparkSession, path) {
 
   import CommitLog._
 
-  def add(batchId: Long): Unit = {
-super.add(batchId, EMPTY_JSON)
-  }
-
-  override def add(batchId: Long, metadata: String): Boolean = {
-throw new UnsupportedOperationException(
-  "CommitLog does not take any metadata, use 'add(batchId)' instead")
-  }
-
-  override protected def deserialize(in: InputStream): String = {
+  override protected def deserialize(in: InputStream): CommitMetadata = {
 // called inside a try-finally where the underlying stream is closed in 
the caller
 val lines = IOSource.fromInputStream(in, UTF_8.name()).getLines()
 if (!lines.hasNext) {
   throw new IllegalStateException("Incomplete log file in the offset 
commit log")
 }
 parseVersion(lines.next.trim, VERSION)
-EMPTY_JSON
+val metadataJson = if (lines.hasNext) lines.next else EMPTY_JSON
+Commi

spark git commit: [SPARK-22187][SS] Update unsaferow format for saved state in flatMapGroupsWithState to allow timeouts with deleted state

2018-07-19 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master 8d707b060 -> b3d88ac02


[SPARK-22187][SS] Update unsaferow format for saved state in 
flatMapGroupsWithState to allow timeouts with deleted state

## What changes were proposed in this pull request?

Currently, the group state of user-defined-type is encoded as top-level columns 
in the UnsafeRows stores in the state store. The timeout timestamp is also 
saved as (when needed) as the last top-level column. Since the group state is 
serialized to top-level columns, you cannot save "null" as a value of state 
(setting null in all the top-level columns is not equivalent). So we don't let 
the user set the timeout without initializing the state for a key. Based on 
user experience, this leads to confusion.

This PR is to change the row format such that the state is saved as nested 
columns. This would allow the state to be set to null, and avoid these 
confusing corner cases. However, queries recovering from existing checkpoint 
will use the previous format to maintain compatibility with existing production 
queries.

## How was this patch tested?
Refactored existing end-to-end tests and added new tests for explicitly testing 
obj-to-row conversion for both state formats.

Author: Tathagata Das 

Closes #21739 from tdas/SPARK-22187-1.


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

Branch: refs/heads/master
Commit: b3d88ac02940eff4c867d3acb79fe5ff9d724e83
Parents: 8d707b0
Author: Tathagata Das 
Authored: Thu Jul 19 13:17:28 2018 -0700
Committer: Tathagata Das 
Committed: Thu Jul 19 13:17:28 2018 -0700

--
 .../sql/catalyst/expressions/Expression.scala   |   3 +-
 .../org/apache/spark/sql/internal/SQLConf.scala |   8 +
 .../spark/sql/execution/SparkStrategies.scala   |   5 +-
 .../streaming/FlatMapGroupsWithStateExec.scala  | 136 +++---
 .../sql/execution/streaming/OffsetSeq.scala |  10 +-
 .../FlatMapGroupsWithStateExecHelper.scala  | 247 ++
 .../commits/0   |   2 +
 .../commits/1   |   2 +
 .../metadata|   1 +
 .../offsets/0   |   3 +
 .../offsets/1   |   3 +
 .../state/0/0/1.delta   | Bin 0 -> 84 bytes
 .../state/0/0/2.delta   | Bin 0 -> 46 bytes
 .../state/0/1/1.delta   | Bin 0 -> 46 bytes
 .../state/0/1/2.delta   | Bin 0 -> 46 bytes
 .../state/0/2/1.delta   | Bin 0 -> 46 bytes
 .../state/0/2/2.delta   | Bin 0 -> 46 bytes
 .../state/0/3/1.delta   | Bin 0 -> 46 bytes
 .../state/0/3/2.delta   | Bin 0 -> 46 bytes
 .../state/0/4/1.delta   | Bin 0 -> 46 bytes
 .../state/0/4/2.delta   | Bin 0 -> 46 bytes
 .../FlatMapGroupsWithStateExecHelperSuite.scala | 218 
 .../streaming/FlatMapGroupsWithStateSuite.scala | 250 ++-
 23 files changed, 708 insertions(+), 180 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/b3d88ac0/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
index f7d1b10..a69b804 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
@@ -715,7 +715,8 @@ trait ComplexTypeMergingExpression extends Expression {
   "The collection of input data types must not be empty.")
 require(
   TypeCoercion.haveSameType(inputTypesForMerging),
-  "All input types must be the same except nullable, containsNull, 
valueContainsNull flags.")
+  "All input types must be the same except nullable, containsNull, 
valueContainsNull flags." +
+s" The input types found 
are\n\t${inputTypesForMerging.mkString("\n\t")}")
 
inputTypesForMerging.reduceLeft(TypeCoercion.findCommonTypeDifferentOnlyInNullFlags(_,
 _).get)
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/b3d88ac0/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
-

spark git commit: [SPARK-24717][SS] Split out max retain version of state for memory in HDFSBackedStateStoreProvider

2018-07-19 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master d05a926e7 -> 8b7d4f842


[SPARK-24717][SS] Split out max retain version of state for memory in 
HDFSBackedStateStoreProvider

## What changes were proposed in this pull request?

This patch proposes breaking down configuration of retaining batch size on 
state into two pieces: files and in memory (cache). While this patch reuses 
existing configuration for files, it introduces new configuration, 
"spark.sql.streaming.maxBatchesToRetainInMemory" to configure max count of 
batch to retain in memory.

## How was this patch tested?

Apply this patch on top of SPARK-24441 
(https://github.com/apache/spark/pull/21469), and manually tested in various 
workloads to ensure overall size of states in memory is around 2x or less of 
the size of latest version of state, while it was 10x ~ 80x before applying the 
patch.

Author: Jungtaek Lim 

Closes #21700 from HeartSaVioR/SPARK-24717.


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

Branch: refs/heads/master
Commit: 8b7d4f842fdc90b8d1c37080bdd9b5e1d070f5c0
Parents: d05a926
Author: Jungtaek Lim 
Authored: Thu Jul 19 00:07:35 2018 -0700
Committer: Tathagata Das 
Committed: Thu Jul 19 00:07:35 2018 -0700

--
 .../org/apache/spark/sql/internal/SQLConf.scala |  11 ++
 .../state/HDFSBackedStateStoreProvider.scala|  57 +--
 .../streaming/state/StateStoreConf.scala|   3 +
 .../streaming/state/StateStoreSuite.scala   | 150 +--
 4 files changed, 196 insertions(+), 25 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/8b7d4f84/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
--
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 41fe0c3..9239d4e 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
@@ -854,6 +854,15 @@ object SQLConf {
 .intConf
 .createWithDefault(100)
 
+  val MAX_BATCHES_TO_RETAIN_IN_MEMORY = 
buildConf("spark.sql.streaming.maxBatchesToRetainInMemory")
+.internal()
+.doc("The maximum number of batches which will be retained in memory to 
avoid " +
+  "loading from files. The value adjusts a trade-off between memory usage 
vs cache miss: " +
+  "'2' covers both success and direct failure cases, '1' covers only 
success case, " +
+  "and '0' covers extreme case - disable cache to maximize memory size of 
executors.")
+.intConf
+.createWithDefault(2)
+
   val UNSUPPORTED_OPERATION_CHECK_ENABLED =
 buildConf("spark.sql.streaming.unsupportedOperationCheck")
   .internal()
@@ -1507,6 +1516,8 @@ class SQLConf extends Serializable with Logging {
 
   def minBatchesToRetain: Int = getConf(MIN_BATCHES_TO_RETAIN)
 
+  def maxBatchesToRetainInMemory: Int = 
getConf(MAX_BATCHES_TO_RETAIN_IN_MEMORY)
+
   def parquetFilterPushDown: Boolean = getConf(PARQUET_FILTER_PUSHDOWN_ENABLED)
 
   def parquetFilterPushDownDate: Boolean = 
getConf(PARQUET_FILTER_PUSHDOWN_DATE_ENABLED)

http://git-wip-us.apache.org/repos/asf/spark/blob/8b7d4f84/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala
index 118c82a..523acef 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala
@@ -18,6 +18,7 @@
 package org.apache.spark.sql.execution.streaming.state
 
 import java.io._
+import java.util
 import java.util.Locale
 
 import scala.collection.JavaConverters._
@@ -203,6 +204,7 @@ private[state] class HDFSBackedStateStoreProvider extends 
StateStoreProvider wit
 this.valueSchema = valueSchema
 this.storeConf = storeConf
 this.hadoopConf = hadoopConf
+this.numberOfVersionsToRetainInMemory = 
storeConf.maxVersionsToRetainInMemory
 fm.mkdirs(baseDir)
   }
 
@@ -220,7 +222,7 @@ private[state] class HDFSBackedStateStoreProvider extends 
StateStoreProvider wit
   }
 
   override def close(): Unit = {
-loadedMaps.values.foreach(_.clear())
+

spark git commit: [SPARK-24697][SS] Fix the reported start offsets in streaming query progress

2018-07-11 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master 59c3c233f -> ff7f6ef75


[SPARK-24697][SS] Fix the reported start offsets in streaming query progress

## What changes were proposed in this pull request?

In ProgressReporter for streams, we use the `committedOffsets` as the 
startOffset and `availableOffsets` as the end offset when reporting the status 
of a trigger in `finishTrigger`. This is a bad pattern that has existed since 
the beginning of ProgressReporter and it is bad because its super hard to 
reason about when `availableOffsets` and `committedOffsets` are updated, and 
when they are recorded. Case in point, this bug silently existed in 
ContinuousExecution, since before MicroBatchExecution was refactored.

The correct fix it to record the offsets explicitly. This PR adds a simple 
method which is explicitly called from MicroBatch/ContinuousExecition before 
updating the `committedOffsets`.

## How was this patch tested?
Added new tests

Author: Tathagata Das 

Closes #21744 from tdas/SPARK-24697.


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

Branch: refs/heads/master
Commit: ff7f6ef75c80633480802d537e66432e3bea4785
Parents: 59c3c23
Author: Tathagata Das 
Authored: Wed Jul 11 12:44:42 2018 -0700
Committer: Tathagata Das 
Committed: Wed Jul 11 12:44:42 2018 -0700

--
 .../streaming/MicroBatchExecution.scala |  3 +++
 .../execution/streaming/ProgressReporter.scala  | 21 
 .../continuous/ContinuousExecution.scala|  3 +++
 .../sql/streaming/StreamingQuerySuite.scala |  6 --
 4 files changed, 27 insertions(+), 6 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/ff7f6ef7/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala
--
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 16651dd..45c43f5 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
@@ -184,6 +184,9 @@ class MicroBatchExecution(
 isCurrentBatchConstructed = 
constructNextBatch(noDataBatchesEnabled)
   }
 
+  // Record the trigger offset range for progress reporting *before* 
processing the batch
+  recordTriggerOffsets(from = committedOffsets, to = availableOffsets)
+
   // Remember whether the current batch has data or not. This will be 
required later
   // for bookkeeping after running the batch, when 
`isNewDataAvailable` will have changed
   // to false as the batch would have already processed the available 
data.

http://git-wip-us.apache.org/repos/asf/spark/blob/ff7f6ef7/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala
index 16ad3ef..47f4b52 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala
@@ -56,8 +56,6 @@ trait ProgressReporter extends Logging {
   protected def logicalPlan: LogicalPlan
   protected def lastExecution: QueryExecution
   protected def newData: Map[BaseStreamingSource, LogicalPlan]
-  protected def availableOffsets: StreamProgress
-  protected def committedOffsets: StreamProgress
   protected def sources: Seq[BaseStreamingSource]
   protected def sink: BaseStreamingSink
   protected def offsetSeqMetadata: OffsetSeqMetadata
@@ -68,8 +66,11 @@ trait ProgressReporter extends Logging {
   // Local timestamps and counters.
   private var currentTriggerStartTimestamp = -1L
   private var currentTriggerEndTimestamp = -1L
+  private var currentTriggerStartOffsets: Map[BaseStreamingSource, String] = _
+  private var currentTriggerEndOffsets: Map[BaseStreamingSource, String] = _
   // TODO: Restore this from the checkpoint when possible.
   private var lastTriggerStartTimestamp = -1L
+
   private val currentDurationsMs = new mutable.HashMap[String, Long]()
 
   /** Flag that signals whether any error with input metrics have already been 
logged */
@@ -114,9 +115,20 @@ trait ProgressReporter extends Logg

spark git commit: [SPARK-24730][SS] Add policy to choose max as global watermark when streaming query has multiple watermarks

2018-07-10 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master 32cb50835 -> 6078b891d


[SPARK-24730][SS] Add policy to choose max as global watermark when streaming 
query has multiple watermarks

## What changes were proposed in this pull request?

Currently, when a streaming query has multiple watermark, the policy is to 
choose the min of them as the global watermark. This is safe to do as the 
global watermark moves with the slowest stream, and is therefore is safe as it 
does not unexpectedly drop some data as late, etc. While this is indeed the 
safe thing to do, in some cases, you may want the watermark to advance with the 
fastest stream, that is, take the max of multiple watermarks. This PR is to add 
that configuration. It makes the following changes.

- Adds a configuration to specify max as the policy.
- Saves the configuration in OffsetSeqMetadata because changing it in the 
middle can lead to unpredictable results.
   - For old checkpoints without the configuration, it assumes the default 
policy as min (irrespective of the policy set at the session where the query is 
being restarted). This is to ensure that existing queries are affected in any 
way.

TODO
- [ ] Add a test for recovery from existing checkpoints.

## How was this patch tested?
New unit test

Author: Tathagata Das 

Closes #21701 from tdas/SPARK-24730.


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

Branch: refs/heads/master
Commit: 6078b891da8fe7fc36579699473168ae7443284c
Parents: 32cb508
Author: Tathagata Das 
Authored: Tue Jul 10 18:03:40 2018 -0700
Committer: Tathagata Das 
Committed: Tue Jul 10 18:03:40 2018 -0700

--
 .../org/apache/spark/sql/internal/SQLConf.scala |  15 ++
 .../streaming/MicroBatchExecution.scala |   4 +-
 .../sql/execution/streaming/OffsetSeq.scala |  37 -
 .../execution/streaming/WatermarkTracker.scala  |  90 ++--
 .../commits/0   |   2 +
 .../commits/1   |   2 +
 .../metadata|   1 +
 .../offsets/0   |   4 +
 .../offsets/1   |   4 +
 .../sql/streaming/EventTimeWatermarkSuite.scala | 136 ++-
 10 files changed, 276 insertions(+), 19 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/6078b891/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
--
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 50965c1..ae56cc9 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
@@ -875,6 +875,21 @@ object SQLConf {
   .stringConf
   
.createWithDefault("org.apache.spark.sql.execution.streaming.ManifestFileCommitProtocol")
 
+  val STREAMING_MULTIPLE_WATERMARK_POLICY =
+buildConf("spark.sql.streaming.multipleWatermarkPolicy")
+  .doc("Policy to calculate the global watermark value when there are 
multiple watermark " +
+"operators in a streaming query. The default value is 'min' which 
chooses " +
+"the minimum watermark reported across multiple operators. Other 
alternative value is" +
+"'max' which chooses the maximum across multiple operators." +
+"Note: This configuration cannot be changed between query restarts 
from the same " +
+"checkpoint location.")
+  .stringConf
+  .checkValue(
+str => Set("min", "max").contains(str.toLowerCase),
+"Invalid value for 'spark.sql.streaming.multipleWatermarkPolicy'. " +
+  "Valid values are 'min' and 'max'")
+  .createWithDefault("min") // must be same as 
MultipleWatermarkPolicy.DEFAULT_POLICY_NAME
+
   val OBJECT_AGG_SORT_BASED_FALLBACK_THRESHOLD =
 buildConf("spark.sql.objectHashAggregate.sortBased.fallbackThreshold")
   .internal()

http://git-wip-us.apache.org/repos/asf/spark/blob/6078b891/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala
--
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 17ffa2a..16651dd 

spark git commit: [SPARK-24662][SQL][SS] Support limit in structured streaming

2018-07-10 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master e0559f238 -> 32cb50835


[SPARK-24662][SQL][SS] Support limit in structured streaming

## What changes were proposed in this pull request?

Support the LIMIT operator in structured streaming.

For streams in append or complete output mode, a stream with a LIMIT operator 
will return no more than the specified number of rows. LIMIT is still 
unsupported for the update output mode.

This change reverts 
https://github.com/apache/spark/commit/e4fee395ecd93ad4579d9afbf0861f82a303e563 
as part of it because it is a better and more complete implementation.

## How was this patch tested?

New and existing unit tests.

Author: Mukul Murthy 

Closes #21662 from mukulmurthy/SPARK-24662.


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

Branch: refs/heads/master
Commit: 32cb50835e7258625afff562939872be002232f2
Parents: e0559f2
Author: Mukul Murthy 
Authored: Tue Jul 10 11:08:04 2018 -0700
Committer: Tathagata Das 
Committed: Tue Jul 10 11:08:04 2018 -0700

--
 .../analysis/UnsupportedOperationChecker.scala  |   6 +-
 .../spark/sql/execution/SparkStrategies.scala   |  26 -
 .../streaming/IncrementalExecution.scala|  11 +-
 .../streaming/StreamingGlobalLimitExec.scala| 102 ++
 .../spark/sql/execution/streaming/memory.scala  |  70 ++--
 .../execution/streaming/sources/memoryV2.scala  |  44 ++--
 .../spark/sql/streaming/DataStreamWriter.scala  |   4 +-
 .../execution/streaming/MemorySinkSuite.scala   |  62 +--
 .../execution/streaming/MemorySinkV2Suite.scala |  80 +-
 .../spark/sql/streaming/StreamSuite.scala   | 108 +++
 .../apache/spark/sql/streaming/StreamTest.scala |   4 +-
 11 files changed, 272 insertions(+), 245 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/32cb5083/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 5ced1ca..f68df5d 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
@@ -315,8 +315,10 @@ object UnsupportedOperationChecker {
 case GroupingSets(_, _, child, _) if child.isStreaming =>
   throwError("GroupingSets is not supported on streaming 
DataFrames/Datasets")
 
-case GlobalLimit(_, _) | LocalLimit(_, _) if 
subPlan.children.forall(_.isStreaming) =>
-  throwError("Limits are not supported on streaming 
DataFrames/Datasets")
+case GlobalLimit(_, _) | LocalLimit(_, _)
+if subPlan.children.forall(_.isStreaming) && outputMode == 
InternalOutputModes.Update =>
+  throwError("Limits are not supported on streaming 
DataFrames/Datasets in Update " +
+"output mode")
 
 case Sort(_, _, _) if !containsCompleteData(subPlan) =>
   throwError("Sorting is not supported on streaming 
DataFrames/Datasets, unless it is on " +

http://git-wip-us.apache.org/repos/asf/spark/blob/32cb5083/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
--
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 cfbcb9a..02e095b 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
@@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.planning._
 import org.apache.spark.sql.catalyst.plans._
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.plans.physical._
+import org.apache.spark.sql.catalyst.streaming.InternalOutputModes
 import org.apache.spark.sql.execution.columnar.{InMemoryRelation, 
InMemoryTableScanExec}
 import org.apache.spark.sql.execution.command._
 import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec
@@ -34,7 +35,7 @@ import org.apache.spark.sql.execution.joins.{BuildLeft, 
BuildRight, BuildSide}
 import org.apache.spark.sql.execution.streaming._
 import org.apache.spark.sql.execution.streaming.sources.MemoryPlanV2
 import 

spark git commit: [SPARK-24386][SS] coalesce(1) aggregates in continuous processing

2018-06-28 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master 2224861f2 -> f6e6899a8


[SPARK-24386][SS] coalesce(1) aggregates in continuous processing

## What changes were proposed in this pull request?

Provide a continuous processing implementation of coalesce(1), as well as 
allowing aggregates on top of it.

The changes in ContinuousQueuedDataReader and such are to use split.index (the 
ID of the partition within the RDD currently being compute()d) rather than 
context.partitionId() (the partition ID of the scheduled task within the Spark 
job - that is, the post coalesce writer). In the absence of a narrow 
dependency, these values were previously always the same, so there was no need 
to distinguish.

## How was this patch tested?

new unit test

Author: Jose Torres 

Closes #21560 from jose-torres/coalesce.


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

Branch: refs/heads/master
Commit: f6e6899a8b8af99cd06e84cae7c69e0fc35bc60a
Parents: 2224861
Author: Jose Torres 
Authored: Thu Jun 28 16:25:40 2018 -0700
Committer: Tathagata Das 
Committed: Thu Jun 28 16:25:40 2018 -0700

--
 .../analysis/UnsupportedOperationChecker.scala  |  11 ++
 .../datasources/v2/DataSourceV2Strategy.scala   |  16 ++-
 .../continuous/ContinuousCoalesceExec.scala |  51 +++
 .../continuous/ContinuousCoalesceRDD.scala  | 136 +++
 .../continuous/ContinuousDataSourceRDD.scala|   7 +-
 .../continuous/ContinuousExecution.scala|   4 +
 .../continuous/ContinuousQueuedDataReader.scala |   6 +-
 .../shuffle/ContinuousShuffleReadRDD.scala  |  10 +-
 .../shuffle/RPCContinuousShuffleReader.scala|   4 +-
 .../sources/ContinuousMemoryStream.scala|  11 +-
 .../continuous/ContinuousAggregationSuite.scala |  63 -
 .../ContinuousQueuedDataReaderSuite.scala   |   2 +-
 .../shuffle/ContinuousShuffleSuite.scala|   7 +-
 13 files changed, 310 insertions(+), 18 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/f6e6899a/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 2bed416..5ced1ca 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
@@ -349,6 +349,17 @@ object UnsupportedOperationChecker {
   _: DeserializeToObject | _: SerializeFromObject | _: 
SubqueryAlias |
   _: TypedFilter) =>
 case node if node.nodeName == "StreamingRelationV2" =>
+case Repartition(1, false, _) =>
+case node: Aggregate =>
+  val aboveSinglePartitionCoalesce = node.find {
+case Repartition(1, false, _) => true
+case _ => false
+  }.isDefined
+
+  if (!aboveSinglePartitionCoalesce) {
+throwError(s"In continuous processing mode, coalesce(1) must be 
called before " +
+  s"aggregate operation ${node.nodeName}.")
+  }
 case node =>
   throwError(s"Continuous processing does not support ${node.nodeName} 
operations.")
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/f6e6899a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
index 182aa29..2a7f1de 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
@@ -22,11 +22,12 @@ import scala.collection.mutable
 import org.apache.spark.sql.{sources, Strategy}
 import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, 
AttributeSet, Expression}
 import org.apache.spark.sql.catalyst.planning.PhysicalOperation
-import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Repartition}
 import org.apache.spark.sql.execution.{FilterExec, ProjectExec, SparkPlan}
 import 

spark git commit: [SPARK-24396][SS][PYSPARK] Add Structured Streaming ForeachWriter for python

2018-06-15 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master 495d8cf09 -> b5ccf0d39


[SPARK-24396][SS][PYSPARK] Add Structured Streaming ForeachWriter for python

## What changes were proposed in this pull request?

This PR adds `foreach` for streaming queries in Python. Users will be able to 
specify their processing logic in two different ways.
- As a function that takes a row as input.
- As an object that has methods `open`, `process`, and `close` methods.

See the python docs in this PR for more details.

## How was this patch tested?
Added java and python unit tests

Author: Tathagata Das 

Closes #21477 from tdas/SPARK-24396.


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

Branch: refs/heads/master
Commit: b5ccf0d3957a444db93893c0ce4417bfbbb11822
Parents: 495d8cf
Author: Tathagata Das 
Authored: Fri Jun 15 12:56:39 2018 -0700
Committer: Tathagata Das 
Committed: Fri Jun 15 12:56:39 2018 -0700

--
 python/pyspark/sql/streaming.py | 162 
 python/pyspark/sql/tests.py | 257 +++
 python/pyspark/tests.py |   4 +-
 .../org/apache/spark/sql/ForeachWriter.scala|  62 -
 .../execution/python/PythonForeachWriter.scala  | 161 
 .../sources/ForeachWriterProvider.scala |  52 +++-
 .../spark/sql/streaming/DataStreamWriter.scala  |  48 +---
 .../python/PythonForeachWriterSuite.scala   | 137 ++
 8 files changed, 811 insertions(+), 72 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/b5ccf0d3/python/pyspark/sql/streaming.py
--
diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py
index fae50b3..4984593 100644
--- a/python/pyspark/sql/streaming.py
+++ b/python/pyspark/sql/streaming.py
@@ -854,6 +854,168 @@ class DataStreamWriter(object):
 self._jwrite = self._jwrite.trigger(jTrigger)
 return self
 
+@since(2.4)
+def foreach(self, f):
+"""
+Sets the output of the streaming query to be processed using the 
provided writer ``f``.
+This is often used to write the output of a streaming query to 
arbitrary storage systems.
+The processing logic can be specified in two ways.
+
+#. A **function** that takes a row as input.
+This is a simple way to express your processing logic. Note that 
this does
+not allow you to deduplicate generated data when failures cause 
reprocessing of
+some input data. That would require you to specify the processing 
logic in the next
+way.
+
+#. An **object** with a ``process`` method and optional ``open`` and 
``close`` methods.
+The object can have the following methods.
+
+* ``open(partition_id, epoch_id)``: *Optional* method that 
initializes the processing
+(for example, open a connection, start a transaction, etc). 
Additionally, you can
+use the `partition_id` and `epoch_id` to deduplicate 
regenerated data
+(discussed later).
+
+* ``process(row)``: *Non-optional* method that processes each 
:class:`Row`.
+
+* ``close(error)``: *Optional* method that finalizes and cleans up 
(for example,
+close connection, commit transaction, etc.) after all rows 
have been processed.
+
+The object will be used by Spark in the following way.
+
+* A single copy of this object is responsible of all the data 
generated by a
+single task in a query. In other words, one instance is 
responsible for
+processing one partition of the data generated in a 
distributed manner.
+
+* This object must be serializable because each task will get a 
fresh
+serialized-deserialized copy of the provided object. Hence, it 
is strongly
+recommended that any initialization for writing data (e.g. 
opening a
+connection or starting a transaction) is done after the 
`open(...)`
+method has been called, which signifies that the task is ready 
to generate data.
+
+* The lifecycle of the methods are as follows.
+
+For each partition with ``partition_id``:
+
+... For each batch/epoch of streaming data with ``epoch_id``:
+
+... Method ``open(partitionId, epochId)`` is called.
+
+... If ``open(...)`` returns true, for each row in the 
partition and
+batch/epoch, method ``proces

spark git commit: [SPARK-24453][SS] Fix error recovering from the failure in a no-data batch

2018-06-05 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master e8c1a0c2f -> 2c2a86b5d


[SPARK-24453][SS] Fix error recovering from the failure in a no-data batch

## What changes were proposed in this pull request?

The error occurs when we are recovering from a failure in a no-data batch (say 
X) that has been planned (i.e. written to offset log) but not executed (i.e. 
not written to commit log). Upon recovery the following sequence of events 
happen.

1. `MicroBatchExecution.populateStartOffsets` sets `currentBatchId` to X. Since 
there was no data in the batch, the `availableOffsets` is same as 
`committedOffsets`, so `isNewDataAvailable` is `false`.
2. When `MicroBatchExecution.constructNextBatch` is called, ideally it should 
immediately return true because the next batch has already been constructed. 
However, the check of whether the batch has been constructed was `if 
(isNewDataAvailable) return true`. Since the planned batch is a no-data batch, 
it escaped this check and proceeded to plan the same batch X *once again*.

The solution is to have an explicit flag that signifies whether a batch has 
already been constructed or not. `populateStartOffsets` is going to set the 
flag appropriately.

## How was this patch tested?

new unit test

Author: Tathagata Das 

Closes #21491 from tdas/SPARK-24453.


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

Branch: refs/heads/master
Commit: 2c2a86b5d5be6f77ee72d16f990b39ae59f479b9
Parents: e8c1a0c
Author: Tathagata Das 
Authored: Tue Jun 5 01:08:55 2018 -0700
Committer: Tathagata Das 
Committed: Tue Jun 5 01:08:55 2018 -0700

--
 .../streaming/MicroBatchExecution.scala | 38 +++
 .../streaming/MicroBatchExecutionSuite.scala| 71 
 .../apache/spark/sql/streaming/StreamTest.scala |  2 +-
 3 files changed, 98 insertions(+), 13 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/2c2a86b5/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala
--
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 7817360..17ffa2a 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
@@ -127,6 +127,12 @@ class MicroBatchExecution(
   }
 
   /**
+   * Signifies whether current batch (i.e. for the batch `currentBatchId`) has 
been constructed
+   * (i.e. written to the offsetLog) and is ready for execution.
+   */
+  private var isCurrentBatchConstructed = false
+
+  /**
* Signals to the thread executing micro-batches that it should stop running 
after the next
* batch. This method blocks until the thread stops running.
*/
@@ -154,7 +160,6 @@ class MicroBatchExecution(
 
 triggerExecutor.execute(() => {
   if (isActive) {
-var currentBatchIsRunnable = false // Whether the current batch is 
runnable / has been run
 var currentBatchHasNewData = false // Whether the current batch had 
new data
 
 startTrigger()
@@ -175,7 +180,9 @@ class MicroBatchExecution(
   // new data to process as `constructNextBatch` may decide to run a 
batch for
   // state cleanup, etc. `isNewDataAvailable` will be updated to 
reflect whether new data
   // is available or not.
-  currentBatchIsRunnable = constructNextBatch(noDataBatchesEnabled)
+  if (!isCurrentBatchConstructed) {
+isCurrentBatchConstructed = 
constructNextBatch(noDataBatchesEnabled)
+  }
 
   // Remember whether the current batch has data or not. This will be 
required later
   // for bookkeeping after running the batch, when 
`isNewDataAvailable` will have changed
@@ -183,7 +190,7 @@ class MicroBatchExecution(
   currentBatchHasNewData = isNewDataAvailable
 
   currentStatus = currentStatus.copy(isDataAvailable = 
isNewDataAvailable)
-  if (currentBatchIsRunnable) {
+  if (isCurrentBatchConstructed) {
 if (currentBatchHasNewData) updateStatusMessage("Processing new 
data")
 else updateStatusMessage("No new data but cleaning up state")
 runBatch(sparkSessionForStream)
@@ -194,9 +201,12 @@ class MicroBatchExecution(
 
 finishTrigger(currentBatchHasNewData)  // Must be outside 
reportTimeTaken so it is recorded
 
-// If the curre

spark git commit: [SPARK-24397][PYSPARK] Added TaskContext.getLocalProperty(key) in Python

2018-05-31 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master 7a82e93b3 -> 223df5d9d


[SPARK-24397][PYSPARK] Added TaskContext.getLocalProperty(key) in Python

## What changes were proposed in this pull request?

This adds a new API `TaskContext.getLocalProperty(key)` to the Python 
TaskContext. It mirrors the Java TaskContext API of returning a string value if 
the key exists, or None if the key does not exist.

## How was this patch tested?
New test added.

Author: Tathagata Das 

Closes #21437 from tdas/SPARK-24397.


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

Branch: refs/heads/master
Commit: 223df5d9d4fbf48db017edb41f9b7e4033679f35
Parents: 7a82e93
Author: Tathagata Das 
Authored: Thu May 31 11:23:57 2018 -0700
Committer: Tathagata Das 
Committed: Thu May 31 11:23:57 2018 -0700

--
 .../org/apache/spark/api/python/PythonRunner.scala|  7 +++
 python/pyspark/taskcontext.py |  7 +++
 python/pyspark/tests.py   | 14 ++
 python/pyspark/worker.py  |  6 ++
 4 files changed, 34 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/223df5d9/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala
--
diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala 
b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala
index f075a7e..41eac10 100644
--- a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala
+++ b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala
@@ -183,6 +183,13 @@ private[spark] abstract class BasePythonRunner[IN, OUT](
 dataOut.writeInt(context.partitionId())
 dataOut.writeInt(context.attemptNumber())
 dataOut.writeLong(context.taskAttemptId())
+val localProps = 
context.asInstanceOf[TaskContextImpl].getLocalProperties.asScala
+dataOut.writeInt(localProps.size)
+localProps.foreach { case (k, v) =>
+  PythonRDD.writeUTF(k, dataOut)
+  PythonRDD.writeUTF(v, dataOut)
+}
+
 // sparkFilesDir
 PythonRDD.writeUTF(SparkFiles.getRootDirectory(), dataOut)
 // Python includes (*.zip and *.egg files)

http://git-wip-us.apache.org/repos/asf/spark/blob/223df5d9/python/pyspark/taskcontext.py
--
diff --git a/python/pyspark/taskcontext.py b/python/pyspark/taskcontext.py
index e5218d9..63ae1f3 100644
--- a/python/pyspark/taskcontext.py
+++ b/python/pyspark/taskcontext.py
@@ -34,6 +34,7 @@ class TaskContext(object):
 _partitionId = None
 _stageId = None
 _taskAttemptId = None
+_localProperties = None
 
 def __new__(cls):
 """Even if users construct TaskContext instead of using get, give them 
the singleton."""
@@ -88,3 +89,9 @@ class TaskContext(object):
 TaskAttemptID.
 """
 return self._taskAttemptId
+
+def getLocalProperty(self, key):
+"""
+Get a local property set upstream in the driver, or None if it is 
missing.
+"""
+return self._localProperties.get(key, None)

http://git-wip-us.apache.org/repos/asf/spark/blob/223df5d9/python/pyspark/tests.py
--
diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py
index 3b37cc0..30723b8 100644
--- a/python/pyspark/tests.py
+++ b/python/pyspark/tests.py
@@ -574,6 +574,20 @@ class TaskContextTests(PySparkTestCase):
 tc = TaskContext.get()
 self.assertTrue(tc is None)
 
+def test_get_local_property(self):
+"""Verify that local properties set on the driver are available in 
TaskContext."""
+key = "testkey"
+value = "testvalue"
+self.sc.setLocalProperty(key, value)
+try:
+rdd = self.sc.parallelize(range(1), 1)
+prop1 = rdd.map(lambda x: 
TaskContext.get().getLocalProperty(key)).collect()[0]
+self.assertEqual(prop1, value)
+prop2 = rdd.map(lambda x: 
TaskContext.get().getLocalProperty("otherkey")).collect()[0]
+self.assertTrue(prop2 is None)
+finally:
+self.sc.setLocalProperty(key, None)
+
 
 class RDDTests(ReusedPySparkTestCase):
 

http://git-wip-us.apache.org/repos/asf/spark/blob/223df5d9/python/pyspark/worker.py
-

spark git commit: [SPARK-24234][SS] Support multiple row writers in continuous processing shuffle reader.

2018-05-24 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master 53c06ddab -> 0fd68cb72


[SPARK-24234][SS] Support multiple row writers in continuous processing shuffle 
reader.

## What changes were proposed in this pull request?

https://docs.google.com/document/d/1IL4kJoKrZWeyIhklKUJqsW-yEN7V7aL05MmM65AYOfE/edit#heading=h.8t3ci57f7uii

Support multiple different row writers in continuous processing shuffle reader.

Note that having multiple read-side buffers ended up being the natural way to 
do this. Otherwise it's hard to express the constraint of sending an epoch 
marker only when all writers have sent one.

## How was this patch tested?

new unit tests

Author: Jose Torres 

Closes #21385 from jose-torres/multipleWrite.


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

Branch: refs/heads/master
Commit: 0fd68cb7278e5fdf106e73b580ee7dd829006386
Parents: 53c06dd
Author: Jose Torres 
Authored: Thu May 24 17:08:52 2018 -0700
Committer: Tathagata Das 
Committed: Thu May 24 17:08:52 2018 -0700

--
 .../shuffle/ContinuousShuffleReadRDD.scala  |  21 ++-
 .../continuous/shuffle/UnsafeRowReceiver.scala  |  87 --
 .../shuffle/ContinuousShuffleReadSuite.scala| 163 +++
 3 files changed, 227 insertions(+), 44 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/0fd68cb7/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 270b1a5..801b28b 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
@@ -25,11 +25,16 @@ import org.apache.spark.sql.catalyst.expressions.UnsafeRow
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.util.NextIterator
 
-case class ContinuousShuffleReadPartition(index: Int, queueSize: Int) extends 
Partition {
+case class ContinuousShuffleReadPartition(
+  index: Int,
+  queueSize: Int,
+  numShuffleWriters: Int,
+  epochIntervalMs: Long)
+extends Partition {
   // 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, env)
+val receiver = new UnsafeRowReceiver(queueSize, numShuffleWriters, 
epochIntervalMs, env)
 val endpoint = 
env.setupEndpoint(s"UnsafeRowReceiver-${UUID.randomUUID()}", receiver)
 TaskContext.get().addTaskCompletionListener { ctx =>
   env.stop(endpoint)
@@ -42,16 +47,24 @@ case class ContinuousShuffleReadPartition(index: Int, 
queueSize: Int) extends Pa
  * RDD at the map side of each continuous processing shuffle task. Upstream 
tasks send their
  * shuffle output to the wrapped receivers in partitions of this RDD; each of 
the RDD's tasks
  * poll from their receiver until an epoch marker is sent.
+ *
+ * @param sc the RDD context
+ * @param numPartitions the number of read partitions for this RDD
+ * @param queueSize the size of the row buffers to use
+ * @param numShuffleWriters the number of continuous shuffle writers feeding 
into this RDD
+ * @param epochIntervalMs the checkpoint interval of the streaming query
  */
 class ContinuousShuffleReadRDD(
 sc: SparkContext,
 numPartitions: Int,
-queueSize: Int = 1024)
+queueSize: Int = 1024,
+numShuffleWriters: Int = 1,
+epochIntervalMs: Long = 1000)
   extends RDD[UnsafeRow](sc, Nil) {
 
   override protected def getPartitions: Array[Partition] = {
 (0 until numPartitions).map { partIndex =>
-  ContinuousShuffleReadPartition(partIndex, queueSize)
+  ContinuousShuffleReadPartition(partIndex, queueSize, numShuffleWriters, 
epochIntervalMs)
 }.toArray
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/0fd68cb7/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/shuffle/UnsafeRowReceiver.scala
--
diff --git 

spark git commit: [SPARK-23416][SS] Add a specific stop method for ContinuousExecution.

2018-05-23 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master b7a036b75 -> f45793329


[SPARK-23416][SS] Add a specific stop method for ContinuousExecution.

## What changes were proposed in this pull request?

Add a specific stop method for ContinuousExecution. The previous 
StreamExecution.stop() method had a race condition as applied to continuous 
processing: if the cancellation was round-tripped to the driver too quickly, 
the generic SparkException it caused would be reported as the query death 
cause. We earlier decided that SparkException should not be added to the 
StreamExecution.isInterruptionException() whitelist, so we need to ensure this 
never happens instead.

## How was this patch tested?

Existing tests. I could consistently reproduce the previous flakiness by 
putting Thread.sleep(1000) between the first job cancellation and thread 
interruption in StreamExecution.stop().

Author: Jose Torres 

Closes #21384 from jose-torres/fixKafka.


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

Branch: refs/heads/master
Commit: f4579332931c9bf424d0b6147fad89bd63da26f6
Parents: b7a036b
Author: Jose Torres 
Authored: Wed May 23 17:21:29 2018 -0700
Committer: Tathagata Das 
Committed: Wed May 23 17:21:29 2018 -0700

--
 .../execution/streaming/MicroBatchExecution.scala | 18 ++
 .../sql/execution/streaming/StreamExecution.scala | 18 --
 .../continuous/ContinuousExecution.scala  | 16 
 3 files changed, 34 insertions(+), 18 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/f4579332/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala
--
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 6709e70..7817360 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
@@ -127,6 +127,24 @@ class MicroBatchExecution(
   }
 
   /**
+   * Signals to the thread executing micro-batches that it should stop running 
after the next
+   * batch. This method blocks until the thread stops running.
+   */
+  override def stop(): Unit = {
+// Set the state to TERMINATED so that the batching thread knows that it 
was interrupted
+// intentionally
+state.set(TERMINATED)
+if (queryExecutionThread.isAlive) {
+  sparkSession.sparkContext.cancelJobGroup(runId.toString)
+  queryExecutionThread.interrupt()
+  queryExecutionThread.join()
+  // microBatchThread may spawn new jobs, so we need to cancel again to 
prevent a leak
+  sparkSession.sparkContext.cancelJobGroup(runId.toString)
+}
+logInfo(s"Query $prettyIdString was stopped")
+  }
+
+  /**
* Repeatedly attempts to run batches as data arrives.
*/
   protected def runActivatedStream(sparkSessionForStream: SparkSession): Unit 
= {

http://git-wip-us.apache.org/repos/asf/spark/blob/f4579332/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 3fc8c78..290de87 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
@@ -379,24 +379,6 @@ abstract class StreamExecution(
   }
 
   /**
-   * Signals to the thread executing micro-batches that it should stop running 
after the next
-   * batch. This method blocks until the thread stops running.
-   */
-  override def stop(): Unit = {
-// Set the state to TERMINATED so that the batching thread knows that it 
was interrupted
-// intentionally
-state.set(TERMINATED)
-if (queryExecutionThread.isAlive) {
-  sparkSession.sparkContext.cancelJobGroup(runId.toString)
-  queryExecutionThread.interrupt()
-  queryExecutionThread.join()
-  // microBatchThread may spawn new jobs, so we need to cancel again to 
prevent a leak
-  sparkSession.sparkContext.cancelJobGroup(runId.toString)
-}
-logInfo(s"Query $prettyIdString was 

spark git commit: [SPARK-24234][SS] Reader for continuous processing shuffle

2018-05-21 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master 03e90f65b -> a33dcf4a0


[SPARK-24234][SS] Reader for continuous processing shuffle

## What changes were proposed in this pull request?

Read RDD for continuous processing shuffle, as well as the initial RPC-based 
row receiver.

https://docs.google.com/document/d/1IL4kJoKrZWeyIhklKUJqsW-yEN7V7aL05MmM65AYOfE/edit#heading=h.8t3ci57f7uii

## How was this patch tested?

new unit tests

Author: Jose Torres 

Closes #21337 from jose-torres/readerRddMaster.


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

Branch: refs/heads/master
Commit: a33dcf4a0bbe20dce6f1e1e6c2e1c3828291fb3d
Parents: 03e90f6
Author: Jose Torres 
Authored: Mon May 21 12:58:05 2018 -0700
Committer: Tathagata Das 
Committed: Mon May 21 12:58:05 2018 -0700

--
 .../shuffle/ContinuousShuffleReadRDD.scala  |  61 ++
 .../shuffle/ContinuousShuffleReader.scala   |  32 
 .../continuous/shuffle/UnsafeRowReceiver.scala  |  75 
 .../shuffle/ContinuousShuffleReadSuite.scala| 184 +++
 4 files changed, 352 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/a33dcf4a/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
new file mode 100644
index 000..270b1a5
--- /dev/null
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/shuffle/ContinuousShuffleReadRDD.scala
@@ -0,0 +1,61 @@
+/*
+ * 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 java.util.UUID
+
+import org.apache.spark.{Partition, SparkContext, SparkEnv, TaskContext}
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.expressions.UnsafeRow
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.util.NextIterator
+
+case class ContinuousShuffleReadPartition(index: Int, queueSize: Int) extends 
Partition {
+  // 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, env)
+val endpoint = 
env.setupEndpoint(s"UnsafeRowReceiver-${UUID.randomUUID()}", receiver)
+TaskContext.get().addTaskCompletionListener { ctx =>
+  env.stop(endpoint)
+}
+(receiver, endpoint)
+  }
+}
+
+/**
+ * RDD at the map side of each continuous processing shuffle task. Upstream 
tasks send their
+ * shuffle output to the wrapped receivers in partitions of this RDD; each of 
the RDD's tasks
+ * poll from their receiver until an epoch marker is sent.
+ */
+class ContinuousShuffleReadRDD(
+sc: SparkContext,
+numPartitions: Int,
+queueSize: Int = 1024)
+  extends RDD[UnsafeRow](sc, Nil) {
+
+  override protected def getPartitions: Array[Partition] = {
+(0 until numPartitions).map { partIndex =>
+  ContinuousShuffleReadPartition(partIndex, queueSize)
+}.toArray
+  }
+
+  override def compute(split: Partition, context: TaskContext): 
Iterator[UnsafeRow] = {
+split.asInstanceOf[ContinuousShuffleReadPartition].reader.read()
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/a33dcf4a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/shuffle/ContinuousShuffleReader.scala
--
diff --git 

spark git commit: [SPARK-23503][SS] Enforce sequencing of committed epochs for Continuous Execution

2018-05-18 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master 710e4e81a -> 434d74e33


[SPARK-23503][SS] Enforce sequencing of committed epochs for Continuous 
Execution

## What changes were proposed in this pull request?

Made changes to EpochCoordinator so that it enforces a commit order. In case a 
message for epoch n is lost and epoch (n + 1) is ready for commit before epoch 
n is, epoch (n + 1) will wait for epoch n to be committed first.

## How was this patch tested?

Existing tests in ContinuousSuite and EpochCoordinatorSuite.

Author: Efim Poberezkin 

Closes #20936 from efimpoberezkin/pr/sequence-commited-epochs.


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

Branch: refs/heads/master
Commit: 434d74e337465d77fa49ab65e2b5461e5ff7b5c7
Parents: 710e4e8
Author: Efim Poberezkin 
Authored: Fri May 18 16:54:39 2018 -0700
Committer: Tathagata Das 
Committed: Fri May 18 16:54:39 2018 -0700

--
 .../streaming/continuous/EpochCoordinator.scala | 69 
 .../continuous/EpochCoordinatorSuite.scala  |  6 +-
 2 files changed, 58 insertions(+), 17 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/434d74e3/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/EpochCoordinator.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/EpochCoordinator.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/EpochCoordinator.scala
index cc68080..8877ebe 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/EpochCoordinator.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/EpochCoordinator.scala
@@ -137,30 +137,71 @@ private[continuous] class EpochCoordinator(
   private val partitionOffsets =
 mutable.Map[(Long, Int), PartitionOffset]()
 
+  private var lastCommittedEpoch = startEpoch - 1
+  // Remembers epochs that have to wait for previous epochs to be committed 
first.
+  private val epochsWaitingToBeCommitted = mutable.HashSet.empty[Long]
+
   private def resolveCommitsAtEpoch(epoch: Long) = {
-val thisEpochCommits =
-  partitionCommits.collect { case ((e, _), msg) if e == epoch => msg }
+val thisEpochCommits = findPartitionCommitsForEpoch(epoch)
 val nextEpochOffsets =
   partitionOffsets.collect { case ((e, _), o) if e == epoch => o }
 
 if (thisEpochCommits.size == numWriterPartitions &&
   nextEpochOffsets.size == numReaderPartitions) {
-  logDebug(s"Epoch $epoch has received commits from all partitions. 
Committing globally.")
-  // Sequencing is important here. We must commit to the writer before 
recording the commit
-  // in the query, or we will end up dropping the commit if we restart in 
the middle.
-  writer.commit(epoch, thisEpochCommits.toArray)
-  query.commit(epoch)
-
-  // Cleanup state from before this epoch, now that we know all partitions 
are forever past it.
-  for (k <- partitionCommits.keys.filter { case (e, _) => e < epoch }) {
-partitionCommits.remove(k)
-  }
-  for (k <- partitionOffsets.keys.filter { case (e, _) => e < epoch }) {
-partitionOffsets.remove(k)
+
+  // Check that last committed epoch is the previous one for sequencing of 
committed epochs.
+  // If not, add the epoch being currently processed to epochs waiting to 
be committed,
+  // otherwise commit it.
+  if (lastCommittedEpoch != epoch - 1) {
+logDebug(s"Epoch $epoch has received commits from all partitions " +
+  s"and is waiting for epoch ${epoch - 1} to be committed first.")
+epochsWaitingToBeCommitted.add(epoch)
+  } else {
+commitEpoch(epoch, thisEpochCommits)
+lastCommittedEpoch = epoch
+
+// Commit subsequent epochs that are waiting to be committed.
+var nextEpoch = lastCommittedEpoch + 1
+while (epochsWaitingToBeCommitted.contains(nextEpoch)) {
+  val nextEpochCommits = findPartitionCommitsForEpoch(nextEpoch)
+  commitEpoch(nextEpoch, nextEpochCommits)
+
+  epochsWaitingToBeCommitted.remove(nextEpoch)
+  lastCommittedEpoch = nextEpoch
+  nextEpoch += 1
+}
+
+// Cleanup state from before last committed epoch,
+// now that we know all partitions are forever past it.
+for (k <- partitionCommits.keys.filter { case (e, _) => e < 
lastCommittedEpoch }) {
+  partitionCommits.remove(k)
+}
+   

spark git commit: [SPARK-24158][SS] Enable no-data batches for streaming joins

2018-05-16 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master 8e60a16b7 -> 991726f31


[SPARK-24158][SS] Enable no-data batches for streaming joins

## What changes were proposed in this pull request?

This is a continuation of the larger task of enabling zero-data batches for 
more eager state cleanup. This PR enables it for stream-stream joins.

## How was this patch tested?
- Updated join tests. Additionally, updated them to not use `CheckLastBatch` 
anywhere to set good precedence for future.

Author: Tathagata Das <tathagata.das1...@gmail.com>

Closes #21253 from tdas/SPARK-24158.


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

Branch: refs/heads/master
Commit: 991726f31a8d182ed6d5b0e59185d97c0c5c532f
Parents: 8e60a16
Author: Tathagata Das <tathagata.das1...@gmail.com>
Authored: Wed May 16 14:55:02 2018 -0700
Committer: Tathagata Das <tathagata.das1...@gmail.com>
Committed: Wed May 16 14:55:02 2018 -0700

--
 .../spark/sql/execution/SparkStrategies.scala   |   2 +-
 .../StreamingSymmetricHashJoinExec.scala|  14 +-
 .../apache/spark/sql/streaming/StreamTest.scala |  15 +-
 .../sql/streaming/StreamingJoinSuite.scala  | 217 ++-
 4 files changed, 130 insertions(+), 118 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/991726f3/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
--
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 82b4eb9..37a0b9d 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
@@ -361,7 +361,7 @@ abstract class SparkStrategies extends 
QueryPlanner[SparkPlan] {
 
 case Join(left, right, _, _) if left.isStreaming && right.isStreaming 
=>
   throw new AnalysisException(
-"Stream stream joins without equality predicate is not supported", 
plan = Some(plan))
+"Stream-stream join without equality predicate is not supported", 
plan = Some(plan))
 
 case _ => Nil
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/991726f3/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala
index fa7c8ee..afa664e 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala
@@ -187,6 +187,17 @@ case class StreamingSymmetricHashJoinExec(
 s"${getClass.getSimpleName} should not take $x as the JoinType")
   }
 
+  override def shouldRunAnotherBatch(newMetadata: OffsetSeqMetadata): Boolean 
= {
+val watermarkUsedForStateCleanup =
+  stateWatermarkPredicates.left.nonEmpty || 
stateWatermarkPredicates.right.nonEmpty
+
+// Latest watermark value is more than that used in this previous executed 
plan
+val watermarkHasChanged =
+  eventTimeWatermark.isDefined && newMetadata.batchWatermarkMs > 
eventTimeWatermark.get
+
+watermarkUsedForStateCleanup && watermarkHasChanged
+  }
+
   protected override def doExecute(): RDD[InternalRow] = {
 val stateStoreCoord = 
sqlContext.sessionState.streamingQueryManager.stateStoreCoordinator
 val stateStoreNames = 
SymmetricHashJoinStateManager.allStateStoreNames(LeftSide, RightSide)
@@ -319,8 +330,7 @@ case class StreamingSymmetricHashJoinExec(
 // outer join) if possible. In all cases, nothing needs to be 
outputted, hence the removal
 // needs to be done greedily by immediately consuming the returned 
iterator.
 val cleanupIter = joinType match {
-  case Inner =>
-leftSideJoiner.removeOldState() ++ rightSideJoiner.removeOldState()
+  case Inner => leftSideJoiner.removeOldState() ++ 
rightSideJoiner.removeOldState()
   case LeftOuter => rightSideJoiner.removeOldState()
   case RightOuter => leftSideJoiner.removeOldState()
   case _ => throwBadJoinTypeException()

http://git-wip-us.apache.org/re

spark git commit: [SPARK-24157][SS] Enabled no-data batches in MicroBatchExecution for streaming aggregation and deduplication.

2018-05-04 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master af4dc5028 -> 47b5b6852


[SPARK-24157][SS] Enabled no-data batches in MicroBatchExecution for streaming 
aggregation and deduplication.

## What changes were proposed in this pull request?

This PR enables the MicroBatchExecution to run no-data batches if some 
SparkPlan requires running another batch to output results based on updated 
watermark / processing time. In this PR, I have enabled streaming aggregations 
and streaming deduplicates to automatically run addition batch even if new data 
is available. See https://issues.apache.org/jira/browse/SPARK-24156 for more 
context.

Major changes/refactoring done in this PR.
- Refactoring MicroBatchExecution - A major point of confusion in 
MicroBatchExecution control flow was always (at least to me) was that 
`populateStartOffsets` internally called `constructNextBatch` which was not 
obvious from just the name "populateStartOffsets" and made the control flow 
from the main trigger execution loop very confusing (main loop in 
`runActivatedStream` called `constructNextBatch` but only if 
`populateStartOffsets` hadn't already called it). Instead, the refactoring 
makes it cleaner.
- `populateStartOffsets` only the updates `availableOffsets` and 
`committedOffsets`. Does not call `constructNextBatch`.
- Main loop in `runActivatedStream` calls `constructNextBatch` which 
returns true or false reflecting whether the next batch is ready for executing. 
This method is now idempotent; if a batch has already been constructed, then it 
will always return true until the batch has been executed.
- If next batch is ready then we call `runBatch` or sleep.
- That's it.

- Refactoring watermark management logic - This has been refactored out from 
`MicroBatchExecution` in a separate class to simplify `MicroBatchExecution`.

- New method `shouldRunAnotherBatch` in `IncrementalExecution` - This returns 
true if there is any stateful operation in the last execution plan that 
requires another batch for state cleanup, etc. This is used to decide whether 
to construct a batch or not in `constructNextBatch`.

- Changes to stream testing framework - Many tests used CheckLastBatch to 
validate answers. This assumed that there will be no more batches after the 
last set of input has been processed, so the last batch is the one that has 
output corresponding to the last input. This is not true anymore. To account 
for that, I made two changes.
- `CheckNewAnswer` is a new test action that verifies the new rows 
generated since the last time the answer was checked by `CheckAnswer`, 
`CheckNewAnswer` or `CheckLastBatch`. This is agnostic to how many batches 
occurred between the last check and now. To do make this easier, I added a 
common trait between MemorySink and MemorySinkV2 to abstract out some common 
methods.
- `assertNumStateRows` has been updated in the same way to be agnostic to 
batches while checking what the total rows and how many state rows were updated 
(sums up updates since the last check).

## How was this patch tested?
- Changes made to existing tests - Tests have been changed in one of the 
following patterns.
- Tests where the last input was given again to force another batch to be 
executed and state cleaned up / output generated, they were simplified by 
removing the extra input.
- Tests using aggregation+watermark where CheckLastBatch were replaced with 
CheckNewAnswer to make them batch agnostic.
- New tests added to check whether the flag works for streaming aggregation and 
deduplication

Author: Tathagata Das <tathagata.das1...@gmail.com>

Closes #21220 from tdas/SPARK-24157.


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

Branch: refs/heads/master
Commit: 47b5b68528c154d32b3f40f388918836d29462b8
Parents: af4dc50
Author: Tathagata Das <tathagata.das1...@gmail.com>
Authored: Fri May 4 16:35:24 2018 -0700
Committer: Tathagata Das <tathagata.das1...@gmail.com>
Committed: Fri May 4 16:35:24 2018 -0700

--
 .../org/apache/spark/sql/internal/SQLConf.scala |  11 +
 .../streaming/IncrementalExecution.scala|  10 +
 .../streaming/MicroBatchExecution.scala | 231 +++
 .../execution/streaming/WatermarkTracker.scala  |  73 +
 .../spark/sql/execution/streaming/memory.scala  |  17 +-
 .../execution/streaming/sources/memoryV2.scala  |   8 +-
 .../execution/streaming/statefulOperators.scala |  16 +
 .../streaming/sources/ForeachWriterSuite.scala  |   8 +-
 .../spark/sql/streaming/DeduplicateSuite.scala  | 285 --
 .../sql/streaming/EventTimeWatermarkSuite.scala | 112 +++
 .../sql/streaming/FileStreamSinkSuite.scala 

spark git commit: [SPARK-24039][SS] Do continuous processing writes with multiple compute() calls

2018-05-04 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master d04806a23 -> af4dc5028


[SPARK-24039][SS] Do continuous processing writes with multiple compute() calls

## What changes were proposed in this pull request?

Do continuous processing writes with multiple compute() calls.

The current strategy (before this PR) is hacky; we just call next() on an 
iterator which has already returned hasNext = false, knowing that all the nodes 
we whitelist handle this properly. This will have to be changed before we can 
support more complex query plans. (In particular, I have a WIP 
https://github.com/jose-torres/spark/pull/13 which should be able to support 
aggregates in a single partition with minimal additional work.)

Most of the changes here are just refactoring to accommodate the new model. The 
behavioral changes are:

* The writer now calls prev.compute(split, context) once per epoch within the 
epoch loop.
* ContinuousDataSourceRDD now spawns a ContinuousQueuedDataReader which is 
shared across multiple calls to compute() for the same partition.

## How was this patch tested?

existing unit tests

Author: Jose Torres 

Closes #21200 from jose-torres/noAggr.


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

Branch: refs/heads/master
Commit: af4dc50280ffcdeda208ef2dc5f8b843389732e5
Parents: d04806a
Author: Jose Torres 
Authored: Fri May 4 14:14:40 2018 -0700
Committer: Tathagata Das 
Committed: Fri May 4 14:14:40 2018 -0700

--
 .../datasources/v2/DataSourceV2ScanExec.scala   |   6 +-
 .../continuous/ContinuousDataSourceRDD.scala| 114 ++
 .../ContinuousDataSourceRDDIter.scala   | 222 ---
 .../continuous/ContinuousQueuedDataReader.scala | 211 ++
 .../continuous/ContinuousWriteRDD.scala |  90 
 .../WriteToContinuousDataSourceExec.scala   |  57 +
 .../ContinuousQueuedDataReaderSuite.scala   | 167 ++
 7 files changed, 592 insertions(+), 275 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/af4dc502/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala
index 41bdda4..77cb707 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala
@@ -96,7 +96,11 @@ case class DataSourceV2ScanExec(
   
sparkContext.getLocalProperty(ContinuousExecution.EPOCH_COORDINATOR_ID_KEY),
   sparkContext.env)
 .askSync[Unit](SetReaderPartitions(readerFactories.size))
-  new ContinuousDataSourceRDD(sparkContext, sqlContext, readerFactories)
+  new ContinuousDataSourceRDD(
+sparkContext,
+sqlContext.conf.continuousStreamingExecutorQueueSize,
+sqlContext.conf.continuousStreamingExecutorPollIntervalMs,
+readerFactories)
 .asInstanceOf[RDD[InternalRow]]
 
 case r: SupportsScanColumnarBatch if r.enableBatchRead() =>

http://git-wip-us.apache.org/repos/asf/spark/blob/af4dc502/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousDataSourceRDD.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousDataSourceRDD.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousDataSourceRDD.scala
new file mode 100644
index 000..0a3b9dc
--- /dev/null
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousDataSourceRDD.scala
@@ -0,0 +1,114 @@
+/*
+ * 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 

spark git commit: [SPARK-24094][SS][MINOR] Change description strings of v2 streaming sources to reflect the change

2018-04-26 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master ffaf0f9fd -> d1eb8d3dd


[SPARK-24094][SS][MINOR] Change description strings of v2 streaming sources to 
reflect the change

## What changes were proposed in this pull request?

This makes it easy to understand at runtime which version is running. Great for 
debugging production issues.

## How was this patch tested?
Not necessary.

Author: Tathagata Das <tathagata.das1...@gmail.com>

Closes #21160 from tdas/SPARK-24094.


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

Branch: refs/heads/master
Commit: d1eb8d3ddc877958512194cc8f5dd8119b41bed0
Parents: ffaf0f9
Author: Tathagata Das <tathagata.das1...@gmail.com>
Authored: Wed Apr 25 23:24:05 2018 -0700
Committer: Tathagata Das <tathagata.das1...@gmail.com>
Committed: Wed Apr 25 23:24:05 2018 -0700

--
 .../org/apache/spark/sql/kafka010/KafkaMicroBatchReader.scala  | 2 +-
 .../execution/streaming/sources/RateStreamMicroBatchReader.scala   | 2 +-
 .../org/apache/spark/sql/execution/streaming/sources/socket.scala  | 2 +-
 3 files changed, 3 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/d1eb8d3d/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 2ed49ba..cbe655f 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
@@ -169,7 +169,7 @@ private[kafka010] class KafkaMicroBatchReader(
 kafkaOffsetReader.close()
   }
 
-  override def toString(): String = s"Kafka[$kafkaOffsetReader]"
+  override def toString(): String = s"KafkaV2[$kafkaOffsetReader]"
 
   /**
* Read initial partition offsets from the checkpoint, or decide the offsets 
and write them to

http://git-wip-us.apache.org/repos/asf/spark/blob/d1eb8d3d/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamMicroBatchReader.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamMicroBatchReader.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamMicroBatchReader.scala
index 6cf8520..f54291b 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamMicroBatchReader.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamMicroBatchReader.scala
@@ -177,7 +177,7 @@ class RateStreamMicroBatchReader(options: 
DataSourceOptions, checkpointLocation:
 
   override def stop(): Unit = {}
 
-  override def toString: String = 
s"MicroBatchRateSource[rowsPerSecond=$rowsPerSecond, " +
+  override def toString: String = s"RateStreamV2[rowsPerSecond=$rowsPerSecond, 
" +
 s"rampUpTimeSeconds=$rampUpTimeSeconds, " +
 s"numPartitions=${options.get(NUM_PARTITIONS).orElse("default")}"
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/d1eb8d3d/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/socket.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/socket.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/socket.scala
index 5aae46b..90f4a5b 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/socket.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/socket.scala
@@ -214,7 +214,7 @@ class TextSocketMicroBatchReader(options: 
DataSourceOptions) extends MicroBatchR
 }
   }
 
-  override def toString: String = s"TextSocket[host: $host, port: $port]"
+  override def toString: String = s"TextSocketV2[host: $host, port: $port]"
 }
 
 class TextSocketSourceProvider extends DataSourceV2


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



spark git commit: [SPARK-24050][SS] Calculate input / processing rates correctly for DataSourceV2 streaming sources

2018-04-25 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master 20ca208bc -> 396938ef0


[SPARK-24050][SS] Calculate input / processing rates correctly for DataSourceV2 
streaming sources

## What changes were proposed in this pull request?

In some streaming queries, the input and processing rates are not calculated at 
all (shows up as zero) because MicroBatchExecution fails to associated metrics 
from the executed plan of a trigger with the sources in the logical plan of the 
trigger. The way this executed-plan-leaf-to-logical-source attribution works is 
as follows. With V1 sources, there was no way to identify which execution plan 
leaves were generated by a streaming source. So did a best-effort attempt to 
match logical and execution plan leaves when the number of leaves were same. In 
cases where the number of leaves is different, we just give up and report zero 
rates. An example where this may happen is as follows.

```
val cachedStaticDF = someStaticDF.union(anotherStaticDF).cache()
val streamingInputDF = ...

val query = streamingInputDF.join(cachedStaticDF).writeStream
```
In this case, the `cachedStaticDF` has multiple logical leaves, but in the 
trigger's execution plan it only has leaf because a cached subplan is 
represented as a single InMemoryTableScanExec leaf. This leads to a mismatch in 
the number of leaves causing the input rates to be computed as zero.

With DataSourceV2, all inputs are represented in the executed plan using 
`DataSourceV2ScanExec`, each of which has a reference to the associated logical 
`DataSource` and `DataSourceReader`. So its easy to associate the metrics to 
the original streaming sources.

In this PR, the solution is as follows. If all the streaming sources in a 
streaming query as v2 sources, then use a new code path where the 
execution-metrics-to-source mapping is done directly. Otherwise we fall back to 
existing mapping logic.

## How was this patch tested?
- New unit tests using V2 memory source
- Existing unit tests using V1 source

Author: Tathagata Das <tathagata.das1...@gmail.com>

Closes #21126 from tdas/SPARK-24050.


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

Branch: refs/heads/master
Commit: 396938ef02c70468e1695872f96b1e9aff28b7ea
Parents: 20ca208
Author: Tathagata Das <tathagata.das1...@gmail.com>
Authored: Wed Apr 25 12:21:55 2018 -0700
Committer: Tathagata Das <tathagata.das1...@gmail.com>
Committed: Wed Apr 25 12:21:55 2018 -0700

--
 .../kafka010/KafkaMicroBatchSourceSuite.scala   |   9 +-
 .../execution/streaming/ProgressReporter.scala  | 146 ++-
 .../sql/streaming/StreamingQuerySuite.scala | 134 -
 3 files changed, 245 insertions(+), 44 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/396938ef/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 e017fd9..d2d04b6 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
@@ -563,7 +563,7 @@ abstract class KafkaMicroBatchSourceSuiteBase extends 
KafkaSourceSuiteBase {
 )
   }
 
-  test("ensure stream-stream self-join generates only one offset in offset 
log") {
+  test("ensure stream-stream self-join generates only one offset in log and 
correct metrics") {
 val topic = newTopic()
 testUtils.createTopic(topic, partitions = 2)
 require(testUtils.getLatestOffsets(Set(topic)).size === 2)
@@ -587,7 +587,12 @@ abstract class KafkaMicroBatchSourceSuiteBase extends 
KafkaSourceSuiteBase {
   AddKafkaData(Set(topic), 1, 2),
   CheckAnswer((1, 1, 1), (2, 2, 2)),
   AddKafkaData(Set(topic), 6, 3),
-  CheckAnswer((1, 1, 1), (2, 2, 2), (3, 3, 3), (1, 6, 1), (1, 1, 6), (1, 
6, 6))
+  CheckAnswer((1, 1, 1), (2, 2, 2), (3, 3, 3), (1, 6, 1), (1, 1, 6), (1, 
6, 6)),
+  AssertOnQuery { q =>
+assert(q.availableOffsets.iterator.size == 1)
+assert(q.recentProgress.map(_.numInputRows).sum == 4)
+true
+  }
 )
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/396938ef/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala
---

spark git commit: [SPARK-24038][SS] Refactor continuous writing to its own class

2018-04-24 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master 7b1e6523a -> d6c26d1c9


[SPARK-24038][SS] Refactor continuous writing to its own class

## What changes were proposed in this pull request?

Refactor continuous writing to its own class.

See WIP https://github.com/jose-torres/spark/pull/13 for the overall direction 
this is going, but I think this PR is very isolated and necessary anyway.

## How was this patch tested?

existing unit tests - refactoring only

Author: Jose Torres 

Closes #21116 from jose-torres/SPARK-24038.


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

Branch: refs/heads/master
Commit: d6c26d1c9a8f747a3e0d281a27ea9eb4d92102e5
Parents: 7b1e652
Author: Jose Torres 
Authored: Tue Apr 24 17:06:03 2018 -0700
Committer: Tathagata Das 
Committed: Tue Apr 24 17:06:03 2018 -0700

--
 .../datasources/v2/DataSourceV2Strategy.scala   |   4 +
 .../datasources/v2/WriteToDataSourceV2.scala|  74 +--
 .../continuous/ContinuousExecution.scala|   2 +-
 .../WriteToContinuousDataSource.scala   |  31 +
 .../WriteToContinuousDataSourceExec.scala   | 124 +++
 5 files changed, 165 insertions(+), 70 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/d6c26d1c/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
index 1ac9572..c2a3144 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
@@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.datasources.v2
 import org.apache.spark.sql.Strategy
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
 import org.apache.spark.sql.execution.SparkPlan
+import 
org.apache.spark.sql.execution.streaming.continuous.{WriteToContinuousDataSource,
 WriteToContinuousDataSourceExec}
 
 object DataSourceV2Strategy extends Strategy {
   override def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
@@ -32,6 +33,9 @@ object DataSourceV2Strategy extends Strategy {
 case WriteToDataSourceV2(writer, query) =>
   WriteToDataSourceV2Exec(writer, planLater(query)) :: Nil
 
+case WriteToContinuousDataSource(writer, query) =>
+  WriteToContinuousDataSourceExec(writer, planLater(query)) :: Nil
+
 case _ => Nil
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/d6c26d1c/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2.scala
index e80b44c..ea283ed 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2.scala
@@ -65,25 +65,10 @@ case class WriteToDataSourceV2Exec(writer: 
DataSourceWriter, query: SparkPlan) e
   s"The input RDD has ${messages.length} partitions.")
 
 try {
-  val runTask = writer match {
-// This case means that we're doing continuous processing. In 
microbatch streaming, the
-// StreamWriter is wrapped in a MicroBatchWriter, which is executed as 
a normal batch.
-case w: StreamWriter =>
-  EpochCoordinatorRef.get(
-
sparkContext.getLocalProperty(ContinuousExecution.EPOCH_COORDINATOR_ID_KEY),
-sparkContext.env)
-.askSync[Unit](SetWriterPartitions(rdd.getNumPartitions))
-
-  (context: TaskContext, iter: Iterator[InternalRow]) =>
-DataWritingSparkTask.runContinuous(writeTask, context, iter)
-case _ =>
-  (context: TaskContext, iter: Iterator[InternalRow]) =>
-DataWritingSparkTask.run(writeTask, context, iter, 
useCommitCoordinator)
-  }
-
   sparkContext.runJob(
 rdd,
-runTask,
+(context: TaskContext, iter: Iterator[InternalRow]) =>
+  DataWritingSparkTask.run(writeTask, context, iter, 
useCommitCoordinator),
   

spark git commit: [SPARK-24056][SS] Make consumer creation lazy in Kafka source for Structured streaming

2018-04-24 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master 379bffa05 -> 7b1e6523a


[SPARK-24056][SS] Make consumer creation lazy in Kafka source for Structured 
streaming

## What changes were proposed in this pull request?

Currently, the driver side of the Kafka source (i.e. KafkaMicroBatchReader) 
eagerly creates a consumer as soon as the Kafk aMicroBatchReader is created. 
However, we create dummy KafkaMicroBatchReader to get the schema and 
immediately stop it. Its better to make the consumer creation lazy, it will be 
created on the first attempt to fetch offsets using the KafkaOffsetReader.

## How was this patch tested?
Existing unit tests

Author: Tathagata Das <tathagata.das1...@gmail.com>

Closes #21134 from tdas/SPARK-24056.


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

Branch: refs/heads/master
Commit: 7b1e6523af3c96043aa8d2763e5f18b6e2781c3d
Parents: 379bffa
Author: Tathagata Das <tathagata.das1...@gmail.com>
Authored: Tue Apr 24 14:33:33 2018 -0700
Committer: Tathagata Das <tathagata.das1...@gmail.com>
Committed: Tue Apr 24 14:33:33 2018 -0700

--
 .../spark/sql/kafka010/KafkaOffsetReader.scala  | 31 +++-
 1 file changed, 17 insertions(+), 14 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/7b1e6523/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
--
diff --git 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
index 551641c..8206669 100644
--- 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
+++ 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
@@ -75,7 +75,17 @@ private[kafka010] class KafkaOffsetReader(
* A KafkaConsumer used in the driver to query the latest Kafka offsets. 
This only queries the
* offsets and never commits them.
*/
-  protected var consumer = createConsumer()
+  @volatile protected var _consumer: Consumer[Array[Byte], Array[Byte]] = null
+
+  protected def consumer: Consumer[Array[Byte], Array[Byte]] = synchronized {
+assert(Thread.currentThread().isInstanceOf[UninterruptibleThread])
+if (_consumer == null) {
+  val newKafkaParams = new ju.HashMap[String, Object](driverKafkaParams)
+  newKafkaParams.put(ConsumerConfig.GROUP_ID_CONFIG, nextGroupId())
+  _consumer = consumerStrategy.createConsumer(newKafkaParams)
+}
+_consumer
+  }
 
   private val maxOffsetFetchAttempts =
 readerOptions.getOrElse("fetchOffset.numRetries", "3").toInt
@@ -95,9 +105,7 @@ private[kafka010] class KafkaOffsetReader(
* Closes the connection to Kafka, and cleans up state.
*/
   def close(): Unit = {
-runUninterruptibly {
-  consumer.close()
-}
+if (_consumer != null) runUninterruptibly { stopConsumer() }
 kafkaReaderThread.shutdown()
   }
 
@@ -304,19 +312,14 @@ private[kafka010] class KafkaOffsetReader(
 }
   }
 
-  /**
-   * Create a consumer using the new generated group id. We always use a new 
consumer to avoid
-   * just using a broken consumer to retry on Kafka errors, which likely will 
fail again.
-   */
-  private def createConsumer(): Consumer[Array[Byte], Array[Byte]] = 
synchronized {
-val newKafkaParams = new ju.HashMap[String, Object](driverKafkaParams)
-newKafkaParams.put(ConsumerConfig.GROUP_ID_CONFIG, nextGroupId())
-consumerStrategy.createConsumer(newKafkaParams)
+  private def stopConsumer(): Unit = synchronized {
+assert(Thread.currentThread().isInstanceOf[UninterruptibleThread])
+if (_consumer != null) _consumer.close()
   }
 
   private def resetConsumer(): Unit = synchronized {
-consumer.close()
-consumer = createConsumer()
+stopConsumer()
+_consumer = null  // will automatically get reinitialized again
   }
 }
 


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



spark git commit: [SPARK-23004][SS] Ensure StateStore.commit is called only once in a streaming aggregation task

2018-04-23 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/branch-2.3 c2f4ee7ba -> 8eb9a411d


[SPARK-23004][SS] Ensure StateStore.commit is called only once in a streaming 
aggregation task

## What changes were proposed in this pull request?

A structured streaming query with a streaming aggregation can throw the 
following error in rare cases. 

```
java.lang.IllegalStateException: Cannot commit after already committed or 
aborted
at 
org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider.org$apache$spark$sql$execution$streaming$state$HDFSBackedStateStoreProvider$$verify(HDFSBackedStateStoreProvider.scala:643)
at 
org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider$HDFSBackedStateStore.commit(HDFSBackedStateStoreProvider.scala:135)
at 
org.apache.spark.sql.execution.streaming.StateStoreSaveExec$$anonfun$doExecute$3$$anon$2$$anonfun$hasNext$2.apply$mcV$sp(statefulOperators.scala:359)
at 
org.apache.spark.sql.execution.streaming.StateStoreWriter$class.timeTakenMs(statefulOperators.scala:102)
at 
org.apache.spark.sql.execution.streaming.StateStoreSaveExec.timeTakenMs(statefulOperators.scala:251)
at 
org.apache.spark.sql.execution.streaming.StateStoreSaveExec$$anonfun$doExecute$3$$anon$2.hasNext(statefulOperators.scala:359)
at 
org.apache.spark.sql.execution.aggregate.ObjectAggregationIterator.processInputs(ObjectAggregationIterator.scala:188)
at 
org.apache.spark.sql.execution.aggregate.ObjectAggregationIterator.(ObjectAggregationIterator.scala:78)
at 
org.apache.spark.sql.execution.aggregate.ObjectHashAggregateExec$$anonfun$doExecute$1$$anonfun$2.apply(ObjectHashAggregateExec.scala:114)
at 
org.apache.spark.sql.execution.aggregate.ObjectHashAggregateExec$$anonfun$doExecute$1$$anonfun$2.apply(ObjectHashAggregateExec.scala:105)
at 
org.apache.spark.rdd.RDD$$anonfun$mapPartitionsWithIndexInternal$1$$anonfun$apply$24.apply(RDD.scala:830)
at 
org.apache.spark.rdd.RDD$$anonfun$mapPartitionsWithIndexInternal$1$$anonfun$apply$24.apply(RDD.scala:830)
at 
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:42)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:336)
```

This can happen when the following conditions are accidentally hit. 
 - Streaming aggregation with aggregation function that is a subset of 
[`TypedImperativeAggregation`](https://github.com/apache/spark/blob/76b8b840ddc951ee6203f9cccd2c2b9671c1b5e8/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala#L473)
 (for example, `collect_set`, `collect_list`, `percentile`, etc.). 
 - Query running in `update}` mode
 - After the shuffle, a partition has exactly 128 records. 

This causes StateStore.commit to be called twice. See the 
[JIRA](https://issues.apache.org/jira/browse/SPARK-23004) for a more detailed 
explanation. The solution is to use `NextIterator` or `CompletionIterator`, 
each of which has a flag to prevent the "onCompletion" task from being called 
more than once. In this PR, I chose to implement using `NextIterator`.

## How was this patch tested?

Added unit test that I have confirm will fail without the fix.

Author: Tathagata Das <tathagata.das1...@gmail.com>

Closes #21124 from tdas/SPARK-23004.

(cherry picked from commit 770add81c3474e754867d7105031a5eaf27159bd)
Signed-off-by: Tathagata Das <tathagata.das1...@gmail.com>


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

Branch: refs/heads/branch-2.3
Commit: 8eb9a411d89e43110e9553e0d19a16cdc37bf789
Parents: c2f4ee7
Author: Tathagata Das <tathagata.das1...@gmail.com>
Authored: Mon Apr 23 13:20:32 2018 -0700
Committer: Tathagata Das <tathagata.das1...@gmail.com>
Committed: Mon Apr 23 13:22:03 2018 -0700

--
 .../execution/streaming/statefulOperators.scala | 40 ++--
 .../streaming/StreamingAggregationSuite.scala   | 25 
 2 files changed, 44 insertions(+), 21 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/8eb9a411/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala
index b9b07a2..c9354ac 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execut

spark git commit: [SPARK-23004][SS] Ensure StateStore.commit is called only once in a streaming aggregation task

2018-04-23 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master 448d248f8 -> 770add81c


[SPARK-23004][SS] Ensure StateStore.commit is called only once in a streaming 
aggregation task

## What changes were proposed in this pull request?

A structured streaming query with a streaming aggregation can throw the 
following error in rare cases. 

```
java.lang.IllegalStateException: Cannot commit after already committed or 
aborted
at 
org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider.org$apache$spark$sql$execution$streaming$state$HDFSBackedStateStoreProvider$$verify(HDFSBackedStateStoreProvider.scala:643)
at 
org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider$HDFSBackedStateStore.commit(HDFSBackedStateStoreProvider.scala:135)
at 
org.apache.spark.sql.execution.streaming.StateStoreSaveExec$$anonfun$doExecute$3$$anon$2$$anonfun$hasNext$2.apply$mcV$sp(statefulOperators.scala:359)
at 
org.apache.spark.sql.execution.streaming.StateStoreWriter$class.timeTakenMs(statefulOperators.scala:102)
at 
org.apache.spark.sql.execution.streaming.StateStoreSaveExec.timeTakenMs(statefulOperators.scala:251)
at 
org.apache.spark.sql.execution.streaming.StateStoreSaveExec$$anonfun$doExecute$3$$anon$2.hasNext(statefulOperators.scala:359)
at 
org.apache.spark.sql.execution.aggregate.ObjectAggregationIterator.processInputs(ObjectAggregationIterator.scala:188)
at 
org.apache.spark.sql.execution.aggregate.ObjectAggregationIterator.(ObjectAggregationIterator.scala:78)
at 
org.apache.spark.sql.execution.aggregate.ObjectHashAggregateExec$$anonfun$doExecute$1$$anonfun$2.apply(ObjectHashAggregateExec.scala:114)
at 
org.apache.spark.sql.execution.aggregate.ObjectHashAggregateExec$$anonfun$doExecute$1$$anonfun$2.apply(ObjectHashAggregateExec.scala:105)
at 
org.apache.spark.rdd.RDD$$anonfun$mapPartitionsWithIndexInternal$1$$anonfun$apply$24.apply(RDD.scala:830)
at 
org.apache.spark.rdd.RDD$$anonfun$mapPartitionsWithIndexInternal$1$$anonfun$apply$24.apply(RDD.scala:830)
at 
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:42)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:336)
```

This can happen when the following conditions are accidentally hit. 
 - Streaming aggregation with aggregation function that is a subset of 
[`TypedImperativeAggregation`](https://github.com/apache/spark/blob/76b8b840ddc951ee6203f9cccd2c2b9671c1b5e8/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala#L473)
 (for example, `collect_set`, `collect_list`, `percentile`, etc.). 
 - Query running in `update}` mode
 - After the shuffle, a partition has exactly 128 records. 

This causes StateStore.commit to be called twice. See the 
[JIRA](https://issues.apache.org/jira/browse/SPARK-23004) for a more detailed 
explanation. The solution is to use `NextIterator` or `CompletionIterator`, 
each of which has a flag to prevent the "onCompletion" task from being called 
more than once. In this PR, I chose to implement using `NextIterator`.

## How was this patch tested?

Added unit test that I have confirm will fail without the fix.

Author: Tathagata Das <tathagata.das1...@gmail.com>

Closes #21124 from tdas/SPARK-23004.


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

Branch: refs/heads/master
Commit: 770add81c3474e754867d7105031a5eaf27159bd
Parents: 448d248
Author: Tathagata Das <tathagata.das1...@gmail.com>
Authored: Mon Apr 23 13:20:32 2018 -0700
Committer: Tathagata Das <tathagata.das1...@gmail.com>
Committed: Mon Apr 23 13:20:32 2018 -0700

--
 .../execution/streaming/statefulOperators.scala | 40 ++--
 .../streaming/StreamingAggregationSuite.scala   | 25 
 2 files changed, 44 insertions(+), 21 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/770add81/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala
index b9b07a2..c9354ac 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala
@@ -340,37 +340,35 @@ case class StateStoreSaveExec(
   // Update and output modified rows from 

spark git commit: [SPARK-23747][STRUCTURED STREAMING] Add EpochCoordinator unit tests

2018-04-17 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master 1cc66a072 -> 05ae74778


[SPARK-23747][STRUCTURED STREAMING] Add EpochCoordinator unit tests

## What changes were proposed in this pull request?

Unit tests for EpochCoordinator that test correct sequencing of committed 
epochs. Several tests are ignored since they test functionality implemented in 
SPARK-23503 which is not yet merged, otherwise they fail.

Author: Efim Poberezkin 

Closes #20983 from efimpoberezkin/pr/EpochCoordinator-tests.


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

Branch: refs/heads/master
Commit: 05ae74778a10fbdd7f2cbf7742de7855966b7d35
Parents: 1cc66a0
Author: Efim Poberezkin 
Authored: Tue Apr 17 04:13:17 2018 -0700
Committer: Tathagata Das 
Committed: Tue Apr 17 04:13:17 2018 -0700

--
 .../continuous/EpochCoordinatorSuite.scala  | 224 +++
 1 file changed, 224 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/05ae7477/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/EpochCoordinatorSuite.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/EpochCoordinatorSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/EpochCoordinatorSuite.scala
new file mode 100644
index 000..99e3056
--- /dev/null
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/EpochCoordinatorSuite.scala
@@ -0,0 +1,224 @@
+/*
+ * 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.streaming.continuous
+
+import org.mockito.InOrder
+import org.mockito.Matchers.{any, eq => eqTo}
+import org.mockito.Mockito._
+import org.scalatest.BeforeAndAfterEach
+import org.scalatest.mockito.MockitoSugar
+
+import org.apache.spark._
+import org.apache.spark.rpc.RpcEndpointRef
+import org.apache.spark.sql.LocalSparkSession
+import org.apache.spark.sql.execution.streaming.continuous._
+import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousReader, 
PartitionOffset}
+import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage
+import org.apache.spark.sql.sources.v2.writer.streaming.StreamWriter
+import org.apache.spark.sql.test.TestSparkSession
+
+class EpochCoordinatorSuite
+  extends SparkFunSuite
+with LocalSparkSession
+with MockitoSugar
+with BeforeAndAfterEach {
+
+  private var epochCoordinator: RpcEndpointRef = _
+
+  private var writer: StreamWriter = _
+  private var query: ContinuousExecution = _
+  private var orderVerifier: InOrder = _
+
+  override def beforeEach(): Unit = {
+val reader = mock[ContinuousReader]
+writer = mock[StreamWriter]
+query = mock[ContinuousExecution]
+orderVerifier = inOrder(writer, query)
+
+spark = new TestSparkSession()
+
+epochCoordinator
+  = EpochCoordinatorRef.create(writer, reader, query, "test", 1, spark, 
SparkEnv.get)
+  }
+
+  test("single epoch") {
+setWriterPartitions(3)
+setReaderPartitions(2)
+
+commitPartitionEpoch(0, 1)
+commitPartitionEpoch(1, 1)
+commitPartitionEpoch(2, 1)
+reportPartitionOffset(0, 1)
+reportPartitionOffset(1, 1)
+
+// Here and in subsequent tests this is called to make a synchronous call 
to EpochCoordinator
+// so that mocks would have been acted upon by the time verification 
happens
+makeSynchronousCall()
+
+verifyCommit(1)
+  }
+
+  test("single epoch, all but one writer partition has committed") {
+setWriterPartitions(3)
+setReaderPartitions(2)
+
+commitPartitionEpoch(0, 1)
+commitPartitionEpoch(1, 1)
+reportPartitionOffset(0, 1)
+reportPartitionOffset(1, 1)
+
+makeSynchronousCall()
+
+verifyNoCommitFor(1)
+  }
+
+  test("single epoch, all but one reader partition has 

spark git commit: [SPARK-23687][SS] Add a memory source for continuous processing.

2018-04-17 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master 14844a62c -> 1cc66a072


[SPARK-23687][SS] Add a memory source for continuous processing.

## What changes were proposed in this pull request?

Add a memory source for continuous processing.

Note that only one of the ContinuousSuite tests is migrated to minimize the 
diff here. I'll submit a second PR for SPARK-23688 to change the rest and get 
rid of waitForRateSourceTriggers.

## How was this patch tested?

unit test

Author: Jose Torres 

Closes #20828 from jose-torres/continuousMemory.


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

Branch: refs/heads/master
Commit: 1cc66a072b7fd3bf140fa41596f6b18f8d1bd7b9
Parents: 14844a6
Author: Jose Torres 
Authored: Tue Apr 17 01:59:38 2018 -0700
Committer: Tathagata Das 
Committed: Tue Apr 17 01:59:38 2018 -0700

--
 .../continuous/ContinuousExecution.scala|   5 +-
 .../spark/sql/execution/streaming/memory.scala  |  59 --
 .../sources/ContinuousMemoryStream.scala| 211 +++
 .../apache/spark/sql/streaming/StreamTest.scala |   4 +-
 .../streaming/continuous/ContinuousSuite.scala  |  31 ++-
 5 files changed, 266 insertions(+), 44 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/1cc66a07/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 1758b38..951d694 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
@@ -31,6 +31,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
 import org.apache.spark.sql.execution.SQLExecution
 import 
org.apache.spark.sql.execution.datasources.v2.{StreamingDataSourceV2Relation, 
WriteToDataSourceV2}
 import org.apache.spark.sql.execution.streaming.{ContinuousExecutionRelation, 
StreamingRelationV2, _}
+import org.apache.spark.sql.sources.v2
 import org.apache.spark.sql.sources.v2.{ContinuousReadSupport, 
DataSourceOptions, StreamWriteSupport}
 import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousReader, 
PartitionOffset}
 import org.apache.spark.sql.streaming.{OutputMode, ProcessingTime, Trigger}
@@ -317,8 +318,10 @@ class ContinuousExecution(
 synchronized {
   if (queryExecutionThread.isAlive) {
 commitLog.add(epoch)
-val offset = offsetLog.get(epoch).get.offsets(0).get
+val offset =
+  
continuousSources(0).deserializeOffset(offsetLog.get(epoch).get.offsets(0).get.json)
 committedOffsets ++= Seq(continuousSources(0) -> offset)
+
continuousSources(0).commit(offset.asInstanceOf[v2.reader.streaming.Offset])
   } else {
 return
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/1cc66a07/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala
index 352d4ce..628923d 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala
@@ -24,17 +24,19 @@ import javax.annotation.concurrent.GuardedBy
 
 import scala.collection.JavaConverters._
 import scala.collection.mutable.{ArrayBuffer, ListBuffer}
+import scala.reflect.ClassTag
 import scala.util.control.NonFatal
 
 import org.apache.spark.internal.Logging
 import org.apache.spark.sql._
-import org.apache.spark.sql.catalyst.encoders.encoderFor
+import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder}
 import org.apache.spark.sql.catalyst.expressions.{Attribute, UnsafeRow}
-import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
+import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, 
Statistics}
 import org.apache.spark.sql.catalyst.streaming.InternalOutputModes._
+import org.apache.spark.sql.execution.streaming.continuous.ContinuousTrigger
 import org.apache.spark.sql.sources.v2.reader.{DataReader, 

spark git commit: [SPARK-23966][SS] Refactoring all checkpoint file writing logic in a common CheckpointFileManager interface

2018-04-13 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master 558f31b31 -> cbb41a0c5


[SPARK-23966][SS] Refactoring all checkpoint file writing logic in a common 
CheckpointFileManager interface

## What changes were proposed in this pull request?

Checkpoint files (offset log files, state store files) in Structured Streaming 
must be written atomically such that no partial files are generated (would 
break fault-tolerance guarantees). Currently, there are 3 locations which try 
to do this individually, and in some cases, incorrectly.

1. HDFSOffsetMetadataLog - This uses a FileManager interface to use any 
implementation of `FileSystem` or `FileContext` APIs. It preferably loads 
`FileContext` implementation as FileContext of HDFS has atomic renames.
1. HDFSBackedStateStore (aka in-memory state store)
  - Writing a version.delta file - This uses FileSystem APIs only to perform a 
rename. This is incorrect as rename is not atomic in HDFS FileSystem 
implementation.
  - Writing a snapshot file - Same as above.

 Current problems:
1. State Store behavior is incorrect - HDFS FileSystem implementation does not 
have atomic rename.
1. Inflexible - Some file systems provide mechanisms other than 
write-to-temp-file-and-rename for writing atomically and more efficiently. For 
example, with S3 you can write directly to the final file and it will be made 
visible only when the entire file is written and closed correctly. Any failure 
can be made to terminate the writing without making any partial files visible 
in S3. The current code does not abstract out this mechanism enough that it can 
be customized.

 Solution:

1. Introduce a common interface that all 3 cases above can use to write 
checkpoint files atomically.
2. This interface must provide the necessary interfaces that allow 
customization of the write-and-rename mechanism.

This PR does that by introducing the interface `CheckpointFileManager` and 
modifying `HDFSMetadataLog` and `HDFSBackedStateStore` to use the interface. 
Similar to earlier `FileManager`, there are implementations based on 
`FileSystem` and `FileContext` APIs, and the latter implementation is preferred 
to make it work correctly with HDFS.

The key method this interface has is `createAtomic(path, overwrite)` which 
returns a `CancellableFSDataOutputStream` that has the method `cancel()`. All 
users of this method need to either call `close()` to successfully write the 
file, or `cancel()` in case of an error.

## How was this patch tested?
New tests in `CheckpointFileManagerSuite` and slightly modified existing tests.

Author: Tathagata Das <tathagata.das1...@gmail.com>

Closes #21048 from tdas/SPARK-23966.


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

Branch: refs/heads/master
Commit: cbb41a0c5b01579c85f06ef42cc0585fbef216c5
Parents: 558f31b
Author: Tathagata Das <tathagata.das1...@gmail.com>
Authored: Fri Apr 13 16:31:39 2018 -0700
Committer: Tathagata Das <tathagata.das1...@gmail.com>
Committed: Fri Apr 13 16:31:39 2018 -0700

--
 .../org/apache/spark/sql/internal/SQLConf.scala |   7 +
 .../streaming/CheckpointFileManager.scala   | 349 +++
 .../execution/streaming/HDFSMetadataLog.scala   | 229 +---
 .../state/HDFSBackedStateStoreProvider.scala| 120 +++
 .../execution/streaming/state/StateStore.scala  |   4 +-
 .../streaming/CheckpointFileManagerSuite.scala  | 192 ++
 .../CompactibleFileStreamLogSuite.scala |   5 -
 .../streaming/HDFSMetadataLogSuite.scala| 116 +-
 .../streaming/state/StateStoreSuite.scala   |  58 ++-
 9 files changed, 678 insertions(+), 402 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/cbb41a0c/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
--
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 1c8ab9c..0dc47bf 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
@@ -930,6 +930,13 @@ object SQLConf {
   .intConf
   .createWithDefault(100)
 
+  val STREAMING_CHECKPOINT_FILE_MANAGER_CLASS =
+buildConf("spark.sql.streaming.checkpointFileManagerClass")
+  .doc("The class used to write checkpoint files atomically. This class 
must be a subclass " +
+"of the interface CheckpointFileManager.")
+  .internal()
+  

spark git commit: [SPARK-23748][SS] Fix SS continuous process doesn't support SubqueryAlias issue

2018-04-12 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/branch-2.3 908c681c6 -> 2995b79d6


[SPARK-23748][SS] Fix SS continuous process doesn't support SubqueryAlias issue

## What changes were proposed in this pull request?

Current SS continuous doesn't support processing on temp table or 
`df.as("xxx")`, SS will throw an exception as LogicalPlan not supported, 
details described in [here](https://issues.apache.org/jira/browse/SPARK-23748).

So here propose to add this support.

## How was this patch tested?

new UT.

Author: jerryshao 

Closes #21017 from jerryshao/SPARK-23748.

(cherry picked from commit 14291b061b9b40eadbf4ed442f9a5021b8e09597)
Signed-off-by: Tathagata Das 


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

Branch: refs/heads/branch-2.3
Commit: 2995b79d6a78bf632aa4c1c99bebfc213fb31c54
Parents: 908c681
Author: jerryshao 
Authored: Thu Apr 12 20:00:25 2018 -0700
Committer: Tathagata Das 
Committed: Thu Apr 12 20:00:40 2018 -0700

--
 .../analysis/UnsupportedOperationChecker.scala   |  2 +-
 .../streaming/continuous/ContinuousSuite.scala   | 19 +++
 2 files changed, 20 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/2995b79d/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 b55043c..ff9d6d7 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,7 @@ object UnsupportedOperationChecker {
 plan.foreachUp { implicit subPlan =>
   subPlan match {
 case (_: Project | _: Filter | _: MapElements | _: MapPartitions |
-  _: DeserializeToObject | _: SerializeFromObject) =>
+  _: DeserializeToObject | _: SerializeFromObject | _: 
SubqueryAlias) =>
 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/2995b79d/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 4b4ed82..95406b3 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
@@ -174,6 +174,25 @@ class ContinuousSuite extends ContinuousSuiteBase {
   "Continuous processing does not support current time operations."))
   }
 
+  test("subquery alias") {
+val df = spark.readStream
+  .format("rate")
+  .option("numPartitions", "5")
+  .option("rowsPerSecond", "5")
+  .load()
+  .createOrReplaceTempView("rate")
+val test = spark.sql("select value from rate where value > 5")
+
+testStream(test, useV2Sink = true)(
+  StartStream(longContinuousTrigger),
+  AwaitEpoch(0),
+  Execute(waitForRateSourceTriggers(_, 2)),
+  IncrementEpoch(),
+  Execute(waitForRateSourceTriggers(_, 4)),
+  IncrementEpoch(),
+  CheckAnswerRowsContains(scala.Range(6, 20).map(Row(_
+  }
+
   test("repeatedly restart") {
 val df = spark.readStream
   .format("rate")


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



spark git commit: [SPARK-23748][SS] Fix SS continuous process doesn't support SubqueryAlias issue

2018-04-12 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master 682002b6d -> 14291b061


[SPARK-23748][SS] Fix SS continuous process doesn't support SubqueryAlias issue

## What changes were proposed in this pull request?

Current SS continuous doesn't support processing on temp table or 
`df.as("xxx")`, SS will throw an exception as LogicalPlan not supported, 
details described in [here](https://issues.apache.org/jira/browse/SPARK-23748).

So here propose to add this support.

## How was this patch tested?

new UT.

Author: jerryshao 

Closes #21017 from jerryshao/SPARK-23748.


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

Branch: refs/heads/master
Commit: 14291b061b9b40eadbf4ed442f9a5021b8e09597
Parents: 682002b
Author: jerryshao 
Authored: Thu Apr 12 20:00:25 2018 -0700
Committer: Tathagata Das 
Committed: Thu Apr 12 20:00:25 2018 -0700

--
 .../analysis/UnsupportedOperationChecker.scala   |  2 +-
 .../streaming/continuous/ContinuousSuite.scala   | 19 +++
 2 files changed, 20 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/14291b06/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 b55043c..ff9d6d7 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,7 @@ object UnsupportedOperationChecker {
 plan.foreachUp { implicit subPlan =>
   subPlan match {
 case (_: Project | _: Filter | _: MapElements | _: MapPartitions |
-  _: DeserializeToObject | _: SerializeFromObject) =>
+  _: DeserializeToObject | _: SerializeFromObject | _: 
SubqueryAlias) =>
 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/14291b06/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 f5884b9..ef74efe 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
@@ -171,6 +171,25 @@ class ContinuousSuite extends ContinuousSuiteBase {
   "Continuous processing does not support current time operations."))
   }
 
+  test("subquery alias") {
+val df = spark.readStream
+  .format("rate")
+  .option("numPartitions", "5")
+  .option("rowsPerSecond", "5")
+  .load()
+  .createOrReplaceTempView("rate")
+val test = spark.sql("select value from rate where value > 5")
+
+testStream(test, useV2Sink = true)(
+  StartStream(longContinuousTrigger),
+  AwaitEpoch(0),
+  Execute(waitForRateSourceTriggers(_, 2)),
+  IncrementEpoch(),
+  Execute(waitForRateSourceTriggers(_, 4)),
+  IncrementEpoch(),
+  CheckAnswerRowsContains(scala.Range(6, 20).map(Row(_
+  }
+
   test("repeatedly restart") {
 val df = spark.readStream
   .format("rate")


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



spark git commit: [SPARK-23099][SS] Migrate foreach sink to DataSourceV2

2018-04-03 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master 7cf9fab33 -> 66a3a5a2d


[SPARK-23099][SS] Migrate foreach sink to DataSourceV2

## What changes were proposed in this pull request?

Migrate foreach sink to DataSourceV2.

Since the previous attempt at this PR #20552, we've changed and strictly 
defined the lifecycle of writer components. This means we no longer need the 
complicated lifecycle shim from that PR; it just naturally works.

## How was this patch tested?

existing tests

Author: Jose Torres 

Closes #20951 from jose-torres/foreach.


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

Branch: refs/heads/master
Commit: 66a3a5a2dc83e03dedcee9839415c1ddc1fb8125
Parents: 7cf9fab
Author: Jose Torres 
Authored: Tue Apr 3 11:05:29 2018 -0700
Committer: Tathagata Das 
Committed: Tue Apr 3 11:05:29 2018 -0700

--
 .../sql/execution/streaming/ForeachSink.scala   |  68 -
 .../sources/ForeachWriterProvider.scala | 111 +++
 .../spark/sql/streaming/DataStreamWriter.scala  |   4 +-
 .../execution/streaming/ForeachSinkSuite.scala  | 305 --
 .../streaming/sources/ForeachWriterSuite.scala  | 306 +++
 .../sql/streaming/StreamingQuerySuite.scala |   1 +
 6 files changed, 420 insertions(+), 375 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/66a3a5a2/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ForeachSink.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ForeachSink.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ForeachSink.scala
deleted file mode 100644
index 2cc5410..000
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ForeachSink.scala
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * 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
-
-import org.apache.spark.TaskContext
-import org.apache.spark.sql.{DataFrame, Encoder, ForeachWriter}
-import org.apache.spark.sql.catalyst.encoders.encoderFor
-
-/**
- * A [[Sink]] that forwards all data into [[ForeachWriter]] according to the 
contract defined by
- * [[ForeachWriter]].
- *
- * @param writer The [[ForeachWriter]] to process all data.
- * @tparam T The expected type of the sink.
- */
-class ForeachSink[T : Encoder](writer: ForeachWriter[T]) extends Sink with 
Serializable {
-
-  override def addBatch(batchId: Long, data: DataFrame): Unit = {
-// This logic should've been as simple as:
-// ```
-//   data.as[T].foreachPartition { iter => ... }
-// ```
-//
-// Unfortunately, doing that would just break the incremental planing. The 
reason is,
-// `Dataset.foreachPartition()` would further call `Dataset.rdd()`, but 
`Dataset.rdd()` will
-// create a new plan. Because StreamExecution uses the existing plan to 
collect metrics and
-// update watermark, we should never create a new plan. Otherwise, metrics 
and watermark are
-// updated in the new plan, and StreamExecution cannot retrieval them.
-//
-// Hence, we need to manually convert internal rows to objects using 
encoder.
-val encoder = encoderFor[T].resolveAndBind(
-  data.logicalPlan.output,
-  data.sparkSession.sessionState.analyzer)
-data.queryExecution.toRdd.foreachPartition { iter =>
-  if (writer.open(TaskContext.getPartitionId(), batchId)) {
-try {
-  while (iter.hasNext) {
-writer.process(encoder.fromRow(iter.next()))
-  }
-} catch {
-  case e: Throwable =>
-writer.close(e)
-throw e
-}
-writer.close(null)
-  } else {
-writer.close(null)
-  }
-}
-  }
-
-  

spark git commit: [SPARK-23827][SS] StreamingJoinExec should ensure that input data is partitioned into specific number of partitions

2018-03-30 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/branch-2.3 3f5955aa0 -> 507cff246


[SPARK-23827][SS] StreamingJoinExec should ensure that input data is 
partitioned into specific number of partitions

## What changes were proposed in this pull request?

Currently, the requiredChildDistribution does not specify the partitions. This 
can cause the weird corner cases where the child's distribution is 
`SinglePartition` which satisfies the required distribution of 
`ClusterDistribution(no-num-partition-requirement)`, thus eliminating the 
shuffle needed to repartition input data into the required number of partitions 
(i.e. same as state stores). That can lead to "file not found" errors on the 
state store delta files as the micro-batch-with-no-shuffle will not run certain 
tasks and therefore not generate the expected state store delta files.

This PR adds the required constraint on the number of partitions.

## How was this patch tested?
Modified test harness to always check that ANY stateful operator should have a 
constraint on the number of partitions. As part of that, the existing opt-in 
checks on child output partitioning were removed, as they are redundant.

Author: Tathagata Das <tathagata.das1...@gmail.com>

Closes #20941 from tdas/SPARK-23827.

(cherry picked from commit 15298b99ac8944e781328423289586176cf824d7)
Signed-off-by: Tathagata Das <tathagata.das1...@gmail.com>


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

Branch: refs/heads/branch-2.3
Commit: 507cff246cd9e15a418d67b66bf762be4ae71c67
Parents: 3f5955a
Author: Tathagata Das <tathagata.das1...@gmail.com>
Authored: Fri Mar 30 16:48:26 2018 -0700
Committer: Tathagata Das <tathagata.das1...@gmail.com>
Committed: Fri Mar 30 16:48:55 2018 -0700

--
 .../streaming/IncrementalExecution.scala|  2 +-
 .../StreamingSymmetricHashJoinExec.scala|  3 +-
 .../spark/sql/streaming/DeduplicateSuite.scala  |  8 +---
 .../streaming/FlatMapGroupsWithStateSuite.scala |  5 +-
 .../sql/streaming/StatefulOperatorTest.scala| 49 
 .../apache/spark/sql/streaming/StreamTest.scala | 19 
 .../streaming/StreamingAggregationSuite.scala   |  4 +-
 7 files changed, 25 insertions(+), 65 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/507cff24/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 a10ed5f..1a83c88 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
@@ -62,7 +62,7 @@ class IncrementalExecution(
   StreamingDeduplicationStrategy :: Nil
   }
 
-  private val numStateStores = 
offsetSeqMetadata.conf.get(SQLConf.SHUFFLE_PARTITIONS.key)
+  private[sql] val numStateStores = 
offsetSeqMetadata.conf.get(SQLConf.SHUFFLE_PARTITIONS.key)
 .map(SQLConf.SHUFFLE_PARTITIONS.valueConverter)
 .getOrElse(sparkSession.sessionState.conf.numShufflePartitions)
 

http://git-wip-us.apache.org/repos/asf/spark/blob/507cff24/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala
index c351f65..fa7c8ee 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala
@@ -167,7 +167,8 @@ case class StreamingSymmetricHashJoinExec(
   val nullRight = new 
GenericInternalRow(right.output.map(_.withNullability(true)).length)
 
   override def requiredChildDistribution: Seq[Distribution] =
-ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil
+ClusteredDistribution(leftKeys, stateInfo.map(_.numPartitions)) ::
+  ClusteredDistribution(rightKeys, stateInfo.map(_.numPartitions)) :: Nil
 
   override def output: Seq[Attribute] = joinType match {
 case _: InnerLike => left.output ++ right.output

http://git-wip-us.apache.org/repos/asf/spark/blob/507

spark git commit: [SPARK-23096][SS] Migrate rate source to V2

2018-03-27 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master 35997b59f -> c68ec4e6a


[SPARK-23096][SS] Migrate rate source to V2

## What changes were proposed in this pull request?

This PR migrate micro batch rate source to V2 API and rewrite UTs to suite V2 
test.

## How was this patch tested?

UTs.

Author: jerryshao 

Closes #20688 from jerryshao/SPARK-23096.


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

Branch: refs/heads/master
Commit: c68ec4e6a1ed9ea13345c7705ea60ff4df7aec7b
Parents: 35997b5
Author: jerryshao 
Authored: Tue Mar 27 14:39:05 2018 -0700
Committer: Tathagata Das 
Committed: Tue Mar 27 14:39:05 2018 -0700

--
 apache.spark.sql.sources.DataSourceRegister |   3 +-
 .../sql/execution/datasources/DataSource.scala  |   6 +-
 .../streaming/RateSourceProvider.scala  | 262 --
 .../continuous/ContinuousRateStreamSource.scala |  25 +-
 .../sources/RateStreamMicroBatchReader.scala| 222 
 .../streaming/sources/RateStreamProvider.scala  | 125 +++
 .../streaming/sources/RateStreamSourceV2.scala  | 187 --
 .../execution/streaming/RateSourceSuite.scala   | 194 ---
 .../execution/streaming/RateSourceV2Suite.scala | 191 --
 .../sources/RateStreamProviderSuite.scala   | 344 +++
 10 files changed, 715 insertions(+), 844 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/c68ec4e6/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
--
diff --git 
a/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
 
b/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
index 1fe9c09..1b37905 100644
--- 
a/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
+++ 
b/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
@@ -5,6 +5,5 @@ org.apache.spark.sql.execution.datasources.orc.OrcFileFormat
 org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
 org.apache.spark.sql.execution.datasources.text.TextFileFormat
 org.apache.spark.sql.execution.streaming.ConsoleSinkProvider
-org.apache.spark.sql.execution.streaming.RateSourceProvider
+org.apache.spark.sql.execution.streaming.sources.RateStreamProvider
 org.apache.spark.sql.execution.streaming.sources.TextSocketSourceProvider
-org.apache.spark.sql.execution.streaming.sources.RateSourceProviderV2

http://git-wip-us.apache.org/repos/asf/spark/blob/c68ec4e6/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
index 31fa89b..b84ea76 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
@@ -41,7 +41,7 @@ import 
org.apache.spark.sql.execution.datasources.json.JsonFileFormat
 import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat
 import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
 import org.apache.spark.sql.execution.streaming._
-import 
org.apache.spark.sql.execution.streaming.sources.TextSocketSourceProvider
+import org.apache.spark.sql.execution.streaming.sources.{RateStreamProvider, 
TextSocketSourceProvider}
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.sources._
 import org.apache.spark.sql.streaming.OutputMode
@@ -566,6 +566,7 @@ object DataSource extends Logging {
 val orc = "org.apache.spark.sql.hive.orc.OrcFileFormat"
 val nativeOrc = classOf[OrcFileFormat].getCanonicalName
 val socket = classOf[TextSocketSourceProvider].getCanonicalName
+val rate = classOf[RateStreamProvider].getCanonicalName
 
 Map(
   "org.apache.spark.sql.jdbc" -> jdbc,
@@ -587,7 +588,8 @@ object DataSource extends Logging {
   "org.apache.spark.ml.source.libsvm.DefaultSource" -> libsvm,
   "org.apache.spark.ml.source.libsvm" -> libsvm,
   "com.databricks.spark.csv" -> csv,
-  "org.apache.spark.sql.execution.streaming.TextSocketSourceProvider" -> 
socket
+  "org.apache.spark.sql.execution.streaming.TextSocketSourceProvider" -> 
socket,
+  

spark git commit: [SPARK-23406][SS] Enable stream-stream self-joins for branch-2.3

2018-03-07 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/branch-2.3 1dd37ff3b -> 404f7e201


[SPARK-23406][SS] Enable stream-stream self-joins for branch-2.3

This is a backport of #20598.

## What changes were proposed in this pull request?

Solved two bugs to enable stream-stream self joins.

### Incorrect analysis due to missing MultiInstanceRelation trait
Streaming leaf nodes did not extend MultiInstanceRelation, which is necessary 
for the catalyst analyzer to convert the self-join logical plan DAG into a tree 
(by creating new instances of the leaf relations). This was causing the error 
`Failure when resolving conflicting references in Join:` (see JIRA for details).

### Incorrect attribute rewrite when splicing batch plans in MicroBatchExecution
When splicing the source's batch plan into the streaming plan (by replacing the 
StreamingExecutionPlan), we were rewriting the attribute reference in the 
streaming plan with the new attribute references from the batch plan. This was 
incorrectly handling the scenario when multiple StreamingExecutionRelation 
point to the same source, and therefore eventually point to the same batch plan 
returned by the source. Here is an example query, and its corresponding plan 
transformations.
```
val df = input.toDF
val join =
  df.select('value % 5 as "key", 'value).join(
df.select('value % 5 as "key", 'value), "key")
```
Streaming logical plan before splicing the batch plan
```
Project [key#6, value#1, value#12]
+- Join Inner, (key#6 = key#9)
   :- Project [(value#1 % 5) AS key#6, value#1]
   :  +- StreamingExecutionRelation Memory[#1], value#1
   +- Project [(value#12 % 5) AS key#9, value#12]
  +- StreamingExecutionRelation Memory[#1], value#12  // two different 
leaves pointing to same source
```
Batch logical plan after splicing the batch plan and before rewriting
```
Project [key#6, value#1, value#12]
+- Join Inner, (key#6 = key#9)
   :- Project [(value#1 % 5) AS key#6, value#1]
   :  +- LocalRelation [value#66]   // replaces 
StreamingExecutionRelation Memory[#1], value#1
   +- Project [(value#12 % 5) AS key#9, value#12]
  +- LocalRelation [value#66]   // replaces 
StreamingExecutionRelation Memory[#1], value#12
```
Batch logical plan after rewriting the attributes. Specifically, for spliced, 
the new output attributes (value#66) replace the earlier output attributes 
(value#12, and value#1, one for each StreamingExecutionRelation).
```
Project [key#6, value#66, value#66]   // both value#1 and value#12 replaces 
by value#66
+- Join Inner, (key#6 = key#9)
   :- Project [(value#66 % 5) AS key#6, value#66]
   :  +- LocalRelation [value#66]
   +- Project [(value#66 % 5) AS key#9, value#66]
  +- LocalRelation [value#66]
```
This causes the optimizer to eliminate value#66 from one side of the join.
```
Project [key#6, value#66, value#66]
+- Join Inner, (key#6 = key#9)
   :- Project [(value#66 % 5) AS key#6, value#66]
   :  +- LocalRelation [value#66]
   +- Project [(value#66 % 5) AS key#9]   // this does not generate value, 
incorrect join results
  +- LocalRelation [value#66]
```

**Solution**: Instead of rewriting attributes, use a Project to introduce 
aliases between the output attribute references and the new reference generated 
by the spliced plans. The analyzer and optimizer will take care of the rest.
```
Project [key#6, value#1, value#12]
+- Join Inner, (key#6 = key#9)
   :- Project [(value#1 % 5) AS key#6, value#1]
   :  +- Project [value#66 AS value#1]   // solution: project with aliases
   : +- LocalRelation [value#66]
   +- Project [(value#12 % 5) AS key#9, value#12]
  +- Project [value#66 AS value#12]// solution: project with aliases
 +- LocalRelation [value#66]
```

## How was this patch tested?
New unit test

Author: Tathagata Das <tathagata.das1...@gmail.com>

Closes #20765 from tdas/SPARK-23406-2.3.


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

Branch: refs/heads/branch-2.3
Commit: 404f7e2013ecfdf993a17fd942d8890d9a8100e7
Parents: 1dd37ff
Author: Tathagata Das <tathagata.das1...@gmail.com>
Authored: Wed Mar 7 21:58:57 2018 -0800
Committer: Tathagata Das <tathagata.das1...@gmail.com>
Committed: Wed Mar 7 21:58:57 2018 -0800

--
 .../streaming/MicroBatchExecution.scala | 16 ++---
 .../execution/streaming/StreamingRelation.scala | 20 +++-
 .../sql/streaming/StreamingJoinSuite.scala  | 25 +++-
 3 files changed, 45 insertions(+), 16 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/404f7e20/sql/core/src/main/scala/org/apache/spar

spark git commit: [SPARK-23559][SS] Add epoch ID to DataWriterFactory.

2018-03-05 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master ba622f45c -> b0f422c38


[SPARK-23559][SS] Add epoch ID to DataWriterFactory.

## What changes were proposed in this pull request?

Add an epoch ID argument to DataWriterFactory for use in streaming. As a side 
effect of passing in this value, DataWriter will now have a consistent 
lifecycle; commit() or abort() ends the lifecycle of a DataWriter instance in 
any execution mode.

I considered making a separate streaming interface and adding the epoch ID only 
to that one, but I think it requires a lot of extra work for no real gain. I 
think it makes sense to define epoch 0 as the one and only epoch of a 
non-streaming query.

## How was this patch tested?

existing unit tests

Author: Jose Torres 

Closes #20710 from jose-torres/api2.


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

Branch: refs/heads/master
Commit: b0f422c3861a5a3831e481b8ffac08f6fa085d00
Parents: ba622f4
Author: Jose Torres 
Authored: Mon Mar 5 13:23:01 2018 -0800
Committer: Tathagata Das 
Committed: Mon Mar 5 13:23:01 2018 -0800

--
 .../spark/sql/kafka010/KafkaStreamWriter.scala  |  5 +++-
 .../spark/sql/sources/v2/writer/DataWriter.java | 12 ++
 .../sources/v2/writer/DataWriterFactory.java|  5 +++-
 .../v2/writer/streaming/StreamWriter.java   | 19 +++
 .../datasources/v2/WriteToDataSourceV2.scala| 25 +---
 .../streaming/MicroBatchExecution.scala |  7 ++
 .../sources/PackedRowWriterFactory.scala|  5 +++-
 .../execution/streaming/sources/memoryV2.scala  |  5 +++-
 .../sources/v2/SimpleWritableDataSource.scala   | 10 ++--
 9 files changed, 65 insertions(+), 28 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/b0f422c3/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaStreamWriter.scala
--
diff --git 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaStreamWriter.scala
 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaStreamWriter.scala
index 9307bfc..ae5b5c5 100644
--- 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaStreamWriter.scala
+++ 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaStreamWriter.scala
@@ -65,7 +65,10 @@ case class KafkaStreamWriterFactory(
 topic: Option[String], producerParams: Map[String, String], schema: 
StructType)
   extends DataWriterFactory[InternalRow] {
 
-  override def createDataWriter(partitionId: Int, attemptNumber: Int): 
DataWriter[InternalRow] = {
+  override def createDataWriter(
+  partitionId: Int,
+  attemptNumber: Int,
+  epochId: Long): DataWriter[InternalRow] = {
 new KafkaStreamDataWriter(topic, producerParams, schema.toAttributes)
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/b0f422c3/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriter.java
--
diff --git 
a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriter.java 
b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriter.java
index 53941a8..39bf458 100644
--- 
a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriter.java
+++ 
b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriter.java
@@ -22,7 +22,7 @@ import java.io.IOException;
 import org.apache.spark.annotation.InterfaceStability;
 
 /**
- * A data writer returned by {@link DataWriterFactory#createDataWriter(int, 
int)} and is
+ * A data writer returned by {@link DataWriterFactory#createDataWriter(int, 
int, long)} and is
  * responsible for writing data for an input RDD partition.
  *
  * One Spark task has one exclusive data writer, so there is no thread-safe 
concern.
@@ -31,13 +31,17 @@ import org.apache.spark.annotation.InterfaceStability;
  * the {@link #write(Object)}, {@link #abort()} is called afterwards and the 
remaining records will
  * not be processed. If all records are successfully written, {@link 
#commit()} is called.
  *
+ * Once a data writer returns successfully from {@link #commit()} or {@link 
#abort()}, its lifecycle
+ * is over and Spark will not use it again.
+ *
  * If this data writer succeeds(all records are successfully written and 
{@link #commit()}
  * succeeds), a {@link WriterCommitMessage} will be sent to the driver side 
and pass to
  * {@link DataSourceWriter#commit(WriterCommitMessage[])} with commit 

spark git commit: [SPARK-23097][SQL][SS] Migrate text socket source to V2

2018-03-02 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master 3a4d15e5d -> 707e6506d


[SPARK-23097][SQL][SS] Migrate text socket source to V2

## What changes were proposed in this pull request?

This PR moves structured streaming text socket source to V2.

Questions: do we need to remove old "socket" source?

## How was this patch tested?

Unit test and manual verification.

Author: jerryshao 

Closes #20382 from jerryshao/SPARK-23097.


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

Branch: refs/heads/master
Commit: 707e6506d0dbdb598a6c99d666f3c66746113b67
Parents: 3a4d15e
Author: jerryshao 
Authored: Fri Mar 2 12:27:42 2018 -0800
Committer: Tathagata Das 
Committed: Fri Mar 2 12:27:42 2018 -0800

--
 apache.spark.sql.sources.DataSourceRegister |   2 +-
 .../sql/execution/datasources/DataSource.scala  |   5 +-
 .../spark/sql/execution/streaming/socket.scala  | 219 -
 .../execution/streaming/sources/socket.scala| 255 
 .../spark/sql/streaming/DataStreamReader.scala  |  21 +-
 .../streaming/TextSocketStreamSuite.scala   | 231 --
 .../sources/TextSocketStreamSuite.scala | 306 +++
 7 files changed, 582 insertions(+), 457 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/707e6506/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
--
diff --git 
a/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
 
b/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
index 0259c77..1fe9c09 100644
--- 
a/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
+++ 
b/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
@@ -5,6 +5,6 @@ org.apache.spark.sql.execution.datasources.orc.OrcFileFormat
 org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
 org.apache.spark.sql.execution.datasources.text.TextFileFormat
 org.apache.spark.sql.execution.streaming.ConsoleSinkProvider
-org.apache.spark.sql.execution.streaming.TextSocketSourceProvider
 org.apache.spark.sql.execution.streaming.RateSourceProvider
+org.apache.spark.sql.execution.streaming.sources.TextSocketSourceProvider
 org.apache.spark.sql.execution.streaming.sources.RateSourceProviderV2

http://git-wip-us.apache.org/repos/asf/spark/blob/707e6506/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
index 6e1b572..35fcff6 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
@@ -41,6 +41,7 @@ import 
org.apache.spark.sql.execution.datasources.json.JsonFileFormat
 import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat
 import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
 import org.apache.spark.sql.execution.streaming._
+import 
org.apache.spark.sql.execution.streaming.sources.TextSocketSourceProvider
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.sources._
 import org.apache.spark.sql.streaming.OutputMode
@@ -563,6 +564,7 @@ object DataSource extends Logging {
 val libsvm = "org.apache.spark.ml.source.libsvm.LibSVMFileFormat"
 val orc = "org.apache.spark.sql.hive.orc.OrcFileFormat"
 val nativeOrc = classOf[OrcFileFormat].getCanonicalName
+val socket = classOf[TextSocketSourceProvider].getCanonicalName
 
 Map(
   "org.apache.spark.sql.jdbc" -> jdbc,
@@ -583,7 +585,8 @@ object DataSource extends Logging {
   "org.apache.spark.sql.execution.datasources.orc" -> nativeOrc,
   "org.apache.spark.ml.source.libsvm.DefaultSource" -> libsvm,
   "org.apache.spark.ml.source.libsvm" -> libsvm,
-  "com.databricks.spark.csv" -> csv
+  "com.databricks.spark.csv" -> csv,
+  "org.apache.spark.sql.execution.streaming.TextSocketSourceProvider" -> 
socket
 )
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/707e6506/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/socket.scala
--
diff 

spark git commit: [SPARK-23491][SS] Remove explicit job cancellation from ContinuousExecution reconfiguring

2018-02-26 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master 185f5bc7d -> 7ec83658f


[SPARK-23491][SS] Remove explicit job cancellation from ContinuousExecution 
reconfiguring

## What changes were proposed in this pull request?

Remove queryExecutionThread.interrupt() from ContinuousExecution. As detailed 
in the JIRA, interrupting the thread is only relevant in the microbatch case; 
for continuous processing the query execution can quickly clean itself up 
without.

## How was this patch tested?

existing tests

Author: Jose Torres 

Closes #20622 from jose-torres/SPARK-23441.


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

Branch: refs/heads/master
Commit: 7ec83658fbc88505dfc2d8a6f76e90db747f1292
Parents: 185f5bc
Author: Jose Torres 
Authored: Mon Feb 26 11:28:44 2018 -0800
Committer: Tathagata Das 
Committed: Mon Feb 26 11:28:44 2018 -0800

--
 .../streaming/continuous/ContinuousExecution.scala| 10 --
 1 file changed, 8 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/7ec83658/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 2c1d6c5..daebd1d 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
@@ -236,9 +236,7 @@ class ContinuousExecution(
 startTrigger()
 
 if (reader.needsReconfiguration() && state.compareAndSet(ACTIVE, 
RECONFIGURING)) {
-  stopSources()
   if (queryExecutionThread.isAlive) {
-sparkSession.sparkContext.cancelJobGroup(runId.toString)
 queryExecutionThread.interrupt()
   }
   false
@@ -266,12 +264,20 @@ class ContinuousExecution(
 SQLExecution.withNewExecutionId(
   sparkSessionForQuery, lastExecution)(lastExecution.toRdd)
   }
+} catch {
+  case t: Throwable
+  if StreamExecution.isInterruptionException(t) && state.get() == 
RECONFIGURING =>
+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)
 }
   }
 


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



spark git commit: [SPARK-23408][SS] Synchronize successive AddData actions in Streaming*JoinSuite

2018-02-23 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master 049f243c5 -> 855ce13d0


[SPARK-23408][SS] Synchronize successive AddData actions in Streaming*JoinSuite

**The best way to review this PR is to ignore whitespace/indent changes. Use 
this link - https://github.com/apache/spark/pull/20650/files?w=1**

## What changes were proposed in this pull request?

The stream-stream join tests add data to multiple sources and expect it all to 
show up in the next batch. But there's a race condition; the new batch might 
trigger when only one of the AddData actions has been reached.

Prior attempt to solve this issue by jose-torres in #20646 attempted to 
simultaneously synchronize on all memory sources together when consecutive 
AddData was found in the actions. However, this carries the risk of deadlock as 
well as unintended modification of stress tests (see the above PR for a 
detailed explanation). Instead, this PR attempts the following.

- A new action called `StreamProgressBlockedActions` that allows multiple 
actions to be executed while the streaming query is blocked from making 
progress. This allows data to be added to multiple sources that are made 
visible simultaneously in the next batch.
- An alias of `StreamProgressBlockedActions` called `MultiAddData` is 
explicitly used in the `Streaming*JoinSuites` to add data to two memory sources 
simultaneously.

This should avoid unintentional modification of the stress tests (or any other 
test for that matter) while making sure that the flaky tests are deterministic.

## How was this patch tested?
Modified test cases in `Streaming*JoinSuites` where there are consecutive 
`AddData` actions.

Author: Tathagata Das <tathagata.das1...@gmail.com>

Closes #20650 from tdas/SPARK-23408.


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

Branch: refs/heads/master
Commit: 855ce13d045569b7b16fdc7eee9c981f4ff3a545
Parents: 049f243
Author: Tathagata Das <tathagata.das1...@gmail.com>
Authored: Fri Feb 23 12:40:58 2018 -0800
Committer: Tathagata Das <tathagata.das1...@gmail.com>
Committed: Fri Feb 23 12:40:58 2018 -0800

--
 .../streaming/MicroBatchExecution.scala |  10 +
 .../apache/spark/sql/streaming/StreamTest.scala | 472 ++-
 .../sql/streaming/StreamingJoinSuite.scala  |  54 +--
 3 files changed, 284 insertions(+), 252 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/855ce13d/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala
--
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 8465501..6bd0397 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
@@ -504,6 +504,16 @@ class MicroBatchExecution(
 }
   }
 
+  /** Execute a function while locking the stream from making an progress */
+  private[sql] def withProgressLocked(f: => Unit): Unit = {
+awaitProgressLock.lock()
+try {
+  f
+} finally {
+  awaitProgressLock.unlock()
+}
+  }
+
   private def toJava(scalaOption: Option[OffsetV2]): Optional[OffsetV2] = {
 Optional.ofNullable(scalaOption.orNull)
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/855ce13d/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala
index 159dd0e..08f722e 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala
@@ -102,6 +102,19 @@ trait StreamTest extends QueryTest with SharedSQLContext 
with TimeLimits with Be
   AddDataMemory(source, data)
   }
 
+  /**
+   * Adds data to multiple memory streams such that all the data will be made 
visible in the
+   * same batch. This is applicable only to MicroBatchExecution, as this 
coordination cannot be
+   * performed at the driver in ContinuousExecutions.
+   */
+  object MultiAddData {
+def apply[A]
+  (source1: MemoryStream[A], data1: A*)(source2: MemoryStream[A], data2: 
A*): StreamAction = {
+  val actions = Seq(AddDataMemory(source1, data1), AddDa

spark git commit: [SPARK-23484][SS] Fix possible race condition in KafkaContinuousReader

2018-02-21 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/branch-2.3 3e7269eb9 -> 373ac642f


[SPARK-23484][SS] Fix possible race condition in KafkaContinuousReader

## What changes were proposed in this pull request?

var `KafkaContinuousReader.knownPartitions` should be threadsafe as it is 
accessed from multiple threads - the query thread at the time of reader factory 
creation, and the epoch tracking thread at the time of `needsReconfiguration`.

## How was this patch tested?

Existing tests.

Author: Tathagata Das <tathagata.das1...@gmail.com>

Closes #20655 from tdas/SPARK-23484.

(cherry picked from commit 3fd0ccb13fea44727d970479af1682ef00592147)
Signed-off-by: Tathagata Das <tathagata.das1...@gmail.com>


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

Branch: refs/heads/branch-2.3
Commit: 373ac642fc145527bdfccae046d0e98f105ee7b3
Parents: 3e7269e
Author: Tathagata Das <tathagata.das1...@gmail.com>
Authored: Wed Feb 21 14:56:13 2018 -0800
Committer: Tathagata Das <tathagata.das1...@gmail.com>
Committed: Wed Feb 21 14:56:23 2018 -0800

--
 .../org/apache/spark/sql/kafka010/KafkaContinuousReader.scala  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/373ac642/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReader.scala
--
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 b049a05..a269a50 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
@@ -66,7 +66,7 @@ class KafkaContinuousReader(
   // Initialized when creating reader factories. If this diverges from the 
partitions at the latest
   // offsets, we need to reconfigure.
   // Exposed outside this object only for unit tests.
-  private[sql] var knownPartitions: Set[TopicPartition] = _
+  @volatile private[sql] var knownPartitions: Set[TopicPartition] = _
 
   override def readSchema: StructType = KafkaOffsetReader.kafkaSchema
 


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



spark git commit: [SPARK-23484][SS] Fix possible race condition in KafkaContinuousReader

2018-02-21 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master e836c27ce -> 3fd0ccb13


[SPARK-23484][SS] Fix possible race condition in KafkaContinuousReader

## What changes were proposed in this pull request?

var `KafkaContinuousReader.knownPartitions` should be threadsafe as it is 
accessed from multiple threads - the query thread at the time of reader factory 
creation, and the epoch tracking thread at the time of `needsReconfiguration`.

## How was this patch tested?

Existing tests.

Author: Tathagata Das <tathagata.das1...@gmail.com>

Closes #20655 from tdas/SPARK-23484.


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

Branch: refs/heads/master
Commit: 3fd0ccb13fea44727d970479af1682ef00592147
Parents: e836c27
Author: Tathagata Das <tathagata.das1...@gmail.com>
Authored: Wed Feb 21 14:56:13 2018 -0800
Committer: Tathagata Das <tathagata.das1...@gmail.com>
Committed: Wed Feb 21 14:56:13 2018 -0800

--
 .../org/apache/spark/sql/kafka010/KafkaContinuousReader.scala  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/3fd0ccb1/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReader.scala
--
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 97a0f66..ecd1170 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
@@ -66,7 +66,7 @@ class KafkaContinuousReader(
   // Initialized when creating reader factories. If this diverges from the 
partitions at the latest
   // offsets, we need to reconfigure.
   // Exposed outside this object only for unit tests.
-  private[sql] var knownPartitions: Set[TopicPartition] = _
+  @volatile private[sql] var knownPartitions: Set[TopicPartition] = _
 
   override def readSchema: StructType = KafkaOffsetReader.kafkaSchema
 


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



spark git commit: [SPARK-23454][SS][DOCS] Added trigger information to the Structured Streaming programming guide

2018-02-20 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master 6d398c05c -> 601d653bf


[SPARK-23454][SS][DOCS] Added trigger information to the Structured Streaming 
programming guide

## What changes were proposed in this pull request?

- Added clear information about triggers
- Made the semantics guarantees of watermarks more clear for streaming 
aggregations and stream-stream joins.

## 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.

Author: Tathagata Das <tathagata.das1...@gmail.com>

Closes #20631 from tdas/SPARK-23454.


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

Branch: refs/heads/master
Commit: 601d653bff9160db8477f86d961e609fc2190237
Parents: 6d398c0
Author: Tathagata Das <tathagata.das1...@gmail.com>
Authored: Tue Feb 20 18:16:10 2018 -0800
Committer: Tathagata Das <tathagata.das1...@gmail.com>
Committed: Tue Feb 20 18:16:10 2018 -0800

--
 docs/structured-streaming-programming-guide.md | 214 +++-
 1 file changed, 207 insertions(+), 7 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/601d653b/docs/structured-streaming-programming-guide.md
--
diff --git a/docs/structured-streaming-programming-guide.md 
b/docs/structured-streaming-programming-guide.md
index 48d6d0b..9a83f15 100644
--- a/docs/structured-streaming-programming-guide.md
+++ b/docs/structured-streaming-programming-guide.md
@@ -904,7 +904,7 @@ windowedCounts <- count(
 
 
 
-### Handling Late Data and Watermarking
+ Handling Late Data and Watermarking
 Now consider what happens if one of the events arrives late to the application.
 For example, say, a word generated at 12:04 (i.e. event time) could be 
received by 
 the application at 12:11. The application should use the time 12:04 instead of 
12:11
@@ -925,7 +925,9 @@ specifying the event time column and the threshold on how 
late the data is expec
 event time. For a specific window starting at time `T`, the engine will 
maintain state and allow late
 data to update the state until `(max event time seen by the engine - late 
threshold > T)`. 
 In other words, late data within the threshold will be aggregated, 
-but data later than the threshold will be dropped. Let's understand this with 
an example. We can 
+but data later than the threshold will start getting dropped
+(see [later]((#semantic-guarantees-of-aggregation-with-watermarking))
+in the section for the exact guarantees). Let's understand this with an 
example. We can
 easily define watermarking on the previous example using `withWatermark()` as 
shown below.
 
 
@@ -1031,7 +1033,9 @@ then drops intermediate state of a window < watermark, 
and appends the final
 counts to the Result Table/sink. For example, the final counts of window 
`12:00 - 12:10` is 
 appended to the Result Table only after the watermark is updated to `12:11`. 
 
-**Conditions for watermarking to clean aggregation state**
+# Conditions for watermarking to clean aggregation state
+{:.no_toc}
+
 It is important to note that the following conditions must be satisfied for 
the watermarking to 
 clean the state in aggregation queries *(as of Spark 2.1.1, subject to change 
in the future)*.
 
@@ -1051,6 +1055,16 @@ from the aggregation column.
 For example, `df.groupBy("time").count().withWatermark("time", "1 min")` is 
invalid in Append 
 output mode.
 
+# Semantic Guarantees of Aggregation with Watermarking
+{:.no_toc}
+
+- A watermark delay (set with `withWatermark`) of "2 hours" guarantees that 
the engine will never
+drop any data that is less than 2 hours delayed. In other words, any data less 
than 2 hours behind
+(in terms of event-time) the latest data processed till then is guaranteed to 
be aggregated.
+
+- However, the guarantee is strict only in one direction. Data delayed by more 
than 2 hours is
+not guaranteed to be dropped; it may or may not get aggregated. More delayed 
is the data, less
+likely is the engine going to process it.
 
 ### Join Operations
 Structured Streaming supports joining a streaming Dataset/DataFrame with a 
static Dataset/DataFrame
@@ -1062,7 +1076,7 @@ Dataset/DataFrame will be the exactly the same as if it 
was with a static Datase
 containing the same data in the stream.
 
 
- Stream-static joins
+ Stream-static Joins
 
 Since the introducti

spark git commit: [SPARK-23454][SS][DOCS] Added trigger information to the Structured Streaming programming guide

2018-02-20 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/branch-2.3 1d78f03ae -> 3e7269eb9


[SPARK-23454][SS][DOCS] Added trigger information to the Structured Streaming 
programming guide

## What changes were proposed in this pull request?

- Added clear information about triggers
- Made the semantics guarantees of watermarks more clear for streaming 
aggregations and stream-stream joins.

## 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.

Author: Tathagata Das <tathagata.das1...@gmail.com>

Closes #20631 from tdas/SPARK-23454.

(cherry picked from commit 601d653bff9160db8477f86d961e609fc2190237)
Signed-off-by: Tathagata Das <tathagata.das1...@gmail.com>


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

Branch: refs/heads/branch-2.3
Commit: 3e7269eb904b591883300d7433e5c99be0b3b5b3
Parents: 1d78f03
Author: Tathagata Das <tathagata.das1...@gmail.com>
Authored: Tue Feb 20 18:16:10 2018 -0800
Committer: Tathagata Das <tathagata.das1...@gmail.com>
Committed: Tue Feb 20 18:16:23 2018 -0800

--
 docs/structured-streaming-programming-guide.md | 214 +++-
 1 file changed, 207 insertions(+), 7 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/3e7269eb/docs/structured-streaming-programming-guide.md
--
diff --git a/docs/structured-streaming-programming-guide.md 
b/docs/structured-streaming-programming-guide.md
index 48d6d0b..9a83f15 100644
--- a/docs/structured-streaming-programming-guide.md
+++ b/docs/structured-streaming-programming-guide.md
@@ -904,7 +904,7 @@ windowedCounts <- count(
 
 
 
-### Handling Late Data and Watermarking
+ Handling Late Data and Watermarking
 Now consider what happens if one of the events arrives late to the application.
 For example, say, a word generated at 12:04 (i.e. event time) could be 
received by 
 the application at 12:11. The application should use the time 12:04 instead of 
12:11
@@ -925,7 +925,9 @@ specifying the event time column and the threshold on how 
late the data is expec
 event time. For a specific window starting at time `T`, the engine will 
maintain state and allow late
 data to update the state until `(max event time seen by the engine - late 
threshold > T)`. 
 In other words, late data within the threshold will be aggregated, 
-but data later than the threshold will be dropped. Let's understand this with 
an example. We can 
+but data later than the threshold will start getting dropped
+(see [later]((#semantic-guarantees-of-aggregation-with-watermarking))
+in the section for the exact guarantees). Let's understand this with an 
example. We can
 easily define watermarking on the previous example using `withWatermark()` as 
shown below.
 
 
@@ -1031,7 +1033,9 @@ then drops intermediate state of a window < watermark, 
and appends the final
 counts to the Result Table/sink. For example, the final counts of window 
`12:00 - 12:10` is 
 appended to the Result Table only after the watermark is updated to `12:11`. 
 
-**Conditions for watermarking to clean aggregation state**
+# Conditions for watermarking to clean aggregation state
+{:.no_toc}
+
 It is important to note that the following conditions must be satisfied for 
the watermarking to 
 clean the state in aggregation queries *(as of Spark 2.1.1, subject to change 
in the future)*.
 
@@ -1051,6 +1055,16 @@ from the aggregation column.
 For example, `df.groupBy("time").count().withWatermark("time", "1 min")` is 
invalid in Append 
 output mode.
 
+# Semantic Guarantees of Aggregation with Watermarking
+{:.no_toc}
+
+- A watermark delay (set with `withWatermark`) of "2 hours" guarantees that 
the engine will never
+drop any data that is less than 2 hours delayed. In other words, any data less 
than 2 hours behind
+(in terms of event-time) the latest data processed till then is guaranteed to 
be aggregated.
+
+- However, the guarantee is strict only in one direction. Data delayed by more 
than 2 hours is
+not guaranteed to be dropped; it may or may not get aggregated. More delayed 
is the data, less
+likely is the engine going to process it.
 
 ### Join Operations
 Structured Streaming supports joining a streaming Dataset/DataFrame with a 
static Dataset/DataFrame
@@ -1062,7 +1076,7 @@ Dataset/DataFrame will be the exactly the sam

[1/2] spark git commit: [SPARK-23362][SS] Migrate Kafka Microbatch source to v2

2018-02-16 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master c5857e496 -> 0a73aa31f


http://git-wip-us.apache.org/repos/asf/spark/blob/0a73aa31/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala
--
diff --git 
a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala
 
b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala
deleted file mode 100644
index 02c8764..000
--- 
a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala
+++ /dev/null
@@ -1,1122 +0,0 @@
-/*
- * 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.kafka010
-
-import java.io._
-import java.nio.charset.StandardCharsets.UTF_8
-import java.nio.file.{Files, Paths}
-import java.util.{Locale, Properties}
-import java.util.concurrent.ConcurrentLinkedQueue
-import java.util.concurrent.atomic.AtomicInteger
-
-import scala.collection.mutable
-import scala.util.Random
-
-import org.apache.kafka.clients.producer.RecordMetadata
-import org.apache.kafka.common.TopicPartition
-import org.scalatest.concurrent.PatienceConfiguration.Timeout
-import org.scalatest.time.SpanSugar._
-
-import org.apache.spark.SparkContext
-import org.apache.spark.sql.{Dataset, ForeachWriter}
-import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
-import org.apache.spark.sql.execution.streaming._
-import org.apache.spark.sql.execution.streaming.continuous.ContinuousExecution
-import org.apache.spark.sql.functions.{count, window}
-import org.apache.spark.sql.kafka010.KafkaSourceProvider._
-import org.apache.spark.sql.streaming.{ProcessingTime, StreamTest}
-import org.apache.spark.sql.streaming.util.StreamManualClock
-import org.apache.spark.sql.test.{SharedSQLContext, TestSparkSession}
-import org.apache.spark.util.Utils
-
-abstract class KafkaSourceTest extends StreamTest with SharedSQLContext {
-
-  protected var testUtils: KafkaTestUtils = _
-
-  override val streamingTimeout = 30.seconds
-
-  protected val brokerProps = Map[String, Object]()
-
-  override def beforeAll(): Unit = {
-super.beforeAll()
-testUtils = new KafkaTestUtils(brokerProps)
-testUtils.setup()
-  }
-
-  override def afterAll(): Unit = {
-if (testUtils != null) {
-  testUtils.teardown()
-  testUtils = null
-}
-super.afterAll()
-  }
-
-  protected def makeSureGetOffsetCalled = AssertOnQuery { q =>
-// Because KafkaSource's initialPartitionOffsets is set lazily, we need to 
make sure
-// its "getOffset" is called before pushing any data. Otherwise, because 
of the race condition,
-// we don't know which data should be fetched when `startingOffsets` is 
latest.
-q match {
-  case c: ContinuousExecution => c.awaitEpoch(0)
-  case m: MicroBatchExecution => m.processAllAvailable()
-}
-true
-  }
-
-  protected def setTopicPartitions(topic: String, newCount: Int, query: 
StreamExecution) : Unit = {
-testUtils.addPartitions(topic, newCount)
-  }
-
-  /**
-   * Add data to Kafka.
-   *
-   * `topicAction` can be used to run actions for each topic before inserting 
data.
-   */
-  case class AddKafkaData(topics: Set[String], data: Int*)
-(implicit ensureDataInMultiplePartition: Boolean = false,
-  concurrent: Boolean = false,
-  message: String = "",
-  topicAction: (String, Option[Int]) => Unit = (_, _) => {}) extends 
AddData {
-
-override def addData(query: Option[StreamExecution]): 
(BaseStreamingSource, Offset) = {
-  query match {
-// Make sure no Spark job is running when deleting a topic
-case Some(m: MicroBatchExecution) => m.processAllAvailable()
-case _ =>
-  }
-
-  val existingTopics = testUtils.getAllTopicsAndPartitionSize().toMap
-  val newTopics = topics.diff(existingTopics.keySet)
-  for (newTopic <- newTopics) {
-topicAction(newTopic, None)
-  }
-  for (existingTopicPartitions <- existingTopics) {
-topicAction(existingTopicPartitions._1, 
Some(existingTopicPartitions._2))
-  }
-
-  require(
-query.nonEmpty,
-

[2/2] spark git commit: [SPARK-23362][SS] Migrate Kafka Microbatch source to v2

2018-02-16 Thread tdas
[SPARK-23362][SS] Migrate Kafka Microbatch source to v2

## What changes were proposed in this pull request?
Migrating KafkaSource (with data source v1) to KafkaMicroBatchReader (with data 
source v2).

Performance comparison:
In a unit test with in-process Kafka broker, I tested the read throughput of V1 
and V2 using 20M records in a single partition. They were comparable.

## How was this patch tested?
Existing tests, few modified to be better tests than the existing ones.

Author: Tathagata Das <tathagata.das1...@gmail.com>

Closes #20554 from tdas/SPARK-23362.


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

Branch: refs/heads/master
Commit: 0a73aa31f41c83503d5d99eff3c9d7b406014ab3
Parents: c5857e4
Author: Tathagata Das <tathagata.das1...@gmail.com>
Authored: Fri Feb 16 14:30:19 2018 -0800
Committer: Tathagata Das <tathagata.das1...@gmail.com>
Committed: Fri Feb 16 14:30:19 2018 -0800

--
 dev/.rat-excludes   |1 +
 .../sql/kafka010/CachedKafkaConsumer.scala  |2 +-
 .../sql/kafka010/KafkaContinuousReader.scala|   29 +-
 .../sql/kafka010/KafkaMicroBatchReader.scala|  403 ++
 .../KafkaRecordToUnsafeRowConverter.scala   |   52 +
 .../apache/spark/sql/kafka010/KafkaSource.scala |   19 +-
 .../sql/kafka010/KafkaSourceProvider.scala  |   70 +-
 ...fka-source-initial-offset-future-version.bin |2 +
 ...afka-source-initial-offset-version-2.1.0.bin |2 +-
 .../kafka010/KafkaMicroBatchSourceSuite.scala   | 1222 ++
 .../spark/sql/kafka010/KafkaSourceSuite.scala   | 1122 
 .../org/apache/spark/sql/internal/SQLConf.scala |   15 +-
 .../streaming/MicroBatchExecution.scala |   20 +-
 13 files changed, 1786 insertions(+), 1173 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/0a73aa31/dev/.rat-excludes
--
diff --git a/dev/.rat-excludes b/dev/.rat-excludes
index 243fbe3..9552d00 100644
--- a/dev/.rat-excludes
+++ b/dev/.rat-excludes
@@ -105,3 +105,4 @@ META-INF/*
 spark-warehouse
 structured-streaming/*
 kafka-source-initial-offset-version-2.1.0.bin
+kafka-source-initial-offset-future-version.bin

http://git-wip-us.apache.org/repos/asf/spark/blob/0a73aa31/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala
--
diff --git 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala
 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala
index 90ed7b1..e97881c 100644
--- 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala
+++ 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala
@@ -27,7 +27,7 @@ import org.apache.kafka.common.TopicPartition
 
 import org.apache.spark.{SparkEnv, SparkException, TaskContext}
 import org.apache.spark.internal.Logging
-import org.apache.spark.sql.kafka010.KafkaSource._
+import org.apache.spark.sql.kafka010.KafkaSourceProvider._
 import org.apache.spark.util.UninterruptibleThread
 
 

http://git-wip-us.apache.org/repos/asf/spark/blob/0a73aa31/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReader.scala
--
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 b049a05..97a0f66 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
@@ -29,7 +29,7 @@ import org.apache.spark.sql.SparkSession
 import org.apache.spark.sql.catalyst.expressions.UnsafeRow
 import org.apache.spark.sql.catalyst.expressions.codegen.{BufferHolder, 
UnsafeRowWriter}
 import org.apache.spark.sql.catalyst.util.DateTimeUtils
-import 
org.apache.spark.sql.kafka010.KafkaSource.{INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_FALSE,
 INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_TRUE}
+import 
org.apache.spark.sql.kafka010.KafkaSourceProvider.{INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_FALSE,
 INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_TRUE}
 import org.apache.spark.sql.sources.v2.reader._
 import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousDataReader, 
ContinuousReader, Offset, Partit

spark git commit: [SPARK-23406][SS] Enable stream-stream self-joins

2018-02-14 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master 400a1d9e2 -> 658d9d9d7


[SPARK-23406][SS] Enable stream-stream self-joins

## What changes were proposed in this pull request?

Solved two bugs to enable stream-stream self joins.

### Incorrect analysis due to missing MultiInstanceRelation trait
Streaming leaf nodes did not extend MultiInstanceRelation, which is necessary 
for the catalyst analyzer to convert the self-join logical plan DAG into a tree 
(by creating new instances of the leaf relations). This was causing the error 
`Failure when resolving conflicting references in Join:` (see JIRA for details).

### Incorrect attribute rewrite when splicing batch plans in MicroBatchExecution
When splicing the source's batch plan into the streaming plan (by replacing the 
StreamingExecutionPlan), we were rewriting the attribute reference in the 
streaming plan with the new attribute references from the batch plan. This was 
incorrectly handling the scenario when multiple StreamingExecutionRelation 
point to the same source, and therefore eventually point to the same batch plan 
returned by the source. Here is an example query, and its corresponding plan 
transformations.
```
val df = input.toDF
val join =
  df.select('value % 5 as "key", 'value).join(
df.select('value % 5 as "key", 'value), "key")
```
Streaming logical plan before splicing the batch plan
```
Project [key#6, value#1, value#12]
+- Join Inner, (key#6 = key#9)
   :- Project [(value#1 % 5) AS key#6, value#1]
   :  +- StreamingExecutionRelation Memory[#1], value#1
   +- Project [(value#12 % 5) AS key#9, value#12]
  +- StreamingExecutionRelation Memory[#1], value#12  // two different 
leaves pointing to same source
```
Batch logical plan after splicing the batch plan and before rewriting
```
Project [key#6, value#1, value#12]
+- Join Inner, (key#6 = key#9)
   :- Project [(value#1 % 5) AS key#6, value#1]
   :  +- LocalRelation [value#66]   // replaces 
StreamingExecutionRelation Memory[#1], value#1
   +- Project [(value#12 % 5) AS key#9, value#12]
  +- LocalRelation [value#66]   // replaces 
StreamingExecutionRelation Memory[#1], value#12
```
Batch logical plan after rewriting the attributes. Specifically, for spliced, 
the new output attributes (value#66) replace the earlier output attributes 
(value#12, and value#1, one for each StreamingExecutionRelation).
```
Project [key#6, value#66, value#66]   // both value#1 and value#12 replaces 
by value#66
+- Join Inner, (key#6 = key#9)
   :- Project [(value#66 % 5) AS key#6, value#66]
   :  +- LocalRelation [value#66]
   +- Project [(value#66 % 5) AS key#9, value#66]
  +- LocalRelation [value#66]
```
This causes the optimizer to eliminate value#66 from one side of the join.
```
Project [key#6, value#66, value#66]
+- Join Inner, (key#6 = key#9)
   :- Project [(value#66 % 5) AS key#6, value#66]
   :  +- LocalRelation [value#66]
   +- Project [(value#66 % 5) AS key#9]   // this does not generate value, 
incorrect join results
  +- LocalRelation [value#66]
```

**Solution**: Instead of rewriting attributes, use a Project to introduce 
aliases between the output attribute references and the new reference generated 
by the spliced plans. The analyzer and optimizer will take care of the rest.
```
Project [key#6, value#1, value#12]
+- Join Inner, (key#6 = key#9)
   :- Project [(value#1 % 5) AS key#6, value#1]
   :  +- Project [value#66 AS value#1]   // solution: project with aliases
   : +- LocalRelation [value#66]
   +- Project [(value#12 % 5) AS key#9, value#12]
  +- Project [value#66 AS value#12]// solution: project with aliases
 +- LocalRelation [value#66]
```

## How was this patch tested?
New unit test

Author: Tathagata Das <tathagata.das1...@gmail.com>

Closes #20598 from tdas/SPARK-23406.


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

Branch: refs/heads/master
Commit: 658d9d9d785a30857bf35d164e6cbbd9799d6959
Parents: 400a1d9
Author: Tathagata Das <tathagata.das1...@gmail.com>
Authored: Wed Feb 14 14:27:02 2018 -0800
Committer: Tathagata Das <tathagata.das1...@gmail.com>
Committed: Wed Feb 14 14:27:02 2018 -0800

--
 .../streaming/MicroBatchExecution.scala | 16 ++---
 .../execution/streaming/StreamingRelation.scala | 20 +++-
 .../sql/streaming/StreamingJoinSuite.scala  | 25 +++-
 3 files changed, 45 insertions(+), 16 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/658d9d9d/sql/core/src/main/scala/org/apache/spar

spark git commit: [SPARK-23092][SQL] Migrate MemoryStream to DataSourceV2 APIs

2018-02-07 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master 9841ae031 -> 30295bf5a


[SPARK-23092][SQL] Migrate MemoryStream to DataSourceV2 APIs

## What changes were proposed in this pull request?

This PR migrates the MemoryStream to DataSourceV2 APIs.

One additional change is in the reported keys in 
StreamingQueryProgress.durationMs. "getOffset" and "getBatch" replaced with 
"setOffsetRange" and "getEndOffset" as tracking these make more sense. Unit 
tests changed accordingly.

## How was this patch tested?
Existing unit tests, few updated unit tests.

Author: Tathagata Das <tathagata.das1...@gmail.com>
Author: Burak Yavuz <brk...@gmail.com>

Closes #20445 from tdas/SPARK-23092.


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

Branch: refs/heads/master
Commit: 30295bf5a6754d0ae43334f7bf00e7a29ed0f1af
Parents: 9841ae0
Author: Tathagata Das <tathagata.das1...@gmail.com>
Authored: Wed Feb 7 15:22:53 2018 -0800
Committer: Tathagata Das <tathagata.das1...@gmail.com>
Committed: Wed Feb 7 15:22:53 2018 -0800

--
 .../sql/execution/streaming/LongOffset.scala|   4 +-
 .../streaming/MicroBatchExecution.scala |  27 ++--
 .../spark/sql/execution/streaming/memory.scala  | 132 +++
 .../streaming/sources/RateStreamSourceV2.scala  |   2 +-
 .../execution/streaming/ForeachSinkSuite.scala  |  55 +++-
 .../spark/sql/streaming/StreamSuite.scala   |   8 +-
 .../apache/spark/sql/streaming/StreamTest.scala |   2 +-
 .../streaming/StreamingQueryListenerSuite.scala |   5 +-
 .../sql/streaming/StreamingQuerySuite.scala |  70 ++
 9 files changed, 171 insertions(+), 134 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/30295bf5/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/LongOffset.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/LongOffset.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/LongOffset.scala
index 5f0b195..3ff5b86 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/LongOffset.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/LongOffset.scala
@@ -17,10 +17,12 @@
 
 package org.apache.spark.sql.execution.streaming
 
+import org.apache.spark.sql.sources.v2.reader.streaming.{Offset => OffsetV2}
+
 /**
  * A simple offset for sources that produce a single linear stream of data.
  */
-case class LongOffset(offset: Long) extends Offset {
+case class LongOffset(offset: Long) extends OffsetV2 {
 
   override val json = offset.toString
 

http://git-wip-us.apache.org/repos/asf/spark/blob/30295bf5/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala
--
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 d9aa857..045d2b4 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
@@ -270,16 +270,17 @@ class MicroBatchExecution(
 }
   case s: MicroBatchReader =>
 updateStatusMessage(s"Getting offsets from $s")
-reportTimeTaken("getOffset") {
-// Once v1 streaming source execution is gone, we can refactor 
this away.
-// For now, we set the range here to get the source to infer the 
available end offset,
-// get that offset, and then set the range again when we later 
execute.
-s.setOffsetRange(
-  toJava(availableOffsets.get(s).map(off => 
s.deserializeOffset(off.json))),
-  Optional.empty())
-
-  (s, Some(s.getEndOffset))
+reportTimeTaken("setOffsetRange") {
+  // Once v1 streaming source execution is gone, we can refactor 
this away.
+  // For now, we set the range here to get the source to infer the 
available end offset,
+  // get that offset, and then set the range again when we later 
execute.
+  s.setOffsetRange(
+toJava(availableOffsets.get(s).map(off => 
s.deserializeOffset(off.json))),
+Optional.empty())
 }
+
+val currentOffset = reportTimeTaken("getEn

spark git commit: [SPARK-23064][SS][DOCS] Stream-stream joins Documentation - follow up

2018-02-02 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/branch-2.3 e5e9f9a43 -> 56eb9a310


[SPARK-23064][SS][DOCS] Stream-stream joins Documentation - follow up

## What changes were proposed in this pull request?
Further clarification of caveats in using stream-stream outer joins.

## How was this patch tested?
N/A

Author: Tathagata Das <tathagata.das1...@gmail.com>

Closes #20494 from tdas/SPARK-23064-2.

(cherry picked from commit eaf35de2471fac4337dd2920026836d52b1ec847)
Signed-off-by: Tathagata Das <tathagata.das1...@gmail.com>


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

Branch: refs/heads/branch-2.3
Commit: 56eb9a310217a5372bdba1e24e4af0d4de1829ca
Parents: e5e9f9a
Author: Tathagata Das <tathagata.das1...@gmail.com>
Authored: Fri Feb 2 17:37:51 2018 -0800
Committer: Tathagata Das <tathagata.das1...@gmail.com>
Committed: Fri Feb 2 17:38:07 2018 -0800

--
 docs/structured-streaming-programming-guide.md | 14 --
 1 file changed, 12 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/56eb9a31/docs/structured-streaming-programming-guide.md
--
diff --git a/docs/structured-streaming-programming-guide.md 
b/docs/structured-streaming-programming-guide.md
index 62589a6..48d6d0b 100644
--- a/docs/structured-streaming-programming-guide.md
+++ b/docs/structured-streaming-programming-guide.md
@@ -1346,10 +1346,20 @@ joined <- join(
 
 
 
-However, note that the outer NULL results will be generated with a delay 
(depends on the specified
-watermark delay and the time range condition) because the engine has to wait 
for that long to ensure
+
+There are a few points to note regarding outer joins.
+
+- *The outer NULL results will be generated with a delay that depends on the 
specified watermark
+delay and the time range condition.* This is because the engine has to wait 
for that long to ensure
 there were no matches and there will be no more matches in future.
 
+- In the current implementation in the micro-batch engine, watermarks are 
advanced at the end of a
+micro-batch, and the next micro-batch uses the updated watermark to clean up 
state and output
+outer results. Since we trigger a micro-batch only when there is new data to 
be processed, the
+generation of the outer result may get delayed if there no new data being 
received in the stream.
+*In short, if any of the two input streams being joined does not receive data 
for a while, the
+outer (both cases, left or right) output may get delayed.*
+
 # Support matrix for joins in streaming queries
 
 


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



spark git commit: [SPARK-23064][SS][DOCS] Stream-stream joins Documentation - follow up

2018-02-02 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master eefec93d1 -> eaf35de24


[SPARK-23064][SS][DOCS] Stream-stream joins Documentation - follow up

## What changes were proposed in this pull request?
Further clarification of caveats in using stream-stream outer joins.

## How was this patch tested?
N/A

Author: Tathagata Das <tathagata.das1...@gmail.com>

Closes #20494 from tdas/SPARK-23064-2.


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

Branch: refs/heads/master
Commit: eaf35de2471fac4337dd2920026836d52b1ec847
Parents: eefec93
Author: Tathagata Das <tathagata.das1...@gmail.com>
Authored: Fri Feb 2 17:37:51 2018 -0800
Committer: Tathagata Das <tathagata.das1...@gmail.com>
Committed: Fri Feb 2 17:37:51 2018 -0800

--
 docs/structured-streaming-programming-guide.md | 14 --
 1 file changed, 12 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/eaf35de2/docs/structured-streaming-programming-guide.md
--
diff --git a/docs/structured-streaming-programming-guide.md 
b/docs/structured-streaming-programming-guide.md
index 62589a6..48d6d0b 100644
--- a/docs/structured-streaming-programming-guide.md
+++ b/docs/structured-streaming-programming-guide.md
@@ -1346,10 +1346,20 @@ joined <- join(
 
 
 
-However, note that the outer NULL results will be generated with a delay 
(depends on the specified
-watermark delay and the time range condition) because the engine has to wait 
for that long to ensure
+
+There are a few points to note regarding outer joins.
+
+- *The outer NULL results will be generated with a delay that depends on the 
specified watermark
+delay and the time range condition.* This is because the engine has to wait 
for that long to ensure
 there were no matches and there will be no more matches in future.
 
+- In the current implementation in the micro-batch engine, watermarks are 
advanced at the end of a
+micro-batch, and the next micro-batch uses the updated watermark to clean up 
state and output
+outer results. Since we trigger a micro-batch only when there is new data to 
be processed, the
+generation of the outer result may get delayed if there no new data being 
received in the stream.
+*In short, if any of the two input streams being joined does not receive data 
for a while, the
+outer (both cases, left or right) output may get delayed.*
+
 # Support matrix for joins in streaming queries
 
 


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



spark git commit: [SPARK-23197][DSTREAMS] Increased timeouts to resolve flakiness

2018-01-23 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/branch-2.3 a23f6b13e -> 3316a9d71


[SPARK-23197][DSTREAMS] Increased timeouts to resolve flakiness

## What changes were proposed in this pull request?

Increased timeout from 50 ms to 300 ms (50 ms was really too low).

## How was this patch tested?
Multiple rounds of tests.

Author: Tathagata Das <tathagata.das1...@gmail.com>

Closes #20371 from tdas/SPARK-23197.

(cherry picked from commit 15adcc8273e73352e5e1c3fc9915c0b004ec4836)
Signed-off-by: Tathagata Das <tathagata.das1...@gmail.com>


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

Branch: refs/heads/branch-2.3
Commit: 3316a9d7104aece977384974cf61e5ec635ad350
Parents: a23f6b1
Author: Tathagata Das <tathagata.das1...@gmail.com>
Authored: Tue Jan 23 16:24:20 2018 -0800
Committer: Tathagata Das <tathagata.das1...@gmail.com>
Committed: Tue Jan 23 16:24:41 2018 -0800

--
 .../test/scala/org/apache/spark/streaming/ReceiverSuite.scala  | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/3316a9d7/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala
--
diff --git 
a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala 
b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala
index 145c48e..fc6218a 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala
@@ -105,13 +105,13 @@ class ReceiverSuite extends TestSuiteBase with TimeLimits 
with Serializable {
 assert(executor.errors.head.eq(exception))
 
 // Verify restarting actually stops and starts the receiver
-receiver.restart("restarting", null, 100)
-eventually(timeout(50 millis), interval(10 millis)) {
+receiver.restart("restarting", null, 600)
+eventually(timeout(300 millis), interval(10 millis)) {
   // receiver will be stopped async
   assert(receiver.isStopped)
   assert(receiver.onStopCalled)
 }
-eventually(timeout(1000 millis), interval(100 millis)) {
+eventually(timeout(1000 millis), interval(10 millis)) {
   // receiver will be started async
   assert(receiver.onStartCalled)
   assert(executor.isReceiverStarted)


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



spark git commit: [SPARK-23197][DSTREAMS] Increased timeouts to resolve flakiness

2018-01-23 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master 44cc4daf3 -> 15adcc827


[SPARK-23197][DSTREAMS] Increased timeouts to resolve flakiness

## What changes were proposed in this pull request?

Increased timeout from 50 ms to 300 ms (50 ms was really too low).

## How was this patch tested?
Multiple rounds of tests.

Author: Tathagata Das <tathagata.das1...@gmail.com>

Closes #20371 from tdas/SPARK-23197.


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

Branch: refs/heads/master
Commit: 15adcc8273e73352e5e1c3fc9915c0b004ec4836
Parents: 44cc4da
Author: Tathagata Das <tathagata.das1...@gmail.com>
Authored: Tue Jan 23 16:24:20 2018 -0800
Committer: Tathagata Das <tathagata.das1...@gmail.com>
Committed: Tue Jan 23 16:24:20 2018 -0800

--
 .../test/scala/org/apache/spark/streaming/ReceiverSuite.scala  | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/15adcc82/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala
--
diff --git 
a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala 
b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala
index 145c48e..fc6218a 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala
@@ -105,13 +105,13 @@ class ReceiverSuite extends TestSuiteBase with TimeLimits 
with Serializable {
 assert(executor.errors.head.eq(exception))
 
 // Verify restarting actually stops and starts the receiver
-receiver.restart("restarting", null, 100)
-eventually(timeout(50 millis), interval(10 millis)) {
+receiver.restart("restarting", null, 600)
+eventually(timeout(300 millis), interval(10 millis)) {
   // receiver will be stopped async
   assert(receiver.isStopped)
   assert(receiver.onStopCalled)
 }
-eventually(timeout(1000 millis), interval(100 millis)) {
+eventually(timeout(1000 millis), interval(10 millis)) {
   // receiver will be started async
   assert(receiver.onStartCalled)
   assert(executor.isReceiverStarted)


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



spark git commit: [SPARK-23142][SS][DOCS] Added docs for continuous processing

2018-01-18 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/branch-2.3 7057e310a -> acf3b70d1


[SPARK-23142][SS][DOCS] Added docs for continuous processing

## What changes were proposed in this pull request?

Added documentation for continuous processing. Modified two locations.
- Modified the overview to have a mention of Continuous Processing.
- Added a new section on Continuous Processing at the end.

![image](https://user-images.githubusercontent.com/663212/35083551-a3dd23f6-fbd4-11e7-9e7e-90866f131ca9.png)
![image](https://user-images.githubusercontent.com/663212/35083618-d844027c-fbd4-11e7-9fde-75992cc517bd.png)

## How was this patch tested?
N/A

Author: Tathagata Das <tathagata.das1...@gmail.com>

Closes #20308 from tdas/SPARK-23142.

(cherry picked from commit 4cd2ecc0c7222fef1337e04f1948333296c3be86)
Signed-off-by: Tathagata Das <tathagata.das1...@gmail.com>


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

Branch: refs/heads/branch-2.3
Commit: acf3b70d16cc4d2416b4ce3f42b3cf95836170ed
Parents: 7057e31
Author: Tathagata Das <tathagata.das1...@gmail.com>
Authored: Thu Jan 18 16:29:45 2018 -0800
Committer: Tathagata Das <tathagata.das1...@gmail.com>
Committed: Thu Jan 18 16:29:56 2018 -0800

--
 docs/structured-streaming-programming-guide.md | 98 -
 1 file changed, 97 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/acf3b70d/docs/structured-streaming-programming-guide.md
--
diff --git a/docs/structured-streaming-programming-guide.md 
b/docs/structured-streaming-programming-guide.md
index 1779a42..2ddba2f 100644
--- a/docs/structured-streaming-programming-guide.md
+++ b/docs/structured-streaming-programming-guide.md
@@ -10,7 +10,9 @@ title: Structured Streaming Programming Guide
 # Overview
 Structured Streaming is a scalable and fault-tolerant stream processing engine 
built on the Spark SQL engine. You can express your streaming computation the 
same way you would express a batch computation on static data. The Spark SQL 
engine will take care of running it incrementally and continuously and updating 
the final result as streaming data continues to arrive. You can use the 
[Dataset/DataFrame API](sql-programming-guide.html) in Scala, Java, Python or R 
to express streaming aggregations, event-time windows, stream-to-batch joins, 
etc. The computation is executed on the same optimized Spark SQL engine. 
Finally, the system ensures end-to-end exactly-once fault-tolerance guarantees 
through checkpointing and Write Ahead Logs. In short, *Structured Streaming 
provides fast, scalable, fault-tolerant, end-to-end exactly-once stream 
processing without the user having to reason about streaming.*
 
-In this guide, we are going to walk you through the programming model and the 
APIs. First, let's start with a simple example - a streaming word count.
+Internally, by default, Structured Streaming queries are processed using a 
*micro-batch processing* engine, which processes data streams as a series of 
small batch jobs thereby achieving end-to-end latencies as low as 100 
milliseconds and exactly-once fault-tolerance guarantees. However, since Spark 
2.3, we have introduced a new low-latency processing mode called **Continuous 
Processing**, which can achieve end-to-end latencies as low as 1 millisecond 
with at-least-once guarantees. Without changing the Dataset/DataFrame 
operations in your queries, you will be able choose the mode based on your 
application requirements. 
+
+In this guide, we are going to walk you through the programming model and the 
APIs. We are going to explain the concepts mostly using the default micro-batch 
processing model, and then [later](#continuous-processing-experimental) discuss 
Continuous Processing model. First, let's start with a simple example of a 
Structured Streaming query - a streaming word count.
 
 # Quick Example
 Let’s say you want to maintain a running word count of text data received 
from a data server listening on a TCP socket. Let’s see how you can express 
this using Structured Streaming. You can see the full code in
@@ -2434,6 +2436,100 @@ write.stream(aggDF, "memory", outputMode = "complete", 
checkpointLocation = "pat
 
 
 
+# Continuous Processing [Experimental]
+**Continuous processing** is a new, experimental streaming execution mode 
introduced in Spark 2.3 that enables low (~1 ms) end-to-end latency with 
at-least-once fault-tolerance guarantees. Compare this with the default 
*micro-batch processing* engine which can achieve exactly-once

spark git commit: [SPARK-23142][SS][DOCS] Added docs for continuous processing

2018-01-18 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master 5d7c4ba4d -> 4cd2ecc0c


[SPARK-23142][SS][DOCS] Added docs for continuous processing

## What changes were proposed in this pull request?

Added documentation for continuous processing. Modified two locations.
- Modified the overview to have a mention of Continuous Processing.
- Added a new section on Continuous Processing at the end.

![image](https://user-images.githubusercontent.com/663212/35083551-a3dd23f6-fbd4-11e7-9e7e-90866f131ca9.png)
![image](https://user-images.githubusercontent.com/663212/35083618-d844027c-fbd4-11e7-9fde-75992cc517bd.png)

## How was this patch tested?
N/A

Author: Tathagata Das <tathagata.das1...@gmail.com>

Closes #20308 from tdas/SPARK-23142.


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

Branch: refs/heads/master
Commit: 4cd2ecc0c7222fef1337e04f1948333296c3be86
Parents: 5d7c4ba
Author: Tathagata Das <tathagata.das1...@gmail.com>
Authored: Thu Jan 18 16:29:45 2018 -0800
Committer: Tathagata Das <tathagata.das1...@gmail.com>
Committed: Thu Jan 18 16:29:45 2018 -0800

--
 docs/structured-streaming-programming-guide.md | 98 -
 1 file changed, 97 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/4cd2ecc0/docs/structured-streaming-programming-guide.md
--
diff --git a/docs/structured-streaming-programming-guide.md 
b/docs/structured-streaming-programming-guide.md
index 1779a42..2ddba2f 100644
--- a/docs/structured-streaming-programming-guide.md
+++ b/docs/structured-streaming-programming-guide.md
@@ -10,7 +10,9 @@ title: Structured Streaming Programming Guide
 # Overview
 Structured Streaming is a scalable and fault-tolerant stream processing engine 
built on the Spark SQL engine. You can express your streaming computation the 
same way you would express a batch computation on static data. The Spark SQL 
engine will take care of running it incrementally and continuously and updating 
the final result as streaming data continues to arrive. You can use the 
[Dataset/DataFrame API](sql-programming-guide.html) in Scala, Java, Python or R 
to express streaming aggregations, event-time windows, stream-to-batch joins, 
etc. The computation is executed on the same optimized Spark SQL engine. 
Finally, the system ensures end-to-end exactly-once fault-tolerance guarantees 
through checkpointing and Write Ahead Logs. In short, *Structured Streaming 
provides fast, scalable, fault-tolerant, end-to-end exactly-once stream 
processing without the user having to reason about streaming.*
 
-In this guide, we are going to walk you through the programming model and the 
APIs. First, let's start with a simple example - a streaming word count.
+Internally, by default, Structured Streaming queries are processed using a 
*micro-batch processing* engine, which processes data streams as a series of 
small batch jobs thereby achieving end-to-end latencies as low as 100 
milliseconds and exactly-once fault-tolerance guarantees. However, since Spark 
2.3, we have introduced a new low-latency processing mode called **Continuous 
Processing**, which can achieve end-to-end latencies as low as 1 millisecond 
with at-least-once guarantees. Without changing the Dataset/DataFrame 
operations in your queries, you will be able choose the mode based on your 
application requirements. 
+
+In this guide, we are going to walk you through the programming model and the 
APIs. We are going to explain the concepts mostly using the default micro-batch 
processing model, and then [later](#continuous-processing-experimental) discuss 
Continuous Processing model. First, let's start with a simple example of a 
Structured Streaming query - a streaming word count.
 
 # Quick Example
 Let’s say you want to maintain a running word count of text data received 
from a data server listening on a TCP socket. Let’s see how you can express 
this using Structured Streaming. You can see the full code in
@@ -2434,6 +2436,100 @@ write.stream(aggDF, "memory", outputMode = "complete", 
checkpointLocation = "pat
 
 
 
+# Continuous Processing [Experimental]
+**Continuous processing** is a new, experimental streaming execution mode 
introduced in Spark 2.3 that enables low (~1 ms) end-to-end latency with 
at-least-once fault-tolerance guarantees. Compare this with the default 
*micro-batch processing* engine which can achieve exactly-once guarantees but 
achieve latencies of ~100ms at best. For some types of queries (discussed 
below), you can choose which mode to execute th

spark git commit: [SPARK-23144][SS] Added console sink for continuous processing

2018-01-18 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/branch-2.3 e6e8bbe84 -> 1f88fcd41


[SPARK-23144][SS] Added console sink for continuous processing

## What changes were proposed in this pull request?
Refactored ConsoleWriter into ConsoleMicrobatchWriter and 
ConsoleContinuousWriter.

## How was this patch tested?
new unit test

Author: Tathagata Das <tathagata.das1...@gmail.com>

Closes #20311 from tdas/SPARK-23144.

(cherry picked from commit bf34d665b9c865e00fac7001500bf6d521c2dff9)
Signed-off-by: Tathagata Das <tathagata.das1...@gmail.com>


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

Branch: refs/heads/branch-2.3
Commit: 1f88fcd41c6c5521d732b25e83d6c9d150d7f24a
Parents: e6e8bbe
Author: Tathagata Das <tathagata.das1...@gmail.com>
Authored: Thu Jan 18 12:33:39 2018 -0800
Committer: Tathagata Das <tathagata.das1...@gmail.com>
Committed: Thu Jan 18 12:33:54 2018 -0800

--
 .../spark/sql/execution/streaming/console.scala | 20 +++--
 .../streaming/sources/ConsoleWriter.scala   | 80 +++-
 .../streaming/sources/ConsoleWriterSuite.scala  | 26 ++-
 3 files changed, 96 insertions(+), 30 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/1f88fcd4/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala
index 9482037..f2aa325 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala
@@ -19,13 +19,12 @@ package org.apache.spark.sql.execution.streaming
 
 import java.util.Optional
 
-import scala.collection.JavaConverters._
-
 import org.apache.spark.sql._
-import org.apache.spark.sql.execution.streaming.sources.ConsoleWriter
+import 
org.apache.spark.sql.execution.streaming.sources.{ConsoleContinuousWriter, 
ConsoleMicroBatchWriter, ConsoleWriter}
 import org.apache.spark.sql.sources.{BaseRelation, CreatableRelationProvider, 
DataSourceRegister}
 import org.apache.spark.sql.sources.v2.{DataSourceV2, DataSourceV2Options}
-import org.apache.spark.sql.sources.v2.streaming.MicroBatchWriteSupport
+import org.apache.spark.sql.sources.v2.streaming.{ContinuousWriteSupport, 
MicroBatchWriteSupport}
+import org.apache.spark.sql.sources.v2.streaming.writer.ContinuousWriter
 import org.apache.spark.sql.sources.v2.writer.DataSourceV2Writer
 import org.apache.spark.sql.streaming.OutputMode
 import org.apache.spark.sql.types.StructType
@@ -37,16 +36,25 @@ case class ConsoleRelation(override val sqlContext: 
SQLContext, data: DataFrame)
 
 class ConsoleSinkProvider extends DataSourceV2
   with MicroBatchWriteSupport
+  with ContinuousWriteSupport
   with DataSourceRegister
   with CreatableRelationProvider {
 
   override def createMicroBatchWriter(
   queryId: String,
-  epochId: Long,
+  batchId: Long,
   schema: StructType,
   mode: OutputMode,
   options: DataSourceV2Options): Optional[DataSourceV2Writer] = {
-Optional.of(new ConsoleWriter(epochId, schema, options))
+Optional.of(new ConsoleMicroBatchWriter(batchId, schema, options))
+  }
+
+  override def createContinuousWriter(
+  queryId: String,
+  schema: StructType,
+  mode: OutputMode,
+  options: DataSourceV2Options): Optional[ContinuousWriter] = {
+Optional.of(new ConsoleContinuousWriter(schema, options))
   }
 
   def createRelation(

http://git-wip-us.apache.org/repos/asf/spark/blob/1f88fcd4/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWriter.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWriter.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWriter.scala
index 3619799..6fb61df 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWriter.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWriter.scala
@@ -20,45 +20,85 @@ package org.apache.spark.sql.execution.streaming.sources
 import org.apache.spark.internal.Logging
 import org.apache.spark.sql.{Row, SparkSession}
 import org.apache.spark.sql.sources.v2.DataSourceV2Options
+import org.apache.spark.sql.sources.v2.streaming.writer.ContinuousWriter
 import org.apache.spark.sql.sources.v2

spark git commit: [SPARK-23144][SS] Added console sink for continuous processing

2018-01-18 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master 2d41f040a -> bf34d665b


[SPARK-23144][SS] Added console sink for continuous processing

## What changes were proposed in this pull request?
Refactored ConsoleWriter into ConsoleMicrobatchWriter and 
ConsoleContinuousWriter.

## How was this patch tested?
new unit test

Author: Tathagata Das <tathagata.das1...@gmail.com>

Closes #20311 from tdas/SPARK-23144.


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

Branch: refs/heads/master
Commit: bf34d665b9c865e00fac7001500bf6d521c2dff9
Parents: 2d41f04
Author: Tathagata Das <tathagata.das1...@gmail.com>
Authored: Thu Jan 18 12:33:39 2018 -0800
Committer: Tathagata Das <tathagata.das1...@gmail.com>
Committed: Thu Jan 18 12:33:39 2018 -0800

--
 .../spark/sql/execution/streaming/console.scala | 20 +++--
 .../streaming/sources/ConsoleWriter.scala   | 80 +++-
 .../streaming/sources/ConsoleWriterSuite.scala  | 26 ++-
 3 files changed, 96 insertions(+), 30 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/bf34d665/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala
index 9482037..f2aa325 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala
@@ -19,13 +19,12 @@ package org.apache.spark.sql.execution.streaming
 
 import java.util.Optional
 
-import scala.collection.JavaConverters._
-
 import org.apache.spark.sql._
-import org.apache.spark.sql.execution.streaming.sources.ConsoleWriter
+import 
org.apache.spark.sql.execution.streaming.sources.{ConsoleContinuousWriter, 
ConsoleMicroBatchWriter, ConsoleWriter}
 import org.apache.spark.sql.sources.{BaseRelation, CreatableRelationProvider, 
DataSourceRegister}
 import org.apache.spark.sql.sources.v2.{DataSourceV2, DataSourceV2Options}
-import org.apache.spark.sql.sources.v2.streaming.MicroBatchWriteSupport
+import org.apache.spark.sql.sources.v2.streaming.{ContinuousWriteSupport, 
MicroBatchWriteSupport}
+import org.apache.spark.sql.sources.v2.streaming.writer.ContinuousWriter
 import org.apache.spark.sql.sources.v2.writer.DataSourceV2Writer
 import org.apache.spark.sql.streaming.OutputMode
 import org.apache.spark.sql.types.StructType
@@ -37,16 +36,25 @@ case class ConsoleRelation(override val sqlContext: 
SQLContext, data: DataFrame)
 
 class ConsoleSinkProvider extends DataSourceV2
   with MicroBatchWriteSupport
+  with ContinuousWriteSupport
   with DataSourceRegister
   with CreatableRelationProvider {
 
   override def createMicroBatchWriter(
   queryId: String,
-  epochId: Long,
+  batchId: Long,
   schema: StructType,
   mode: OutputMode,
   options: DataSourceV2Options): Optional[DataSourceV2Writer] = {
-Optional.of(new ConsoleWriter(epochId, schema, options))
+Optional.of(new ConsoleMicroBatchWriter(batchId, schema, options))
+  }
+
+  override def createContinuousWriter(
+  queryId: String,
+  schema: StructType,
+  mode: OutputMode,
+  options: DataSourceV2Options): Optional[ContinuousWriter] = {
+Optional.of(new ConsoleContinuousWriter(schema, options))
   }
 
   def createRelation(

http://git-wip-us.apache.org/repos/asf/spark/blob/bf34d665/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWriter.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWriter.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWriter.scala
index 3619799..6fb61df 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWriter.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWriter.scala
@@ -20,45 +20,85 @@ package org.apache.spark.sql.execution.streaming.sources
 import org.apache.spark.internal.Logging
 import org.apache.spark.sql.{Row, SparkSession}
 import org.apache.spark.sql.sources.v2.DataSourceV2Options
+import org.apache.spark.sql.sources.v2.streaming.writer.ContinuousWriter
 import org.apache.spark.sql.sources.v2.writer.{DataSourceV2Writer, 
DataWriterFactory, WriterCommitMessage}
 import org.apache.spark.sql.types.StructType
 
-/**
- * A [[DataSourceV2Writer]] 

spark git commit: [SPARK-23143][SS][PYTHON] Added python API for setting continuous trigger

2018-01-18 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master 9678941f5 -> 2d41f040a


[SPARK-23143][SS][PYTHON] Added python API for setting continuous trigger

## What changes were proposed in this pull request?
Self-explanatory.

## How was this patch tested?
New python tests.

Author: Tathagata Das <tathagata.das1...@gmail.com>

Closes #20309 from tdas/SPARK-23143.


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

Branch: refs/heads/master
Commit: 2d41f040a34d6483919fd5d491cf90eee5429290
Parents: 9678941
Author: Tathagata Das <tathagata.das1...@gmail.com>
Authored: Thu Jan 18 12:25:52 2018 -0800
Committer: Tathagata Das <tathagata.das1...@gmail.com>
Committed: Thu Jan 18 12:25:52 2018 -0800

--
 python/pyspark/sql/streaming.py | 23 +++
 python/pyspark/sql/tests.py |  6 ++
 2 files changed, 25 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/2d41f040/python/pyspark/sql/streaming.py
--
diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py
index 24ae377..e2a97ac 100644
--- a/python/pyspark/sql/streaming.py
+++ b/python/pyspark/sql/streaming.py
@@ -786,7 +786,7 @@ class DataStreamWriter(object):
 
 @keyword_only
 @since(2.0)
-def trigger(self, processingTime=None, once=None):
+def trigger(self, processingTime=None, once=None, continuous=None):
 """Set the trigger for the stream query. If this is not set it will 
run the query as fast
 as possible, which is equivalent to setting the trigger to 
``processingTime='0 seconds'``.
 
@@ -802,23 +802,38 @@ class DataStreamWriter(object):
 >>> writer = sdf.writeStream.trigger(processingTime='5 seconds')
 >>> # trigger the query for just once batch of data
 >>> writer = sdf.writeStream.trigger(once=True)
+>>> # trigger the query for execution every 5 seconds
+>>> writer = sdf.writeStream.trigger(continuous='5 seconds')
 """
+params = [processingTime, once, continuous]
+
+if params.count(None) == 3:
+raise ValueError('No trigger provided')
+elif params.count(None) < 2:
+raise ValueError('Multiple triggers not allowed.')
+
 jTrigger = None
 if processingTime is not None:
-if once is not None:
-raise ValueError('Multiple triggers not allowed.')
 if type(processingTime) != str or len(processingTime.strip()) == 0:
 raise ValueError('Value for processingTime must be a non empty 
string. Got: %s' %
  processingTime)
 interval = processingTime.strip()
 jTrigger = 
self._spark._sc._jvm.org.apache.spark.sql.streaming.Trigger.ProcessingTime(
 interval)
+
 elif once is not None:
 if once is not True:
 raise ValueError('Value for once must be True. Got: %s' % once)
 jTrigger = 
self._spark._sc._jvm.org.apache.spark.sql.streaming.Trigger.Once()
+
 else:
-raise ValueError('No trigger provided')
+if type(continuous) != str or len(continuous.strip()) == 0:
+raise ValueError('Value for continuous must be a non empty 
string. Got: %s' %
+ continuous)
+interval = continuous.strip()
+jTrigger = 
self._spark._sc._jvm.org.apache.spark.sql.streaming.Trigger.Continuous(
+interval)
+
 self._jwrite = self._jwrite.trigger(jTrigger)
 return self
 

http://git-wip-us.apache.org/repos/asf/spark/blob/2d41f040/python/pyspark/sql/tests.py
--
diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py
index f84aa3d..2548359 100644
--- a/python/pyspark/sql/tests.py
+++ b/python/pyspark/sql/tests.py
@@ -1538,6 +1538,12 @@ class SQLTests(ReusedSQLTestCase):
 except ValueError:
 pass
 
+# Should not take multiple args
+try:
+df.writeStream.trigger(processingTime='5 seconds', continuous='1 
second')
+except ValueError:
+pass
+
 # Should take only keyword args
 try:
 df.writeStream.trigger('5 seconds')


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



spark git commit: [SPARK-23052][SS] Migrate ConsoleSink to data source V2 api.

2018-01-17 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/branch-2.3 3a80cc59b -> 2a87c3a77


[SPARK-23052][SS] Migrate ConsoleSink to data source V2 api.

## What changes were proposed in this pull request?

Migrate ConsoleSink to data source V2 api.

Note that this includes a missing piece in DataStreamWriter required to specify 
a data source V2 writer.

Note also that I've removed the "Rerun batch" part of the sink, because as far 
as I can tell this would never have actually happened. A MicroBatchExecution 
object will only commit each batch once for its lifetime, and a new 
MicroBatchExecution object would have a new ConsoleSink object which doesn't 
know it's retrying a batch. So I think this represents an anti-feature rather 
than a weakness in the V2 API.

## How was this patch tested?

new unit test

Author: Jose Torres 

Closes #20243 from jose-torres/console-sink.

(cherry picked from commit 1c76a91e5fae11dcb66c453889e587b48039fdc9)
Signed-off-by: Tathagata Das 


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

Branch: refs/heads/branch-2.3
Commit: 2a87c3a77cbe40cbe5a8bdef41e3c37a660e2308
Parents: 3a80cc5
Author: Jose Torres 
Authored: Wed Jan 17 22:36:29 2018 -0800
Committer: Tathagata Das 
Committed: Wed Jan 17 22:36:41 2018 -0800

--
 .../streaming/MicroBatchExecution.scala |   7 +-
 .../spark/sql/execution/streaming/console.scala |  62 ++---
 .../continuous/ContinuousExecution.scala|   9 +-
 .../streaming/sources/ConsoleWriter.scala   |  64 +
 .../sources/PackedRowWriterFactory.scala|  60 +
 .../spark/sql/streaming/DataStreamWriter.scala  |  16 +-
 apache.spark.sql.sources.DataSourceRegister |   8 +
 .../streaming/sources/ConsoleWriterSuite.scala  | 135 ++
 .../sources/StreamingDataSourceV2Suite.scala| 249 +++
 .../test/DataStreamReaderWriterSuite.scala  |  25 --
 10 files changed, 551 insertions(+), 84 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/2a87c3a7/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala
--
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 70407f0..7c38045 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
@@ -91,11 +91,14 @@ class MicroBatchExecution(
   nextSourceId += 1
   StreamingExecutionRelation(reader, output)(sparkSession)
 })
-  case s @ StreamingRelationV2(_, _, _, output, v1Relation) =>
+  case s @ StreamingRelationV2(_, sourceName, _, output, v1Relation) =>
 v2ToExecutionRelationMap.getOrElseUpdate(s, {
   // Materialize source to avoid creating it in every batch
   val metadataPath = s"$resolvedCheckpointRoot/sources/$nextSourceId"
-  assert(v1Relation.isDefined, "v2 execution didn't match but v1 was 
unavailable")
+  if (v1Relation.isEmpty) {
+throw new UnsupportedOperationException(
+  s"Data source $sourceName does not support microbatch 
processing.")
+  }
   val source = v1Relation.get.dataSource.createSource(metadataPath)
   nextSourceId += 1
   StreamingExecutionRelation(source, output)(sparkSession)

http://git-wip-us.apache.org/repos/asf/spark/blob/2a87c3a7/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala
index 71eaabe..9482037 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala
@@ -17,58 +17,36 @@
 
 package org.apache.spark.sql.execution.streaming
 
-import org.apache.spark.internal.Logging
-import org.apache.spark.sql.{DataFrame, SaveMode, SQLContext}
-import org.apache.spark.sql.execution.SQLExecution
-import org.apache.spark.sql.sources.{BaseRelation, CreatableRelationProvider, 
DataSourceRegister, StreamSinkProvider}
-import org.apache.spark.sql.streaming.OutputMode
-import 

spark git commit: [SPARK-23052][SS] Migrate ConsoleSink to data source V2 api.

2018-01-17 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master 39d244d92 -> 1c76a91e5


[SPARK-23052][SS] Migrate ConsoleSink to data source V2 api.

## What changes were proposed in this pull request?

Migrate ConsoleSink to data source V2 api.

Note that this includes a missing piece in DataStreamWriter required to specify 
a data source V2 writer.

Note also that I've removed the "Rerun batch" part of the sink, because as far 
as I can tell this would never have actually happened. A MicroBatchExecution 
object will only commit each batch once for its lifetime, and a new 
MicroBatchExecution object would have a new ConsoleSink object which doesn't 
know it's retrying a batch. So I think this represents an anti-feature rather 
than a weakness in the V2 API.

## How was this patch tested?

new unit test

Author: Jose Torres 

Closes #20243 from jose-torres/console-sink.


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

Branch: refs/heads/master
Commit: 1c76a91e5fae11dcb66c453889e587b48039fdc9
Parents: 39d244d
Author: Jose Torres 
Authored: Wed Jan 17 22:36:29 2018 -0800
Committer: Tathagata Das 
Committed: Wed Jan 17 22:36:29 2018 -0800

--
 .../streaming/MicroBatchExecution.scala |   7 +-
 .../spark/sql/execution/streaming/console.scala |  62 ++---
 .../continuous/ContinuousExecution.scala|   9 +-
 .../streaming/sources/ConsoleWriter.scala   |  64 +
 .../sources/PackedRowWriterFactory.scala|  60 +
 .../spark/sql/streaming/DataStreamWriter.scala  |  16 +-
 apache.spark.sql.sources.DataSourceRegister |   8 +
 .../streaming/sources/ConsoleWriterSuite.scala  | 135 ++
 .../sources/StreamingDataSourceV2Suite.scala| 249 +++
 .../test/DataStreamReaderWriterSuite.scala  |  25 --
 10 files changed, 551 insertions(+), 84 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/1c76a91e/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala
--
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 70407f0..7c38045 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
@@ -91,11 +91,14 @@ class MicroBatchExecution(
   nextSourceId += 1
   StreamingExecutionRelation(reader, output)(sparkSession)
 })
-  case s @ StreamingRelationV2(_, _, _, output, v1Relation) =>
+  case s @ StreamingRelationV2(_, sourceName, _, output, v1Relation) =>
 v2ToExecutionRelationMap.getOrElseUpdate(s, {
   // Materialize source to avoid creating it in every batch
   val metadataPath = s"$resolvedCheckpointRoot/sources/$nextSourceId"
-  assert(v1Relation.isDefined, "v2 execution didn't match but v1 was 
unavailable")
+  if (v1Relation.isEmpty) {
+throw new UnsupportedOperationException(
+  s"Data source $sourceName does not support microbatch 
processing.")
+  }
   val source = v1Relation.get.dataSource.createSource(metadataPath)
   nextSourceId += 1
   StreamingExecutionRelation(source, output)(sparkSession)

http://git-wip-us.apache.org/repos/asf/spark/blob/1c76a91e/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala
index 71eaabe..9482037 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala
@@ -17,58 +17,36 @@
 
 package org.apache.spark.sql.execution.streaming
 
-import org.apache.spark.internal.Logging
-import org.apache.spark.sql.{DataFrame, SaveMode, SQLContext}
-import org.apache.spark.sql.execution.SQLExecution
-import org.apache.spark.sql.sources.{BaseRelation, CreatableRelationProvider, 
DataSourceRegister, StreamSinkProvider}
-import org.apache.spark.sql.streaming.OutputMode
-import org.apache.spark.sql.types.StructType
-
-class ConsoleSink(options: Map[String, String]) extends Sink with Logging {
-  // Number of rows to display, by 

spark git commit: [SPARK-23033][SS] Don't use task level retry for continuous processing

2018-01-17 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/branch-2.3 1a6dfaf25 -> dbd2a5566


[SPARK-23033][SS] Don't use task level retry for continuous processing

## What changes were proposed in this pull request?

Continuous processing tasks will fail on any attempt number greater than 0. 
ContinuousExecution will catch these failures and restart globally from the 
last recorded checkpoints.
## How was this patch tested?
unit test

Author: Jose Torres 

Closes #20225 from jose-torres/no-retry.

(cherry picked from commit 86a845031824a5334db6a5299c6f5dcc982bc5b8)
Signed-off-by: Tathagata Das 


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

Branch: refs/heads/branch-2.3
Commit: dbd2a5566d8924ab340c3c840d31e83e5af92242
Parents: 1a6dfaf
Author: Jose Torres 
Authored: Wed Jan 17 13:52:51 2018 -0800
Committer: Tathagata Das 
Committed: Wed Jan 17 13:53:02 2018 -0800

--
 .../spark/sql/kafka010/KafkaSourceSuite.scala   |  8 +--
 .../ContinuousDataSourceRDDIter.scala   |  5 ++
 .../continuous/ContinuousExecution.scala|  2 +-
 .../ContinuousTaskRetryException.scala  | 26 +++
 .../apache/spark/sql/streaming/StreamTest.scala |  9 ++-
 .../streaming/continuous/ContinuousSuite.scala  | 71 
 6 files changed, 84 insertions(+), 37 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/dbd2a556/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala
--
diff --git 
a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala
 
b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala
index 62f6a34..27dbb3f 100644
--- 
a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala
+++ 
b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala
@@ -808,16 +808,14 @@ class KafkaSourceSuiteBase extends KafkaSourceTest {
 val query = kafka
   .writeStream
   .format("memory")
-  .outputMode("append")
   .queryName("kafkaColumnTypes")
   .trigger(defaultTrigger)
   .start()
-var rows: Array[Row] = Array()
 eventually(timeout(streamingTimeout)) {
-  rows = spark.table("kafkaColumnTypes").collect()
-  assert(rows.length === 1, s"Unexpected results: ${rows.toList}")
+  assert(spark.table("kafkaColumnTypes").count == 1,
+s"Unexpected results: 
${spark.table("kafkaColumnTypes").collectAsList()}")
 }
-val row = rows(0)
+val row = spark.table("kafkaColumnTypes").head()
 assert(row.getAs[Array[Byte]]("key") === null, s"Unexpected results: $row")
 assert(row.getAs[Array[Byte]]("value") === "1".getBytes(UTF_8), 
s"Unexpected results: $row")
 assert(row.getAs[String]("topic") === topic, s"Unexpected results: $row")

http://git-wip-us.apache.org/repos/asf/spark/blob/dbd2a556/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousDataSourceRDDIter.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousDataSourceRDDIter.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousDataSourceRDDIter.scala
index 66eb42d..dcb3b54 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousDataSourceRDDIter.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousDataSourceRDDIter.scala
@@ -52,6 +52,11 @@ class ContinuousDataSourceRDD(
   }
 
   override def compute(split: Partition, context: TaskContext): 
Iterator[UnsafeRow] = {
+// If attempt number isn't 0, this is a task retry, which we don't support.
+if (context.attemptNumber() != 0) {
+  throw new ContinuousTaskRetryException()
+}
+
 val reader = 
split.asInstanceOf[DataSourceRDDPartition[UnsafeRow]].readTask.createDataReader()
 
 val runId = context.getLocalProperty(ContinuousExecution.RUN_ID_KEY)

http://git-wip-us.apache.org/repos/asf/spark/blob/dbd2a556/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
 

spark git commit: [SPARK-23033][SS] Don't use task level retry for continuous processing

2018-01-17 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master c132538a1 -> 86a845031


[SPARK-23033][SS] Don't use task level retry for continuous processing

## What changes were proposed in this pull request?

Continuous processing tasks will fail on any attempt number greater than 0. 
ContinuousExecution will catch these failures and restart globally from the 
last recorded checkpoints.
## How was this patch tested?
unit test

Author: Jose Torres 

Closes #20225 from jose-torres/no-retry.


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

Branch: refs/heads/master
Commit: 86a845031824a5334db6a5299c6f5dcc982bc5b8
Parents: c132538
Author: Jose Torres 
Authored: Wed Jan 17 13:52:51 2018 -0800
Committer: Tathagata Das 
Committed: Wed Jan 17 13:52:51 2018 -0800

--
 .../spark/sql/kafka010/KafkaSourceSuite.scala   |  8 +--
 .../ContinuousDataSourceRDDIter.scala   |  5 ++
 .../continuous/ContinuousExecution.scala|  2 +-
 .../ContinuousTaskRetryException.scala  | 26 +++
 .../apache/spark/sql/streaming/StreamTest.scala |  9 ++-
 .../streaming/continuous/ContinuousSuite.scala  | 71 
 6 files changed, 84 insertions(+), 37 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/86a84503/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala
--
diff --git 
a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala
 
b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala
index 62f6a34..27dbb3f 100644
--- 
a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala
+++ 
b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala
@@ -808,16 +808,14 @@ class KafkaSourceSuiteBase extends KafkaSourceTest {
 val query = kafka
   .writeStream
   .format("memory")
-  .outputMode("append")
   .queryName("kafkaColumnTypes")
   .trigger(defaultTrigger)
   .start()
-var rows: Array[Row] = Array()
 eventually(timeout(streamingTimeout)) {
-  rows = spark.table("kafkaColumnTypes").collect()
-  assert(rows.length === 1, s"Unexpected results: ${rows.toList}")
+  assert(spark.table("kafkaColumnTypes").count == 1,
+s"Unexpected results: 
${spark.table("kafkaColumnTypes").collectAsList()}")
 }
-val row = rows(0)
+val row = spark.table("kafkaColumnTypes").head()
 assert(row.getAs[Array[Byte]]("key") === null, s"Unexpected results: $row")
 assert(row.getAs[Array[Byte]]("value") === "1".getBytes(UTF_8), 
s"Unexpected results: $row")
 assert(row.getAs[String]("topic") === topic, s"Unexpected results: $row")

http://git-wip-us.apache.org/repos/asf/spark/blob/86a84503/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousDataSourceRDDIter.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousDataSourceRDDIter.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousDataSourceRDDIter.scala
index 66eb42d..dcb3b54 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousDataSourceRDDIter.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousDataSourceRDDIter.scala
@@ -52,6 +52,11 @@ class ContinuousDataSourceRDD(
   }
 
   override def compute(split: Partition, context: TaskContext): 
Iterator[UnsafeRow] = {
+// If attempt number isn't 0, this is a task retry, which we don't support.
+if (context.attemptNumber() != 0) {
+  throw new ContinuousTaskRetryException()
+}
+
 val reader = 
split.asInstanceOf[DataSourceRDDPartition[UnsafeRow]].readTask.createDataReader()
 
 val runId = context.getLocalProperty(ContinuousExecution.RUN_ID_KEY)

http://git-wip-us.apache.org/repos/asf/spark/blob/86a84503/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 667410e..45b794c 100644
--- 

[2/2] spark git commit: [SPARK-22908][SS] Roll forward continuous processing Kafka support with fix to continuous Kafka data reader

2018-01-16 Thread tdas
[SPARK-22908][SS] Roll forward continuous processing Kafka support with fix to 
continuous Kafka data reader

## What changes were proposed in this pull request?

The Kafka reader is now interruptible and can close itself.
## How was this patch tested?

I locally ran one of the ContinuousKafkaSourceSuite tests in a tight loop. 
Before the fix, my machine ran out of open file descriptors a few iterations 
in; now it works fine.

Author: Jose Torres 

Closes #20253 from jose-torres/fix-data-reader.

(cherry picked from commit 16670578519a7b787b0c63888b7d2873af12d5b9)
Signed-off-by: Tathagata Das 


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

Branch: refs/heads/branch-2.3
Commit: 0a441d2edb0a3f6c6c7c370db8917e1c07f211e7
Parents: 08252bb
Author: Jose Torres 
Authored: Tue Jan 16 18:11:27 2018 -0800
Committer: Tathagata Das 
Committed: Tue Jan 16 18:14:03 2018 -0800

--
 .../sql/kafka010/KafkaContinuousReader.scala| 260 +
 .../sql/kafka010/KafkaContinuousWriter.scala| 119 
 .../spark/sql/kafka010/KafkaOffsetReader.scala  |  21 +-
 .../apache/spark/sql/kafka010/KafkaSource.scala |  17 +-
 .../spark/sql/kafka010/KafkaSourceOffset.scala  |   7 +-
 .../sql/kafka010/KafkaSourceProvider.scala  | 105 +++-
 .../spark/sql/kafka010/KafkaWriteTask.scala |  71 ++-
 .../apache/spark/sql/kafka010/KafkaWriter.scala |   5 +-
 .../sql/kafka010/KafkaContinuousSinkSuite.scala | 476 
 .../kafka010/KafkaContinuousSourceSuite.scala   |  96 
 .../sql/kafka010/KafkaContinuousTest.scala  |  94 
 .../spark/sql/kafka010/KafkaSourceSuite.scala   | 539 ++-
 .../org/apache/spark/sql/DataFrameReader.scala  |  32 +-
 .../org/apache/spark/sql/DataFrameWriter.scala  |  25 +-
 .../datasources/v2/WriteToDataSourceV2.scala|   8 +-
 .../execution/streaming/StreamExecution.scala   |  15 +-
 .../ContinuousDataSourceRDDIter.scala   |   4 +-
 .../continuous/ContinuousExecution.scala|  67 ++-
 .../streaming/continuous/EpochCoordinator.scala |  21 +-
 .../spark/sql/streaming/DataStreamWriter.scala  |  26 +-
 .../apache/spark/sql/streaming/StreamTest.scala |  36 +-
 21 files changed, 1628 insertions(+), 416 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/0a441d2e/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReader.scala
--
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
new file mode 100644
index 000..fc97797
--- /dev/null
+++ 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReader.scala
@@ -0,0 +1,260 @@
+/*
+ * 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.kafka010
+
+import java.{util => ju}
+import java.util.concurrent.TimeoutException
+
+import org.apache.kafka.clients.consumer.{ConsumerRecord, 
OffsetOutOfRangeException}
+import org.apache.kafka.common.TopicPartition
+
+import org.apache.spark.TaskContext
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.expressions.UnsafeRow
+import org.apache.spark.sql.catalyst.expressions.codegen.{BufferHolder, 
UnsafeRowWriter}
+import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import 
org.apache.spark.sql.kafka010.KafkaSource.{INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_FALSE,
 INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_TRUE}
+import org.apache.spark.sql.sources.v2.reader._
+import org.apache.spark.sql.sources.v2.streaming.reader.{ContinuousDataReader, 
ContinuousReader, Offset, PartitionOffset}
+import 

[1/2] spark git commit: [SPARK-22908][SS] Roll forward continuous processing Kafka support with fix to continuous Kafka data reader

2018-01-16 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/branch-2.3 08252bb38 -> 0a441d2ed


http://git-wip-us.apache.org/repos/asf/spark/blob/0a441d2e/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala
--
diff --git 
a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala
 
b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala
index a0f5695..1acff61 100644
--- 
a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala
+++ 
b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala
@@ -34,11 +34,14 @@ import 
org.scalatest.concurrent.PatienceConfiguration.Timeout
 import org.scalatest.time.SpanSugar._
 
 import org.apache.spark.SparkContext
-import org.apache.spark.sql.ForeachWriter
+import org.apache.spark.sql.{DataFrame, Dataset, ForeachWriter, Row}
+import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, 
WriteToDataSourceV2Exec}
 import org.apache.spark.sql.execution.streaming._
+import org.apache.spark.sql.execution.streaming.continuous.ContinuousExecution
+import org.apache.spark.sql.execution.streaming.sources.ContinuousMemoryWriter
 import org.apache.spark.sql.functions.{count, window}
 import org.apache.spark.sql.kafka010.KafkaSourceProvider._
-import org.apache.spark.sql.streaming.{ProcessingTime, StreamTest}
+import org.apache.spark.sql.streaming.{ProcessingTime, StreamTest, Trigger}
 import org.apache.spark.sql.streaming.util.StreamManualClock
 import org.apache.spark.sql.test.{SharedSQLContext, TestSparkSession}
 import org.apache.spark.util.Utils
@@ -49,9 +52,11 @@ abstract class KafkaSourceTest extends StreamTest with 
SharedSQLContext {
 
   override val streamingTimeout = 30.seconds
 
+  protected val brokerProps = Map[String, Object]()
+
   override def beforeAll(): Unit = {
 super.beforeAll()
-testUtils = new KafkaTestUtils
+testUtils = new KafkaTestUtils(brokerProps)
 testUtils.setup()
   }
 
@@ -59,18 +64,25 @@ abstract class KafkaSourceTest extends StreamTest with 
SharedSQLContext {
 if (testUtils != null) {
   testUtils.teardown()
   testUtils = null
-  super.afterAll()
 }
+super.afterAll()
   }
 
   protected def makeSureGetOffsetCalled = AssertOnQuery { q =>
 // Because KafkaSource's initialPartitionOffsets is set lazily, we need to 
make sure
-// its "getOffset" is called before pushing any data. Otherwise, because 
of the race contion,
+// its "getOffset" is called before pushing any data. Otherwise, because 
of the race condition,
 // we don't know which data should be fetched when `startingOffsets` is 
latest.
-q.processAllAvailable()
+q match {
+  case c: ContinuousExecution => c.awaitEpoch(0)
+  case m: MicroBatchExecution => m.processAllAvailable()
+}
 true
   }
 
+  protected def setTopicPartitions(topic: String, newCount: Int, query: 
StreamExecution) : Unit = {
+testUtils.addPartitions(topic, newCount)
+  }
+
   /**
* Add data to Kafka.
*
@@ -82,10 +94,11 @@ abstract class KafkaSourceTest extends StreamTest with 
SharedSQLContext {
   message: String = "",
   topicAction: (String, Option[Int]) => Unit = (_, _) => {}) extends 
AddData {
 
-override def addData(query: Option[StreamExecution]): (Source, Offset) = {
-  if (query.get.isActive) {
+override def addData(query: Option[StreamExecution]): 
(BaseStreamingSource, Offset) = {
+  query match {
 // Make sure no Spark job is running when deleting a topic
-query.get.processAllAvailable()
+case Some(m: MicroBatchExecution) => m.processAllAvailable()
+case _ =>
   }
 
   val existingTopics = testUtils.getAllTopicsAndPartitionSize().toMap
@@ -97,16 +110,18 @@ abstract class KafkaSourceTest extends StreamTest with 
SharedSQLContext {
 topicAction(existingTopicPartitions._1, 
Some(existingTopicPartitions._2))
   }
 
-  // Read all topics again in case some topics are delete.
-  val allTopics = testUtils.getAllTopicsAndPartitionSize().toMap.keys
   require(
 query.nonEmpty,
 "Cannot add data when there is no query for finding the active kafka 
source")
 
   val sources = query.get.logicalPlan.collect {
-case StreamingExecutionRelation(source, _) if 
source.isInstanceOf[KafkaSource] =>
-  source.asInstanceOf[KafkaSource]
-  }
+case StreamingExecutionRelation(source: KafkaSource, _) => source
+  } ++ (query.get.lastExecution match {
+case null => Seq()
+case e => e.logical.collect {
+  case DataSourceV2Relation(_, reader: KafkaContinuousReader) => reader
+}
+  })
   if (sources.isEmpty) {
 throw new Exception(
   "Could not find Kafka source in the StreamExecution logical plan to 

[2/2] spark git commit: [SPARK-22908][SS] Roll forward continuous processing Kafka support with fix to continuous Kafka data reader

2018-01-16 Thread tdas
[SPARK-22908][SS] Roll forward continuous processing Kafka support with fix to 
continuous Kafka data reader

## What changes were proposed in this pull request?

The Kafka reader is now interruptible and can close itself.
## How was this patch tested?

I locally ran one of the ContinuousKafkaSourceSuite tests in a tight loop. 
Before the fix, my machine ran out of open file descriptors a few iterations 
in; now it works fine.

Author: Jose Torres 

Closes #20253 from jose-torres/fix-data-reader.


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

Branch: refs/heads/master
Commit: 16670578519a7b787b0c63888b7d2873af12d5b9
Parents: a9b845e
Author: Jose Torres 
Authored: Tue Jan 16 18:11:27 2018 -0800
Committer: Tathagata Das 
Committed: Tue Jan 16 18:11:27 2018 -0800

--
 .../sql/kafka010/KafkaContinuousReader.scala| 260 +
 .../sql/kafka010/KafkaContinuousWriter.scala| 119 
 .../spark/sql/kafka010/KafkaOffsetReader.scala  |  21 +-
 .../apache/spark/sql/kafka010/KafkaSource.scala |  17 +-
 .../spark/sql/kafka010/KafkaSourceOffset.scala  |   7 +-
 .../sql/kafka010/KafkaSourceProvider.scala  | 105 +++-
 .../spark/sql/kafka010/KafkaWriteTask.scala |  71 ++-
 .../apache/spark/sql/kafka010/KafkaWriter.scala |   5 +-
 .../sql/kafka010/KafkaContinuousSinkSuite.scala | 476 
 .../kafka010/KafkaContinuousSourceSuite.scala   |  96 
 .../sql/kafka010/KafkaContinuousTest.scala  |  94 
 .../spark/sql/kafka010/KafkaSourceSuite.scala   | 539 ++-
 .../org/apache/spark/sql/DataFrameReader.scala  |  32 +-
 .../org/apache/spark/sql/DataFrameWriter.scala  |  25 +-
 .../datasources/v2/WriteToDataSourceV2.scala|   8 +-
 .../execution/streaming/StreamExecution.scala   |  15 +-
 .../ContinuousDataSourceRDDIter.scala   |   4 +-
 .../continuous/ContinuousExecution.scala|  67 ++-
 .../streaming/continuous/EpochCoordinator.scala |  21 +-
 .../spark/sql/streaming/DataStreamWriter.scala  |  26 +-
 .../apache/spark/sql/streaming/StreamTest.scala |  36 +-
 21 files changed, 1628 insertions(+), 416 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/16670578/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReader.scala
--
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
new file mode 100644
index 000..fc97797
--- /dev/null
+++ 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReader.scala
@@ -0,0 +1,260 @@
+/*
+ * 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.kafka010
+
+import java.{util => ju}
+import java.util.concurrent.TimeoutException
+
+import org.apache.kafka.clients.consumer.{ConsumerRecord, 
OffsetOutOfRangeException}
+import org.apache.kafka.common.TopicPartition
+
+import org.apache.spark.TaskContext
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.expressions.UnsafeRow
+import org.apache.spark.sql.catalyst.expressions.codegen.{BufferHolder, 
UnsafeRowWriter}
+import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import 
org.apache.spark.sql.kafka010.KafkaSource.{INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_FALSE,
 INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_TRUE}
+import org.apache.spark.sql.sources.v2.reader._
+import org.apache.spark.sql.sources.v2.streaming.reader.{ContinuousDataReader, 
ContinuousReader, Offset, PartitionOffset}
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.unsafe.types.UTF8String
+
+/**
+ * A [[ContinuousReader]] for data from kafka.

[1/2] spark git commit: [SPARK-22908][SS] Roll forward continuous processing Kafka support with fix to continuous Kafka data reader

2018-01-16 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master a9b845ebb -> 166705785


http://git-wip-us.apache.org/repos/asf/spark/blob/16670578/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala
--
diff --git 
a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala
 
b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala
index a0f5695..1acff61 100644
--- 
a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala
+++ 
b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala
@@ -34,11 +34,14 @@ import 
org.scalatest.concurrent.PatienceConfiguration.Timeout
 import org.scalatest.time.SpanSugar._
 
 import org.apache.spark.SparkContext
-import org.apache.spark.sql.ForeachWriter
+import org.apache.spark.sql.{DataFrame, Dataset, ForeachWriter, Row}
+import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, 
WriteToDataSourceV2Exec}
 import org.apache.spark.sql.execution.streaming._
+import org.apache.spark.sql.execution.streaming.continuous.ContinuousExecution
+import org.apache.spark.sql.execution.streaming.sources.ContinuousMemoryWriter
 import org.apache.spark.sql.functions.{count, window}
 import org.apache.spark.sql.kafka010.KafkaSourceProvider._
-import org.apache.spark.sql.streaming.{ProcessingTime, StreamTest}
+import org.apache.spark.sql.streaming.{ProcessingTime, StreamTest, Trigger}
 import org.apache.spark.sql.streaming.util.StreamManualClock
 import org.apache.spark.sql.test.{SharedSQLContext, TestSparkSession}
 import org.apache.spark.util.Utils
@@ -49,9 +52,11 @@ abstract class KafkaSourceTest extends StreamTest with 
SharedSQLContext {
 
   override val streamingTimeout = 30.seconds
 
+  protected val brokerProps = Map[String, Object]()
+
   override def beforeAll(): Unit = {
 super.beforeAll()
-testUtils = new KafkaTestUtils
+testUtils = new KafkaTestUtils(brokerProps)
 testUtils.setup()
   }
 
@@ -59,18 +64,25 @@ abstract class KafkaSourceTest extends StreamTest with 
SharedSQLContext {
 if (testUtils != null) {
   testUtils.teardown()
   testUtils = null
-  super.afterAll()
 }
+super.afterAll()
   }
 
   protected def makeSureGetOffsetCalled = AssertOnQuery { q =>
 // Because KafkaSource's initialPartitionOffsets is set lazily, we need to 
make sure
-// its "getOffset" is called before pushing any data. Otherwise, because 
of the race contion,
+// its "getOffset" is called before pushing any data. Otherwise, because 
of the race condition,
 // we don't know which data should be fetched when `startingOffsets` is 
latest.
-q.processAllAvailable()
+q match {
+  case c: ContinuousExecution => c.awaitEpoch(0)
+  case m: MicroBatchExecution => m.processAllAvailable()
+}
 true
   }
 
+  protected def setTopicPartitions(topic: String, newCount: Int, query: 
StreamExecution) : Unit = {
+testUtils.addPartitions(topic, newCount)
+  }
+
   /**
* Add data to Kafka.
*
@@ -82,10 +94,11 @@ abstract class KafkaSourceTest extends StreamTest with 
SharedSQLContext {
   message: String = "",
   topicAction: (String, Option[Int]) => Unit = (_, _) => {}) extends 
AddData {
 
-override def addData(query: Option[StreamExecution]): (Source, Offset) = {
-  if (query.get.isActive) {
+override def addData(query: Option[StreamExecution]): 
(BaseStreamingSource, Offset) = {
+  query match {
 // Make sure no Spark job is running when deleting a topic
-query.get.processAllAvailable()
+case Some(m: MicroBatchExecution) => m.processAllAvailable()
+case _ =>
   }
 
   val existingTopics = testUtils.getAllTopicsAndPartitionSize().toMap
@@ -97,16 +110,18 @@ abstract class KafkaSourceTest extends StreamTest with 
SharedSQLContext {
 topicAction(existingTopicPartitions._1, 
Some(existingTopicPartitions._2))
   }
 
-  // Read all topics again in case some topics are delete.
-  val allTopics = testUtils.getAllTopicsAndPartitionSize().toMap.keys
   require(
 query.nonEmpty,
 "Cannot add data when there is no query for finding the active kafka 
source")
 
   val sources = query.get.logicalPlan.collect {
-case StreamingExecutionRelation(source, _) if 
source.isInstanceOf[KafkaSource] =>
-  source.asInstanceOf[KafkaSource]
-  }
+case StreamingExecutionRelation(source: KafkaSource, _) => source
+  } ++ (query.get.lastExecution match {
+case null => Seq()
+case e => e.logical.collect {
+  case DataSourceV2Relation(_, reader: KafkaContinuousReader) => reader
+}
+  })
   if (sources.isEmpty) {
 throw new Exception(
   "Could not find Kafka source in the StreamExecution logical plan to 
add 

[1/2] spark git commit: [SPARK-22908] Add kafka source and sink for continuous processing.

2018-01-11 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/branch-2.3 b94debd2b -> f891ee324


http://git-wip-us.apache.org/repos/asf/spark/blob/f891ee32/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala
index 3304f36..97f12ff 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala
@@ -255,17 +255,24 @@ final class DataFrameWriter[T] private[sql](ds: 
Dataset[T]) {
 }
   }
 
-case _ => throw new AnalysisException(s"$cls does not support data 
writing.")
+// Streaming also uses the data source V2 API. So it may be that the 
data source implements
+// v2, but has no v2 implementation for batch writes. In that case, we 
fall back to saving
+// as though it's a V1 source.
+case _ => saveToV1Source()
   }
 } else {
-  // Code path for data source v1.
-  runCommand(df.sparkSession, "save") {
-DataSource(
-  sparkSession = df.sparkSession,
-  className = source,
-  partitionColumns = partitioningColumns.getOrElse(Nil),
-  options = extraOptions.toMap).planForWriting(mode, 
AnalysisBarrier(df.logicalPlan))
-  }
+  saveToV1Source()
+}
+  }
+
+  private def saveToV1Source(): Unit = {
+// Code path for data source v1.
+runCommand(df.sparkSession, "save") {
+  DataSource(
+sparkSession = df.sparkSession,
+className = source,
+partitionColumns = partitioningColumns.getOrElse(Nil),
+options = extraOptions.toMap).planForWriting(mode, 
AnalysisBarrier(df.logicalPlan))
 }
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/f891ee32/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2.scala
index f0bdf84..a4a857f 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2.scala
@@ -81,9 +81,11 @@ case class WriteToDataSourceV2Exec(writer: 
DataSourceV2Writer, query: SparkPlan)
 (index, message: WriterCommitMessage) => messages(index) = message
   )
 
-  logInfo(s"Data source writer $writer is committing.")
-  writer.commit(messages)
-  logInfo(s"Data source writer $writer committed.")
+  if (!writer.isInstanceOf[ContinuousWriter]) {
+logInfo(s"Data source writer $writer is committing.")
+writer.commit(messages)
+logInfo(s"Data source writer $writer committed.")
+  }
 } catch {
   case _: InterruptedException if writer.isInstanceOf[ContinuousWriter] =>
 // Interruption is how continuous queries are ended, so accept and 
ignore the exception.

http://git-wip-us.apache.org/repos/asf/spark/blob/f891ee32/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 24a8b00..cf27e1a 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
@@ -142,7 +142,8 @@ abstract class StreamExecution(
 
   override val id: UUID = UUID.fromString(streamMetadata.id)
 
-  override val runId: UUID = UUID.randomUUID
+  override def runId: UUID = currentRunId
+  protected var currentRunId = UUID.randomUUID
 
   /**
* Pretty identified string of printing in logs. Format is
@@ -418,11 +419,17 @@ abstract class StreamExecution(
* Blocks the current thread until processing for data from the given 
`source` has reached at
* least the given `Offset`. This method is intended for use primarily when 
writing tests.
*/
-  private[sql] def awaitOffset(source: BaseStreamingSource, newOffset: 
Offset): Unit = {
+  private[sql] def awaitOffset(sourceIndex: Int, newOffset: Offset): Unit = {
 assertAwaitThread()
 def notDone = {
   val localCommittedOffsets = committedOffsets
-  !localCommittedOffsets.contains(source) || localCommittedOffsets(source) 
!= newOffset
+  if (sources == null) {
+// sources might not be initialized yet
+false
+  } else {
+

[2/2] spark git commit: [SPARK-22908] Add kafka source and sink for continuous processing.

2018-01-11 Thread tdas
[SPARK-22908] Add kafka source and sink for continuous processing.

## What changes were proposed in this pull request?

Add kafka source and sink for continuous processing. This involves two small 
changes to the execution engine:

* Bring data reader close() into the normal data reader thread to avoid thread 
safety issues.
* Fix up the semantics of the RECONFIGURING StreamExecution state. State 
updates are now atomic, and we don't have to deal with swallowing an exception.

## How was this patch tested?

new unit tests

Author: Jose Torres 

Closes #20096 from jose-torres/continuous-kafka.

(cherry picked from commit 6f7aaed805070d29dcba32e04ca7a1f581fa54b9)
Signed-off-by: Tathagata Das 


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

Branch: refs/heads/branch-2.3
Commit: f891ee3249e04576dd579cbab6f8f1632550e6bd
Parents: b94debd
Author: Jose Torres 
Authored: Thu Jan 11 10:52:12 2018 -0800
Committer: Tathagata Das 
Committed: Thu Jan 11 10:52:26 2018 -0800

--
 .../sql/kafka010/KafkaContinuousReader.scala| 232 +
 .../sql/kafka010/KafkaContinuousWriter.scala| 119 +
 .../spark/sql/kafka010/KafkaOffsetReader.scala  |  21 +-
 .../apache/spark/sql/kafka010/KafkaSource.scala |  17 +-
 .../spark/sql/kafka010/KafkaSourceOffset.scala  |   7 +-
 .../sql/kafka010/KafkaSourceProvider.scala  | 105 +++-
 .../spark/sql/kafka010/KafkaWriteTask.scala |  71 +--
 .../apache/spark/sql/kafka010/KafkaWriter.scala |   5 +-
 .../sql/kafka010/KafkaContinuousSinkSuite.scala | 474 +++
 .../kafka010/KafkaContinuousSourceSuite.scala   |  96 
 .../sql/kafka010/KafkaContinuousTest.scala  |  64 +++
 .../spark/sql/kafka010/KafkaSourceSuite.scala   | 470 +-
 .../org/apache/spark/sql/DataFrameReader.scala  |  32 +-
 .../org/apache/spark/sql/DataFrameWriter.scala  |  25 +-
 .../datasources/v2/WriteToDataSourceV2.scala|   8 +-
 .../execution/streaming/StreamExecution.scala   |  15 +-
 .../ContinuousDataSourceRDDIter.scala   |   3 +-
 .../continuous/ContinuousExecution.scala|  67 +--
 .../streaming/continuous/EpochCoordinator.scala |  21 +-
 .../spark/sql/streaming/DataStreamWriter.scala  |  26 +-
 .../apache/spark/sql/streaming/StreamTest.scala |  36 +-
 21 files changed, 1531 insertions(+), 383 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/f891ee32/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReader.scala
--
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
new file mode 100644
index 000..9283795
--- /dev/null
+++ 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReader.scala
@@ -0,0 +1,232 @@
+/*
+ * 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.kafka010
+
+import java.{util => ju}
+
+import org.apache.kafka.clients.consumer.ConsumerRecord
+import org.apache.kafka.common.TopicPartition
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.expressions.UnsafeRow
+import org.apache.spark.sql.catalyst.expressions.codegen.{BufferHolder, 
UnsafeRowWriter}
+import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import 
org.apache.spark.sql.kafka010.KafkaSource.{INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_FALSE,
 INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_TRUE}
+import org.apache.spark.sql.sources.v2.reader._
+import org.apache.spark.sql.sources.v2.streaming.reader.{ContinuousDataReader, 
ContinuousReader, Offset, PartitionOffset}
+import 

[2/2] spark git commit: [SPARK-22908] Add kafka source and sink for continuous processing.

2018-01-11 Thread tdas
[SPARK-22908] Add kafka source and sink for continuous processing.

## What changes were proposed in this pull request?

Add kafka source and sink for continuous processing. This involves two small 
changes to the execution engine:

* Bring data reader close() into the normal data reader thread to avoid thread 
safety issues.
* Fix up the semantics of the RECONFIGURING StreamExecution state. State 
updates are now atomic, and we don't have to deal with swallowing an exception.

## How was this patch tested?

new unit tests

Author: Jose Torres 

Closes #20096 from jose-torres/continuous-kafka.


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

Branch: refs/heads/master
Commit: 6f7aaed805070d29dcba32e04ca7a1f581fa54b9
Parents: 0b2eefb
Author: Jose Torres 
Authored: Thu Jan 11 10:52:12 2018 -0800
Committer: Tathagata Das 
Committed: Thu Jan 11 10:52:12 2018 -0800

--
 .../sql/kafka010/KafkaContinuousReader.scala| 232 +
 .../sql/kafka010/KafkaContinuousWriter.scala| 119 +
 .../spark/sql/kafka010/KafkaOffsetReader.scala  |  21 +-
 .../apache/spark/sql/kafka010/KafkaSource.scala |  17 +-
 .../spark/sql/kafka010/KafkaSourceOffset.scala  |   7 +-
 .../sql/kafka010/KafkaSourceProvider.scala  | 105 +++-
 .../spark/sql/kafka010/KafkaWriteTask.scala |  71 +--
 .../apache/spark/sql/kafka010/KafkaWriter.scala |   5 +-
 .../sql/kafka010/KafkaContinuousSinkSuite.scala | 474 +++
 .../kafka010/KafkaContinuousSourceSuite.scala   |  96 
 .../sql/kafka010/KafkaContinuousTest.scala  |  64 +++
 .../spark/sql/kafka010/KafkaSourceSuite.scala   | 470 +-
 .../org/apache/spark/sql/DataFrameReader.scala  |  32 +-
 .../org/apache/spark/sql/DataFrameWriter.scala  |  25 +-
 .../datasources/v2/WriteToDataSourceV2.scala|   8 +-
 .../execution/streaming/StreamExecution.scala   |  15 +-
 .../ContinuousDataSourceRDDIter.scala   |   3 +-
 .../continuous/ContinuousExecution.scala|  67 +--
 .../streaming/continuous/EpochCoordinator.scala |  21 +-
 .../spark/sql/streaming/DataStreamWriter.scala  |  26 +-
 .../apache/spark/sql/streaming/StreamTest.scala |  36 +-
 21 files changed, 1531 insertions(+), 383 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/6f7aaed8/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReader.scala
--
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
new file mode 100644
index 000..9283795
--- /dev/null
+++ 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReader.scala
@@ -0,0 +1,232 @@
+/*
+ * 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.kafka010
+
+import java.{util => ju}
+
+import org.apache.kafka.clients.consumer.ConsumerRecord
+import org.apache.kafka.common.TopicPartition
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.expressions.UnsafeRow
+import org.apache.spark.sql.catalyst.expressions.codegen.{BufferHolder, 
UnsafeRowWriter}
+import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import 
org.apache.spark.sql.kafka010.KafkaSource.{INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_FALSE,
 INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_TRUE}
+import org.apache.spark.sql.sources.v2.reader._
+import org.apache.spark.sql.sources.v2.streaming.reader.{ContinuousDataReader, 
ContinuousReader, Offset, PartitionOffset}
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.unsafe.types.UTF8String
+
+/**
+ * A [[ContinuousReader]] for data from kafka.
+ *
+ * 

[1/2] spark git commit: [SPARK-22908] Add kafka source and sink for continuous processing.

2018-01-11 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master 0b2eefb67 -> 6f7aaed80


http://git-wip-us.apache.org/repos/asf/spark/blob/6f7aaed8/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala
index 3304f36..97f12ff 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala
@@ -255,17 +255,24 @@ final class DataFrameWriter[T] private[sql](ds: 
Dataset[T]) {
 }
   }
 
-case _ => throw new AnalysisException(s"$cls does not support data 
writing.")
+// Streaming also uses the data source V2 API. So it may be that the 
data source implements
+// v2, but has no v2 implementation for batch writes. In that case, we 
fall back to saving
+// as though it's a V1 source.
+case _ => saveToV1Source()
   }
 } else {
-  // Code path for data source v1.
-  runCommand(df.sparkSession, "save") {
-DataSource(
-  sparkSession = df.sparkSession,
-  className = source,
-  partitionColumns = partitioningColumns.getOrElse(Nil),
-  options = extraOptions.toMap).planForWriting(mode, 
AnalysisBarrier(df.logicalPlan))
-  }
+  saveToV1Source()
+}
+  }
+
+  private def saveToV1Source(): Unit = {
+// Code path for data source v1.
+runCommand(df.sparkSession, "save") {
+  DataSource(
+sparkSession = df.sparkSession,
+className = source,
+partitionColumns = partitioningColumns.getOrElse(Nil),
+options = extraOptions.toMap).planForWriting(mode, 
AnalysisBarrier(df.logicalPlan))
 }
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/6f7aaed8/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2.scala
index f0bdf84..a4a857f 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2.scala
@@ -81,9 +81,11 @@ case class WriteToDataSourceV2Exec(writer: 
DataSourceV2Writer, query: SparkPlan)
 (index, message: WriterCommitMessage) => messages(index) = message
   )
 
-  logInfo(s"Data source writer $writer is committing.")
-  writer.commit(messages)
-  logInfo(s"Data source writer $writer committed.")
+  if (!writer.isInstanceOf[ContinuousWriter]) {
+logInfo(s"Data source writer $writer is committing.")
+writer.commit(messages)
+logInfo(s"Data source writer $writer committed.")
+  }
 } catch {
   case _: InterruptedException if writer.isInstanceOf[ContinuousWriter] =>
 // Interruption is how continuous queries are ended, so accept and 
ignore the exception.

http://git-wip-us.apache.org/repos/asf/spark/blob/6f7aaed8/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 24a8b00..cf27e1a 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
@@ -142,7 +142,8 @@ abstract class StreamExecution(
 
   override val id: UUID = UUID.fromString(streamMetadata.id)
 
-  override val runId: UUID = UUID.randomUUID
+  override def runId: UUID = currentRunId
+  protected var currentRunId = UUID.randomUUID
 
   /**
* Pretty identified string of printing in logs. Format is
@@ -418,11 +419,17 @@ abstract class StreamExecution(
* Blocks the current thread until processing for data from the given 
`source` has reached at
* least the given `Offset`. This method is intended for use primarily when 
writing tests.
*/
-  private[sql] def awaitOffset(source: BaseStreamingSource, newOffset: 
Offset): Unit = {
+  private[sql] def awaitOffset(sourceIndex: Int, newOffset: Offset): Unit = {
 assertAwaitThread()
 def notDone = {
   val localCommittedOffsets = committedOffsets
-  !localCommittedOffsets.contains(source) || localCommittedOffsets(source) 
!= newOffset
+  if (sources == null) {
+// sources might not be initialized yet
+false
+  } else {
+val 

spark git commit: [SPARK-22912] v2 data source support in MicroBatchExecution

2018-01-09 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/branch-2.3 be5991902 -> 44763d93c


[SPARK-22912] v2 data source support in MicroBatchExecution

## What changes were proposed in this pull request?

Support for v2 data sources in microbatch streaming.

## How was this patch tested?

A very basic new unit test on the toy v2 implementation of rate source. Once we 
have a v1 source fully migrated to v2, we'll need to do more detailed 
compatibility testing.

Author: Jose Torres 

Closes #20097 from jose-torres/v2-impl.


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

Branch: refs/heads/branch-2.3
Commit: 44763d93c0d923977c114d63586abfc1b68ad7fc
Parents: be59919
Author: Jose Torres 
Authored: Mon Jan 8 13:24:08 2018 -0800
Committer: Tathagata Das 
Committed: Tue Jan 9 12:15:19 2018 -0800

--
 apache.spark.sql.sources.DataSourceRegister |   1 +
 .../datasources/v2/DataSourceV2Relation.scala   |  10 ++
 .../streaming/MicroBatchExecution.scala | 112 +++
 .../execution/streaming/ProgressReporter.scala  |   6 +-
 .../streaming/RateSourceProvider.scala  |  10 +-
 .../execution/streaming/StreamExecution.scala   |   4 +-
 .../execution/streaming/StreamingRelation.scala |   4 +-
 .../continuous/ContinuousExecution.scala|   4 +-
 .../continuous/ContinuousRateStreamSource.scala |  17 +--
 .../streaming/sources/RateStreamSourceV2.scala  |  31 -
 .../spark/sql/streaming/DataStreamReader.scala  |  25 -
 .../sql/streaming/StreamingQueryManager.scala   |  24 ++--
 .../execution/streaming/RateSourceV2Suite.scala |  68 +--
 .../apache/spark/sql/streaming/StreamTest.scala |   2 +-
 .../streaming/continuous/ContinuousSuite.scala  |   2 +-
 15 files changed, 241 insertions(+), 79 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/44763d93/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
--
diff --git 
a/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
 
b/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
index 6cdfe2f..0259c77 100644
--- 
a/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
+++ 
b/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
@@ -7,3 +7,4 @@ org.apache.spark.sql.execution.datasources.text.TextFileFormat
 org.apache.spark.sql.execution.streaming.ConsoleSinkProvider
 org.apache.spark.sql.execution.streaming.TextSocketSourceProvider
 org.apache.spark.sql.execution.streaming.RateSourceProvider
+org.apache.spark.sql.execution.streaming.sources.RateSourceProviderV2

http://git-wip-us.apache.org/repos/asf/spark/blob/44763d93/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala
index 7eb99a6..cba20dd 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala
@@ -35,6 +35,16 @@ case class DataSourceV2Relation(
   }
 }
 
+/**
+ * A specialization of DataSourceV2Relation with the streaming bit set to 
true. Otherwise identical
+ * to the non-streaming relation.
+ */
+class StreamingDataSourceV2Relation(
+fullOutput: Seq[AttributeReference],
+reader: DataSourceV2Reader) extends DataSourceV2Relation(fullOutput, 
reader) {
+  override def isStreaming: Boolean = true
+}
+
 object DataSourceV2Relation {
   def apply(reader: DataSourceV2Reader): DataSourceV2Relation = {
 new DataSourceV2Relation(reader.readSchema().toAttributes, reader)

http://git-wip-us.apache.org/repos/asf/spark/blob/44763d93/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala
--
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 9a7a13f..42240ee 100644
--- 

spark git commit: [SPARK-22912] v2 data source support in MicroBatchExecution

2018-01-08 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master eed82a0b2 -> 4f7e75883


[SPARK-22912] v2 data source support in MicroBatchExecution

## What changes were proposed in this pull request?

Support for v2 data sources in microbatch streaming.

## How was this patch tested?

A very basic new unit test on the toy v2 implementation of rate source. Once we 
have a v1 source fully migrated to v2, we'll need to do more detailed 
compatibility testing.

Author: Jose Torres 

Closes #20097 from jose-torres/v2-impl.


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

Branch: refs/heads/master
Commit: 4f7e75883436069c2d9028c4cd5daa78e8d59560
Parents: eed82a0
Author: Jose Torres 
Authored: Mon Jan 8 13:24:08 2018 -0800
Committer: Tathagata Das 
Committed: Mon Jan 8 13:24:08 2018 -0800

--
 apache.spark.sql.sources.DataSourceRegister |   1 +
 .../datasources/v2/DataSourceV2Relation.scala   |  10 ++
 .../streaming/MicroBatchExecution.scala | 112 +++
 .../execution/streaming/ProgressReporter.scala  |   6 +-
 .../streaming/RateSourceProvider.scala  |  10 +-
 .../execution/streaming/StreamExecution.scala   |   4 +-
 .../execution/streaming/StreamingRelation.scala |   4 +-
 .../continuous/ContinuousExecution.scala|   4 +-
 .../continuous/ContinuousRateStreamSource.scala |  17 +--
 .../streaming/sources/RateStreamSourceV2.scala  |  31 -
 .../spark/sql/streaming/DataStreamReader.scala  |  25 -
 .../sql/streaming/StreamingQueryManager.scala   |  24 ++--
 .../execution/streaming/RateSourceV2Suite.scala |  68 +--
 .../apache/spark/sql/streaming/StreamTest.scala |   2 +-
 .../streaming/continuous/ContinuousSuite.scala  |   2 +-
 15 files changed, 241 insertions(+), 79 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/4f7e7588/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
--
diff --git 
a/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
 
b/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
index 6cdfe2f..0259c77 100644
--- 
a/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
+++ 
b/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
@@ -7,3 +7,4 @@ org.apache.spark.sql.execution.datasources.text.TextFileFormat
 org.apache.spark.sql.execution.streaming.ConsoleSinkProvider
 org.apache.spark.sql.execution.streaming.TextSocketSourceProvider
 org.apache.spark.sql.execution.streaming.RateSourceProvider
+org.apache.spark.sql.execution.streaming.sources.RateSourceProviderV2

http://git-wip-us.apache.org/repos/asf/spark/blob/4f7e7588/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala
index 7eb99a6..cba20dd 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala
@@ -35,6 +35,16 @@ case class DataSourceV2Relation(
   }
 }
 
+/**
+ * A specialization of DataSourceV2Relation with the streaming bit set to 
true. Otherwise identical
+ * to the non-streaming relation.
+ */
+class StreamingDataSourceV2Relation(
+fullOutput: Seq[AttributeReference],
+reader: DataSourceV2Reader) extends DataSourceV2Relation(fullOutput, 
reader) {
+  override def isStreaming: Boolean = true
+}
+
 object DataSourceV2Relation {
   def apply(reader: DataSourceV2Reader): DataSourceV2Relation = {
 new DataSourceV2Relation(reader.readSchema().toAttributes, reader)

http://git-wip-us.apache.org/repos/asf/spark/blob/4f7e7588/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala
--
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 9a7a13f..42240ee 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala

spark git commit: [SPARK-22278][SS] Expose current event time watermark and current processing time in GroupState

2017-10-17 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master 1437e344e -> f3137feec


[SPARK-22278][SS] Expose current event time watermark and current processing 
time in GroupState

## What changes were proposed in this pull request?

Complex state-updating and/or timeout-handling logic in mapGroupsWithState 
functions may require taking decisions based on the current event-time 
watermark and/or processing time. Currently, you can use the SQL function 
`current_timestamp` to get the current processing time, but it needs to be 
passed inserted in every row with a select, and then passed through the 
encoder, which isn't efficient. Furthermore, there is no way to get the current 
watermark.

This PR exposes both of them through the GroupState API.
Additionally, it also cleans up some of the GroupState docs.

## How was this patch tested?

New unit tests

Author: Tathagata Das <tathagata.das1...@gmail.com>

Closes #19495 from tdas/SPARK-22278.


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

Branch: refs/heads/master
Commit: f3137feecd30c74c47dbddb0e22b4ddf8cf2f912
Parents: 1437e34
Author: Tathagata Das <tathagata.das1...@gmail.com>
Authored: Tue Oct 17 20:09:12 2017 -0700
Committer: Tathagata Das <tathagata.das1...@gmail.com>
Committed: Tue Oct 17 20:09:12 2017 -0700

--
 .../apache/spark/sql/execution/objects.scala|   8 +-
 .../streaming/FlatMapGroupsWithStateExec.scala  |   7 +-
 .../execution/streaming/GroupStateImpl.scala|  50 +++---
 .../apache/spark/sql/streaming/GroupState.scala |  92 +++
 .../streaming/FlatMapGroupsWithStateSuite.scala | 160 ---
 5 files changed, 238 insertions(+), 79 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/f3137fee/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala
index c68975b..d861109 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala
@@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.expressions.codegen._
 import org.apache.spark.sql.catalyst.expressions.objects.Invoke
-import org.apache.spark.sql.catalyst.plans.logical.{FunctionUtils, 
LogicalGroupState}
+import org.apache.spark.sql.catalyst.plans.logical.{EventTimeWatermark, 
FunctionUtils, LogicalGroupState}
 import org.apache.spark.sql.catalyst.plans.physical._
 import org.apache.spark.sql.execution.streaming.GroupStateImpl
 import org.apache.spark.sql.streaming.GroupStateTimeout
@@ -361,8 +361,12 @@ object MapGroupsExec {
   outputObjAttr: Attribute,
   timeoutConf: GroupStateTimeout,
   child: SparkPlan): MapGroupsExec = {
+val watermarkPresent = child.output.exists {
+  case a: Attribute if a.metadata.contains(EventTimeWatermark.delayKey) => 
true
+  case _ => false
+}
 val f = (key: Any, values: Iterator[Any]) => {
-  func(key, values, GroupStateImpl.createForBatch(timeoutConf))
+  func(key, values, GroupStateImpl.createForBatch(timeoutConf, 
watermarkPresent))
 }
 new MapGroupsExec(f, keyDeserializer, valueDeserializer,
   groupingAttributes, dataAttributes, outputObjAttr, child)

http://git-wip-us.apache.org/repos/asf/spark/blob/f3137fee/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 c81f1a8..29f38fa 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
@@ -61,6 +61,10 @@ case class FlatMapGroupsWithStateExec(
 
   private val isTimeoutEnabled = timeoutConf != NoTimeout
   val stateManager = new FlatMapGroupsWithState_StateManager(stateEncoder, 
isTimeoutEnabled)
+  val watermarkPresent = child.output.exists {
+case a: Attribute if a.metadata.contains(EventTimeWatermark.delayKey) => 
true
+case _ => false
+  }
 
   /** Distribute by grouping attributes */
   override def require

spark git commit: [SPARK-22136][SS] Evaluate one-sided conditions early in stream-stream joins.

2017-10-17 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master e1960c3d6 -> 75d666b95


[SPARK-22136][SS] Evaluate one-sided conditions early in stream-stream joins.

## What changes were proposed in this pull request?

Evaluate one-sided conditions early in stream-stream joins.

This is in addition to normal filter pushdown, because integrating it with the 
join logic allows it to take place in outer join scenarios. This means that 
rows which can never satisfy the join condition won't clog up the state.

## How was this patch tested?
new unit tests

Author: Jose Torres 

Closes #19452 from joseph-torres/SPARK-22136.


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

Branch: refs/heads/master
Commit: 75d666b95a711787355ca3895057dabadd429023
Parents: e1960c3
Author: Jose Torres 
Authored: Tue Oct 17 12:26:53 2017 -0700
Committer: Tathagata Das 
Committed: Tue Oct 17 12:26:53 2017 -0700

--
 .../streaming/IncrementalExecution.scala|   2 +-
 .../StreamingSymmetricHashJoinExec.scala| 134 +++--
 .../StreamingSymmetricHashJoinHelper.scala  |  70 -
 .../sql/streaming/StreamingJoinSuite.scala  | 150 ++-
 .../StreamingSymmetricHashJoinHelperSuite.scala | 130 
 5 files changed, 433 insertions(+), 53 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/75d666b9/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 2e37863..a10ed5f 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
@@ -133,7 +133,7 @@ class IncrementalExecution(
   eventTimeWatermark = Some(offsetSeqMetadata.batchWatermarkMs),
   stateWatermarkPredicates =
 StreamingSymmetricHashJoinHelper.getStateWatermarkPredicates(
-  j.left.output, j.right.output, j.leftKeys, j.rightKeys, 
j.condition,
+  j.left.output, j.right.output, j.leftKeys, j.rightKeys, 
j.condition.full,
   Some(offsetSeqMetadata.batchWatermarkMs))
 )
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/75d666b9/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala
index 9bd2127..c351f65 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala
@@ -21,7 +21,7 @@ import java.util.concurrent.TimeUnit.NANOSECONDS
 
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions.{Attribute, BindReferences, 
Expression, GenericInternalRow, JoinedRow, Literal, NamedExpression, 
PreciseTimestampConversion, UnsafeProjection, UnsafeRow}
+import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, 
GenericInternalRow, JoinedRow, Literal, UnsafeProjection, UnsafeRow}
 import org.apache.spark.sql.catalyst.plans._
 import org.apache.spark.sql.catalyst.plans.logical.EventTimeWatermark._
 import org.apache.spark.sql.catalyst.plans.physical._
@@ -29,7 +29,6 @@ import org.apache.spark.sql.execution.{BinaryExecNode, 
SparkPlan}
 import 
org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper._
 import org.apache.spark.sql.execution.streaming.state._
 import org.apache.spark.sql.internal.SessionState
-import org.apache.spark.sql.types.{LongType, TimestampType}
 import org.apache.spark.util.{CompletionIterator, SerializableConfiguration}
 
 
@@ -115,7 +114,8 @@ import org.apache.spark.util.{CompletionIterator, 
SerializableConfiguration}
  * @param leftKeys  Expression to generate key rows for joining from left input
  * @param rightKeys Expression to generate key rows for joining from right 
input
  * @param joinType  Type of join (inner, left outer, etc.)
- * @param condition 

spark git commit: [SPARK-22238] Fix plan resolution bug caused by EnsureStatefulOpPartitioning

2017-10-14 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master 014dc8471 -> e8547ffb4


[SPARK-22238] Fix plan resolution bug caused by EnsureStatefulOpPartitioning

## What changes were proposed in this pull request?

In EnsureStatefulOpPartitioning, we check that the inputRDD to a SparkPlan has 
the expected partitioning for Streaming Stateful Operators. The problem is that 
we are not allowed to access this information during planning.
The reason we added that check was because CoalesceExec could actually create 
RDDs with 0 partitions. We should fix it such that when CoalesceExec says that 
there is a SinglePartition, there is in fact an inputRDD of 1 partition instead 
of 0 partitions.

## How was this patch tested?

Regression test in StreamingQuerySuite

Author: Burak Yavuz 

Closes #19467 from brkyvz/stateful-op.


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

Branch: refs/heads/master
Commit: e8547ffb49071525c06876c856cecc0d4731b918
Parents: 014dc84
Author: Burak Yavuz 
Authored: Sat Oct 14 17:39:15 2017 -0700
Committer: Tathagata Das 
Committed: Sat Oct 14 17:39:15 2017 -0700

--
 .../catalyst/plans/physical/partitioning.scala  |  15 +-
 .../sql/execution/basicPhysicalOperators.scala  |  27 +++-
 .../execution/exchange/EnsureRequirements.scala |   5 +-
 .../streaming/FlatMapGroupsWithStateExec.scala  |   2 +-
 .../streaming/IncrementalExecution.scala|  39 ++
 .../execution/streaming/statefulOperators.scala |  11 +-
 .../org/apache/spark/sql/DataFrameSuite.scala   |   2 +
 .../spark/sql/execution/PlannerSuite.scala  |  17 +++
 .../streaming/state/StateStoreRDDSuite.scala|   2 +-
 .../SymmetricHashJoinStateManagerSuite.scala|   2 +-
 .../spark/sql/streaming/DeduplicateSuite.scala  |  11 +-
 .../EnsureStatefulOpPartitioningSuite.scala | 138 ---
 .../streaming/FlatMapGroupsWithStateSuite.scala |   6 +-
 .../sql/streaming/StatefulOperatorTest.scala|  49 +++
 .../streaming/StreamingAggregationSuite.scala   |   8 +-
 .../sql/streaming/StreamingJoinSuite.scala  |   2 +-
 .../sql/streaming/StreamingQuerySuite.scala |  13 ++
 17 files changed, 160 insertions(+), 189 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/e8547ffb/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
index 51d78dd..e57c842 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
@@ -49,7 +49,9 @@ case object AllTuples extends Distribution
  * can mean such tuples are either co-located in the same partition or they 
will be contiguous
  * within a single partition.
  */
-case class ClusteredDistribution(clustering: Seq[Expression]) extends 
Distribution {
+case class ClusteredDistribution(
+clustering: Seq[Expression],
+numPartitions: Option[Int] = None) extends Distribution {
   require(
 clustering != Nil,
 "The clustering expressions of a ClusteredDistribution should not be Nil. 
" +
@@ -221,6 +223,7 @@ case object SinglePartition extends Partitioning {
 
   override def satisfies(required: Distribution): Boolean = required match {
 case _: BroadcastDistribution => false
+case ClusteredDistribution(_, desiredPartitions) => 
desiredPartitions.forall(_ == 1)
 case _ => true
   }
 
@@ -243,8 +246,9 @@ case class HashPartitioning(expressions: Seq[Expression], 
numPartitions: Int)
 
   override def satisfies(required: Distribution): Boolean = required match {
 case UnspecifiedDistribution => true
-case ClusteredDistribution(requiredClustering) =>
-  expressions.forall(x => requiredClustering.exists(_.semanticEquals(x)))
+case ClusteredDistribution(requiredClustering, desiredPartitions) =>
+  expressions.forall(x => requiredClustering.exists(_.semanticEquals(x))) 
&&
+desiredPartitions.forall(_ == numPartitions) // if desiredPartitions = 
None, returns true
 case _ => false
   }
 
@@ -289,8 +293,9 @@ case class RangePartitioning(ordering: Seq[SortOrder], 
numPartitions: Int)
 case OrderedDistribution(requiredOrdering) =>
   val minSize = Seq(requiredOrdering.size, ordering.size).min
   requiredOrdering.take(minSize) == 

spark git commit: [SPARK-22187][SS] Update unsaferow format for saved state such that we can set timeouts when state is null

2017-10-04 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master bb035f1ee -> 969ffd631


[SPARK-22187][SS] Update unsaferow format for saved state such that we can set 
timeouts when state is null

## What changes were proposed in this pull request?

Currently, the group state of user-defined-type is encoded as top-level columns 
in the UnsafeRows stores in the state store. The timeout timestamp is also 
saved as (when needed) as the last top-level column. Since the group state is 
serialized to top-level columns, you cannot save "null" as a value of state 
(setting null in all the top-level columns is not equivalent). So we don't let 
the user set the timeout without initializing the state for a key. Based on 
user experience, this leads to confusion.

This PR is to change the row format such that the state is saved as nested 
columns. This would allow the state to be set to null, and avoid these 
confusing corner cases.

## How was this patch tested?
Refactored tests.

Author: Tathagata Das <tathagata.das1...@gmail.com>

Closes #19416 from tdas/SPARK-22187.


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

Branch: refs/heads/master
Commit: 969ffd631746125eb2b83722baf6f6e7ddd2092c
Parents: bb035f1
Author: Tathagata Das <tathagata.das1...@gmail.com>
Authored: Wed Oct 4 19:25:22 2017 -0700
Committer: Tathagata Das <tathagata.das1...@gmail.com>
Committed: Wed Oct 4 19:25:22 2017 -0700

--
 .../streaming/FlatMapGroupsWithStateExec.scala  | 133 +++-
 .../FlatMapGroupsWithState_StateManager.scala   | 153 +++
 .../streaming/FlatMapGroupsWithStateSuite.scala | 130 
 3 files changed, 246 insertions(+), 170 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/969ffd63/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 ab690fd..aab06d6 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
@@ -23,10 +23,8 @@ import org.apache.spark.sql.catalyst.expressions.{Ascending, 
Attribute, Attribut
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, 
Distribution}
 import org.apache.spark.sql.execution._
-import org.apache.spark.sql.execution.streaming.GroupStateImpl.NO_TIMESTAMP
 import org.apache.spark.sql.execution.streaming.state._
 import org.apache.spark.sql.streaming.{GroupStateTimeout, OutputMode}
-import org.apache.spark.sql.types.IntegerType
 import org.apache.spark.util.CompletionIterator
 
 /**
@@ -62,26 +60,7 @@ case class FlatMapGroupsWithStateExec(
   import GroupStateImpl._
 
   private val isTimeoutEnabled = timeoutConf != NoTimeout
-  private val timestampTimeoutAttribute =
-AttributeReference("timeoutTimestamp", dataType = IntegerType, nullable = 
false)()
-  private val stateAttributes: Seq[Attribute] = {
-val encSchemaAttribs = stateEncoder.schema.toAttributes
-if (isTimeoutEnabled) encSchemaAttribs :+ timestampTimeoutAttribute else 
encSchemaAttribs
-  }
-  // Get the serializer for the state, taking into account whether we need to 
save timestamps
-  private val stateSerializer = {
-val encoderSerializer = stateEncoder.namedExpressions
-if (isTimeoutEnabled) {
-  encoderSerializer :+ Literal(GroupStateImpl.NO_TIMESTAMP)
-} else {
-  encoderSerializer
-}
-  }
-  // Get the deserializer for the state. Note that this must be done in the 
driver, as
-  // resolving and binding of deserializer expressions to the encoded type can 
be safely done
-  // only in the driver.
-  private val stateDeserializer = stateEncoder.resolveAndBind().deserializer
-
+  val stateManager = new FlatMapGroupsWithState_StateManager(stateEncoder, 
isTimeoutEnabled)
 
   /** Distribute by grouping attributes */
   override def requiredChildDistribution: Seq[Distribution] =
@@ -109,11 +88,11 @@ case class FlatMapGroupsWithStateExec(
 child.execute().mapPartitionsWithStateStore[InternalRow](
   getStateInfo,
   groupingAttributes.toStructType,
-  stateAttributes.toStructType,
+  stateManager.stateSchema,
   indexOrdinal = None,
   sqlContext.sessionState,
   Some

[1/2] spark git commit: [SPARK-22136][SS] Implement stream-stream outer joins.

2017-10-03 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master 5f6943345 -> 3099c574c


http://git-wip-us.apache.org/repos/asf/spark/blob/3099c574/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala
index 533e116..a6593b7 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala
@@ -24,8 +24,9 @@ import scala.util.Random
 import org.scalatest.BeforeAndAfter
 
 import org.apache.spark.scheduler.ExecutorCacheTaskLocation
-import org.apache.spark.sql.{AnalysisException, SparkSession}
-import org.apache.spark.sql.catalyst.expressions.{AttributeReference, 
AttributeSet}
+import org.apache.spark.sql.{AnalysisException, DataFrame, Row, SparkSession}
+import org.apache.spark.sql.catalyst.analysis.StreamingJoinHelper
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, 
AttributeSet, Literal}
 import org.apache.spark.sql.catalyst.plans.logical.{EventTimeWatermark, Filter}
 import org.apache.spark.sql.execution.LogicalRDD
 import org.apache.spark.sql.execution.streaming.{MemoryStream, 
StatefulOperatorStateInfo, StreamingSymmetricHashJoinHelper}
@@ -35,7 +36,7 @@ import org.apache.spark.sql.types._
 import org.apache.spark.util.Utils
 
 
-class StreamingJoinSuite extends StreamTest with StateStoreMetricsTest with 
BeforeAndAfter {
+class StreamingInnerJoinSuite extends StreamTest with StateStoreMetricsTest 
with BeforeAndAfter {
 
   before {
 SparkSession.setActiveSession(spark)  // set this before force 
initializing 'joinExec'
@@ -322,111 +323,6 @@ class StreamingJoinSuite extends StreamTest with 
StateStoreMetricsTest with Befo
 assert(e.toString.contains("Stream stream joins without equality predicate 
is not supported"))
   }
 
-  testQuietly("extract watermark from time condition") {
-val attributesToFindConstraintFor = Seq(
-  AttributeReference("leftTime", TimestampType)(),
-  AttributeReference("leftOther", IntegerType)())
-val metadataWithWatermark = new MetadataBuilder()
-  .putLong(EventTimeWatermark.delayKey, 1000)
-  .build()
-val attributesWithWatermark = Seq(
-  AttributeReference("rightTime", TimestampType, metadata = 
metadataWithWatermark)(),
-  AttributeReference("rightOther", IntegerType)())
-
-def watermarkFrom(
-conditionStr: String,
-rightWatermark: Option[Long] = Some(1)): Option[Long] = {
-  val conditionExpr = Some(conditionStr).map { str =>
-val plan =
-  Filter(
-spark.sessionState.sqlParser.parseExpression(str),
-LogicalRDD(
-  attributesToFindConstraintFor ++ attributesWithWatermark,
-  spark.sparkContext.emptyRDD)(spark))
-plan.queryExecution.optimizedPlan.asInstanceOf[Filter].condition
-  }
-  StreamingSymmetricHashJoinHelper.getStateValueWatermark(
-AttributeSet(attributesToFindConstraintFor), 
AttributeSet(attributesWithWatermark),
-conditionExpr, rightWatermark)
-}
-
-// Test comparison directionality. E.g. if leftTime < rightTime and 
rightTime > watermark,
-// then cannot define constraint on leftTime.
-assert(watermarkFrom("leftTime > rightTime") === Some(1))
-assert(watermarkFrom("leftTime >= rightTime") === Some())
-assert(watermarkFrom("leftTime < rightTime") === None)
-assert(watermarkFrom("leftTime <= rightTime") === None)
-assert(watermarkFrom("rightTime > leftTime") === None)
-assert(watermarkFrom("rightTime >= leftTime") === None)
-assert(watermarkFrom("rightTime < leftTime") === Some(1))
-assert(watermarkFrom("rightTime <= leftTime") === Some())
-
-// Test type conversions
-assert(watermarkFrom("CAST(leftTime AS LONG) > CAST(rightTime AS LONG)") 
=== Some(1))
-assert(watermarkFrom("CAST(leftTime AS LONG) < CAST(rightTime AS LONG)") 
=== None)
-assert(watermarkFrom("CAST(leftTime AS DOUBLE) > CAST(rightTime AS 
DOUBLE)") === Some(1))
-assert(watermarkFrom("CAST(leftTime AS LONG) > CAST(rightTime AS DOUBLE)") 
=== Some(1))
-assert(watermarkFrom("CAST(leftTime AS LONG) > CAST(rightTime AS FLOAT)") 
=== Some(1))
-assert(watermarkFrom("CAST(leftTime AS DOUBLE) > CAST(rightTime AS 
FLOAT)") === Some(1))
-assert(watermarkFrom("CAST(leftTime AS STRING) > CAST(rightTime AS 
STRING)") === None)
-
-// Test with timestamp type + calendar interval on either side of equation
-// Note: timestamptype and calendar interval don't commute, so less valid 
combinations to test.
-assert(watermarkFrom("leftTime > rightTime + interval 1 second") === 
Some(11000))
-assert(watermarkFrom("leftTime + 

[2/2] spark git commit: [SPARK-22136][SS] Implement stream-stream outer joins.

2017-10-03 Thread tdas
[SPARK-22136][SS] Implement stream-stream outer joins.

## What changes were proposed in this pull request?

Allow one-sided outer joins between two streams when a watermark is defined.

## How was this patch tested?

new unit tests

Author: Jose Torres 

Closes #19327 from joseph-torres/outerjoin.


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

Branch: refs/heads/master
Commit: 3099c574c56cab86c3fcf759864f89151643f837
Parents: 5f69433
Author: Jose Torres 
Authored: Tue Oct 3 21:42:51 2017 -0700
Committer: Tathagata Das 
Committed: Tue Oct 3 21:42:51 2017 -0700

--
 .../catalyst/analysis/StreamingJoinHelper.scala | 286 ++
 .../analysis/UnsupportedOperationChecker.scala  |  53 +++-
 .../analysis/StreamingJoinHelperSuite.scala | 140 +
 .../analysis/UnsupportedOperationsSuite.scala   | 108 ++-
 .../StreamingSymmetricHashJoinExec.scala| 152 --
 .../StreamingSymmetricHashJoinHelper.scala  | 241 +--
 .../state/SymmetricHashJoinStateManager.scala   | 200 +
 .../SymmetricHashJoinStateManagerSuite.scala|   6 +-
 .../sql/streaming/StreamingJoinSuite.scala  | 298 ---
 9 files changed, 1051 insertions(+), 433 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/3099c574/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/StreamingJoinHelper.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/StreamingJoinHelper.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/StreamingJoinHelper.scala
new file mode 100644
index 000..072dc95
--- /dev/null
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/StreamingJoinHelper.scala
@@ -0,0 +1,286 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.control.NonFatal
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.expressions.{Add, AttributeReference, 
AttributeSet, Cast, CheckOverflow, Expression, ExpressionSet, GreaterThan, 
GreaterThanOrEqual, LessThan, LessThanOrEqual, Literal, Multiply, 
PreciseTimestampConversion, PredicateHelper, Subtract, TimeAdd, TimeSub, 
UnaryMinus}
+import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys
+import org.apache.spark.sql.catalyst.plans.logical.{EventTimeWatermark, 
LogicalPlan}
+import org.apache.spark.sql.catalyst.plans.logical.EventTimeWatermark._
+import org.apache.spark.sql.types._
+import org.apache.spark.unsafe.types.CalendarInterval
+
+
+/**
+ * Helper object for stream joins. See [[StreamingSymmetricHashJoinExec]] in 
SQL for more details.
+ */
+object StreamingJoinHelper extends PredicateHelper with Logging {
+
+  /**
+   * Check the provided logical plan to see if its join keys contain a 
watermark attribute.
+   *
+   * Will return false if the plan is not an equijoin.
+   * @param plan the logical plan to check
+   */
+  def isWatermarkInJoinKeys(plan: LogicalPlan): Boolean = {
+plan match {
+  case ExtractEquiJoinKeys(_, leftKeys, rightKeys, _, _, _) =>
+(leftKeys ++ rightKeys).exists {
+  case a: AttributeReference => 
a.metadata.contains(EventTimeWatermark.delayKey)
+  case _ => false
+}
+  case _ => false
+}
+  }
+
+  /**
+   * Get state value watermark (see [[StreamingSymmetricHashJoinExec]] for 
context about it)
+   * given the join condition and the event time watermark. This is how it 
works.
+   * - The condition is split into conjunctive predicates, and we find the 
predicates of the
+   *   form `leftTime + c1 < rightTime + c2`   (or <=, >, >=).
+   * - We canoncalize the predicate and solve it with the event time 

[2/2] spark git commit: [SPARK-22053][SS] Stream-stream inner join in Append Mode

2017-09-21 Thread tdas
[SPARK-22053][SS] Stream-stream inner join in Append Mode

## What changes were proposed in this pull request?

 Architecture
This PR implements stream-stream inner join using a two-way symmetric hash 
join. At a high level, we want to do the following.

1. For each stream, we maintain the past rows as state in State Store.
  - For each joining key, there can be multiple rows that have been received.
  - So, we have to effectively maintain a key-to-list-of-values multimap as 
state for each stream.
2. In each batch, for each input row in each stream
  - Look up the other streams state to see if there are matching rows, and 
output them if they satisfy the joining condition
  - Add the input row to corresponding stream’s state.
  - If the data has a timestamp/window column with watermark, then we will use 
that to calculate the threshold for keys that are required to buffered for 
future matches and drop the rest from the state.

Cleaning up old unnecessary state rows depends completely on whether watermark 
has been defined and what are join conditions. We definitely want to support 
state clean up two types of queries that are likely to be common.

- Queries to time range conditions - E.g. `SELECT * FROM leftTable, rightTable 
ON leftKey = rightKey AND leftTime > rightTime - INTERVAL 8 MINUTES AND 
leftTime < rightTime + INTERVAL 1 HOUR`
- Queries with windows as the matching key - E.g. `SELECT * FROM leftTable, 
rightTable ON leftKey = rightKey AND window(leftTime, "1 hour") = 
window(rightTime, "1 hour")` (pseudo-SQL)

 Implementation
The stream-stream join is primarily implemented in three classes
- `StreamingSymmetricHashJoinExec` implements the above symmetric join 
algorithm.
- `SymmetricsHashJoinStateManagers` manages the streaming state for the join. 
This essentially is a fault-tolerant key-to-list-of-values multimap built on 
the StateStore APIs. `StreamingSymmetricHashJoinExec` instantiates two such 
managers, one for each join side.
- `StreamingSymmetricHashJoinExecHelper` is a helper class to extract threshold 
for the state based on the join conditions and the event watermark.

Refer to the scaladocs class for more implementation details.

Besides the implementation of stream-stream inner join SparkPlan. Some 
additional changes are
- Allowed inner join in append mode in UnsupportedOperationChecker
- Prevented stream-stream join on an empty batch dataframe to be collapsed by 
the optimizer

## How was this patch tested?
- New tests in StreamingJoinSuite
- Updated tests UnsupportedOperationSuite

Author: Tathagata Das <tathagata.das1...@gmail.com>

Closes #19271 from tdas/SPARK-22053.


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

Branch: refs/heads/master
Commit: f32a8425051eabdef2d69002cfc843c01d98df0d
Parents: a8a5cd2
Author: Tathagata Das <tathagata.das1...@gmail.com>
Authored: Thu Sep 21 15:39:07 2017 -0700
Committer: Tathagata Das <tathagata.das1...@gmail.com>
Committed: Thu Sep 21 15:39:07 2017 -0700

--
 .../spark/sql/catalyst/analysis/Analyzer.scala  |   2 +-
 .../analysis/UnsupportedOperationChecker.scala  |   7 +-
 .../catalyst/expressions/namedExpressions.scala |   9 +-
 .../optimizer/PropagateEmptyRelation.scala  |  25 +-
 .../analysis/UnsupportedOperationsSuite.scala   |  22 +-
 .../spark/sql/execution/SparkStrategies.scala   |  21 +-
 .../streaming/IncrementalExecution.scala|  11 +
 .../execution/streaming/StreamExecution.scala   |   2 +
 .../StreamingSymmetricHashJoinExec.scala| 346 ++
 .../StreamingSymmetricHashJoinHelper.scala  | 415 
 .../execution/streaming/state/StateStore.scala  |  21 +
 .../streaming/state/StateStoreCoordinator.scala |   6 +-
 .../state/SymmetricHashJoinStateManager.scala   | 395 
 .../sql/execution/streaming/state/package.scala |   2 -
 .../execution/streaming/statefulOperators.scala |  55 ++-
 .../SymmetricHashJoinStateManagerSuite.scala| 172 +++
 .../sql/streaming/StateStoreMetricsTest.scala   |   2 +-
 .../sql/streaming/StreamingJoinSuite.scala  | 472 +++
 18 files changed, 1940 insertions(+), 45 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/f32a8425/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index 45ec204..8edf575 100644
--- 
a/sql/catalyst/

[1/2] spark git commit: [SPARK-22053][SS] Stream-stream inner join in Append Mode

2017-09-21 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master a8a5cd24e -> f32a84250


http://git-wip-us.apache.org/repos/asf/spark/blob/f32a8425/sql/core/src/test/scala/org/apache/spark/sql/streaming/StateStoreMetricsTest.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StateStoreMetricsTest.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StateStoreMetricsTest.scala
index 894786c..368c460 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StateStoreMetricsTest.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StateStoreMetricsTest.scala
@@ -20,7 +20,7 @@ package org.apache.spark.sql.streaming
 trait StateStoreMetricsTest extends StreamTest {
 
   def assertNumStateRows(total: Seq[Long], updated: Seq[Long]): AssertOnQuery =
-AssertOnQuery { q =>
+AssertOnQuery(s"Check total state rows = $total, updated state rows = 
$updated") { q =>
   val progressWithData = q.recentProgress.filter(_.numInputRows > 
0).lastOption.get
   assert(
 progressWithData.stateOperators.map(_.numRowsTotal) === total,

http://git-wip-us.apache.org/repos/asf/spark/blob/f32a8425/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala
new file mode 100644
index 000..533e116
--- /dev/null
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala
@@ -0,0 +1,472 @@
+/*
+ * 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.streaming
+
+import java.util.UUID
+
+import scala.util.Random
+
+import org.scalatest.BeforeAndAfter
+
+import org.apache.spark.scheduler.ExecutorCacheTaskLocation
+import org.apache.spark.sql.{AnalysisException, SparkSession}
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, 
AttributeSet}
+import org.apache.spark.sql.catalyst.plans.logical.{EventTimeWatermark, Filter}
+import org.apache.spark.sql.execution.LogicalRDD
+import org.apache.spark.sql.execution.streaming.{MemoryStream, 
StatefulOperatorStateInfo, StreamingSymmetricHashJoinHelper}
+import org.apache.spark.sql.execution.streaming.state.{StateStore, 
StateStoreProviderId}
+import org.apache.spark.sql.functions._
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+class StreamingJoinSuite extends StreamTest with StateStoreMetricsTest with 
BeforeAndAfter {
+
+  before {
+SparkSession.setActiveSession(spark)  // set this before force 
initializing 'joinExec'
+spark.streams.stateStoreCoordinator   // initialize the lazy coordinator
+  }
+
+  after {
+StateStore.stop()
+  }
+
+  import testImplicits._
+  test("stream stream inner join on non-time column") {
+val input1 = MemoryStream[Int]
+val input2 = MemoryStream[Int]
+
+val df1 = input1.toDF.select('value as "key", ('value * 2) as "leftValue")
+val df2 = input2.toDF.select('value as "key", ('value * 3) as "rightValue")
+val joined = df1.join(df2, "key")
+
+testStream(joined)(
+  AddData(input1, 1),
+  CheckAnswer(),
+  AddData(input2, 1, 10),   // 1 arrived on input1 first, then input2, 
should join
+  CheckLastBatch((1, 2, 3)),
+  AddData(input1, 10),  // 10 arrived on input2 first, then 
input1, should join
+  CheckLastBatch((10, 20, 30)),
+  AddData(input2, 1),   // another 1 in input2 should join with 1 
input1
+  CheckLastBatch((1, 2, 3)),
+  StopStream,
+  StartStream(),
+  AddData(input1, 1), // multiple 1s should be kept in state causing 
multiple (1, 2, 3)
+  CheckLastBatch((1, 2, 3), (1, 2, 3)),
+  StopStream,
+  StartStream(),
+  AddData(input1, 100),
+  AddData(input2, 100),
+  CheckLastBatch((100, 200, 300))
+)
+  }
+
+  test("stream stream inner join on windows - without watermark") {
+val input1 = MemoryStream[Int]
+val input2 = MemoryStream[Int]
+
+val df1 = input1.toDF

spark git commit: [SPARK-22017] Take minimum of all watermark execs in StreamExecution.

2017-09-15 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master c7307acda -> 0bad10d3e


[SPARK-22017] Take minimum of all watermark execs in StreamExecution.

## What changes were proposed in this pull request?

Take the minimum of all watermark exec nodes as the "real" watermark in 
StreamExecution, rather than picking one arbitrarily.

## How was this patch tested?

new unit test

Author: Jose Torres 

Closes #19239 from joseph-torres/SPARK-22017.


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

Branch: refs/heads/master
Commit: 0bad10d3e36d3238c7ee7c0fc5465072734b3ae4
Parents: c7307ac
Author: Jose Torres 
Authored: Fri Sep 15 21:10:07 2017 -0700
Committer: Tathagata Das 
Committed: Fri Sep 15 21:10:07 2017 -0700

--
 .../streaming/IncrementalExecution.scala|  2 +-
 .../execution/streaming/StreamExecution.scala   | 39 --
 .../sql/streaming/EventTimeWatermarkSuite.scala | 78 
 3 files changed, 113 insertions(+), 6 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/0bad10d3/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 258a642..19d9598 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
@@ -39,7 +39,7 @@ class IncrementalExecution(
 val checkpointLocation: String,
 val runId: UUID,
 val currentBatchId: Long,
-offsetSeqMetadata: OffsetSeqMetadata)
+val offsetSeqMetadata: OffsetSeqMetadata)
   extends QueryExecution(sparkSession, logicalPlan) with Logging {
 
   // Modified planner with stateful operations.

http://git-wip-us.apache.org/repos/asf/spark/blob/0bad10d3/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 952e431..b27a59b 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
@@ -130,6 +130,16 @@ class StreamExecution(
   protected var offsetSeqMetadata = OffsetSeqMetadata(
 batchWatermarkMs = 0, batchTimestampMs = 0, sparkSession.conf)
 
+  /**
+   * A map of current watermarks, keyed by the position of the watermark 
operator in the
+   * physical plan.
+   *
+   * This state is 'soft state', which does not affect the correctness and 
semantics of watermarks
+   * and is not persisted across query restarts.
+   * The fault-tolerant watermark state is in offsetSeqMetadata.
+   */
+  protected val watermarkMsMap: MutableMap[Int, Long] = MutableMap()
+
   override val id: UUID = UUID.fromString(streamMetadata.id)
 
   override val runId: UUID = UUID.randomUUID
@@ -560,13 +570,32 @@ class StreamExecution(
 }
 if (hasNewData) {
   var batchWatermarkMs = offsetSeqMetadata.batchWatermarkMs
-  // Update the eventTime watermark if we find one in the plan.
+  // Update the eventTime watermarks if we find any in the plan.
   if (lastExecution != null) {
 lastExecution.executedPlan.collect {
-  case e: EventTimeWatermarkExec if e.eventTimeStats.value.count > 0 =>
-logDebug(s"Observed event time stats: ${e.eventTimeStats.value}")
-e.eventTimeStats.value.max - e.delayMs
-}.headOption.foreach { newWatermarkMs =>
+  case e: EventTimeWatermarkExec => e
+}.zipWithIndex.foreach {
+  case (e, index) if e.eventTimeStats.value.count > 0 =>
+logDebug(s"Observed event time stats $index: 
${e.eventTimeStats.value}")
+val newWatermarkMs = e.eventTimeStats.value.max - e.delayMs
+val prevWatermarkMs = watermarkMsMap.get(index)
+if (prevWatermarkMs.isEmpty || newWatermarkMs > 
prevWatermarkMs.get) {
+  watermarkMsMap.put(index, newWatermarkMs)
+}
+
+  // Populate 0 if we haven't seen any data yet for this watermark 
node.
+  case (_, index) =>
+

spark git commit: [SPARK-22018][SQL] Preserve top-level alias metadata when collapsing projects

2017-09-14 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master a28728a9a -> 88661747f


[SPARK-22018][SQL] Preserve top-level alias metadata when collapsing projects

## What changes were proposed in this pull request?
If there are two projects like as follows.
```
Project [a_with_metadata#27 AS b#26]
+- Project [a#0 AS a_with_metadata#27]
   +- LocalRelation , [a#0, b#1]
```
Child Project has an output column with a metadata in it, and the parent 
Project has an alias that implicitly forwards the metadata. So this metadata is 
visible for higher operators. Upon applying CollapseProject optimizer rule, the 
metadata is not preserved.
```
Project [a#0 AS b#26]
+- LocalRelation , [a#0, b#1]
```
This is incorrect, as downstream operators that expect certain metadata (e.g. 
watermark in structured streaming) to identify certain fields will fail to do 
so. This PR fixes it by preserving the metadata of top-level aliases.

## How was this patch tested?
New unit test

Author: Tathagata Das <tathagata.das1...@gmail.com>

Closes #19240 from tdas/SPARK-22018.


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

Branch: refs/heads/master
Commit: 88661747f506e73c79de36711daebb0330de7b0d
Parents: a28728a
Author: Tathagata Das <tathagata.das1...@gmail.com>
Authored: Thu Sep 14 22:32:16 2017 -0700
Committer: Tathagata Das <tathagata.das1...@gmail.com>
Committed: Thu Sep 14 22:32:16 2017 -0700

--
 .../spark/sql/catalyst/analysis/Analyzer.scala  |  5 -
 .../optimizer/CollapseProjectSuite.scala| 23 ++--
 2 files changed, 25 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/88661747/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index 0880bd6..db276fb 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -2256,7 +2256,10 @@ object CleanupAliases extends Rule[LogicalPlan] {
 
   def trimNonTopLevelAliases(e: Expression): Expression = e match {
 case a: Alias =>
-  a.withNewChildren(trimAliases(a.child) :: Nil)
+  a.copy(child = trimAliases(a.child))(
+exprId = a.exprId,
+qualifier = a.qualifier,
+explicitMetadata = Some(a.metadata))
 case other => trimAliases(other)
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/88661747/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CollapseProjectSuite.scala
--
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CollapseProjectSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CollapseProjectSuite.scala
index 587437e..e7a5bce 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CollapseProjectSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CollapseProjectSuite.scala
@@ -20,10 +20,11 @@ package org.apache.spark.sql.catalyst.optimizer
 import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases
 import org.apache.spark.sql.catalyst.dsl.expressions._
 import org.apache.spark.sql.catalyst.dsl.plans._
-import org.apache.spark.sql.catalyst.expressions.Rand
+import org.apache.spark.sql.catalyst.expressions.{Alias, Rand}
 import org.apache.spark.sql.catalyst.plans.PlanTest
-import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan}
+import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, 
LogicalPlan, Project}
 import org.apache.spark.sql.catalyst.rules.RuleExecutor
+import org.apache.spark.sql.types.MetadataBuilder
 
 class CollapseProjectSuite extends PlanTest {
   object Optimize extends RuleExecutor[LogicalPlan] {
@@ -119,4 +120,22 @@ class CollapseProjectSuite extends PlanTest {
 
 comparePlans(optimized, correctAnswer)
   }
+
+  test("preserve top-level alias metadata while collapsing projects") {
+def hasMetadata(logicalPlan: LogicalPlan): Boolean = {
+  
logicalPlan.asInstanceOf[Project].projectList.exists(_.metadata.contains("key"))
+}
+
+val metadata = new MetadataBuilder().putLong("key", 1).build()
+val analyzed =
+  Project(Seq(Alias('a_with_metadata, "b")()),
+Project

spark git commit: [SPARK-21765] Check that optimization doesn't affect isStreaming bit.

2017-09-06 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master 36b48ee6e -> acdf45fb5


[SPARK-21765] Check that optimization doesn't affect isStreaming bit.

## What changes were proposed in this pull request?

Add an assert in logical plan optimization that the isStreaming bit stays the 
same, and fix empty relation rules where that wasn't happening.

## How was this patch tested?

new and existing unit tests

Author: Jose Torres 
Author: Jose Torres 

Closes #19056 from joseph-torres/SPARK-21765-followup.


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

Branch: refs/heads/master
Commit: acdf45fb52e29a0308cccdbef0ec0dca0815d300
Parents: 36b48ee
Author: Jose Torres 
Authored: Wed Sep 6 11:19:46 2017 -0700
Committer: Tathagata Das 
Committed: Wed Sep 6 11:19:46 2017 -0700

--
 .../sql/catalyst/optimizer/Optimizer.scala  |  6 +-
 .../optimizer/PropagateEmptyRelation.scala  | 11 ++-
 .../catalyst/plans/logical/LocalRelation.scala  |  2 +-
 .../spark/sql/catalyst/rules/RuleExecutor.scala |  1 -
 .../optimizer/PropagateEmptyRelationSuite.scala | 44 
 .../execution/streaming/StreamExecution.scala   |  2 +-
 .../spark/sql/execution/streaming/socket.scala  | 17 ++---
 .../streaming/TextSocketStreamSuite.scala   | 72 +++-
 8 files changed, 103 insertions(+), 52 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/acdf45fb/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
index d7e5906..02d6778 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
@@ -724,8 +724,10 @@ object PruneFilters extends Rule[LogicalPlan] with 
PredicateHelper {
 case Filter(Literal(true, BooleanType), child) => child
 // If the filter condition always evaluate to null or false,
 // replace the input with an empty relation.
-case Filter(Literal(null, _), child) => LocalRelation(child.output, data = 
Seq.empty)
-case Filter(Literal(false, BooleanType), child) => 
LocalRelation(child.output, data = Seq.empty)
+case Filter(Literal(null, _), child) =>
+  LocalRelation(child.output, data = Seq.empty, isStreaming = 
plan.isStreaming)
+case Filter(Literal(false, BooleanType), child) =>
+  LocalRelation(child.output, data = Seq.empty, isStreaming = 
plan.isStreaming)
 // If any deterministic condition is guaranteed to be true given the 
constraints on the child's
 // output, remove the condition
 case f @ Filter(fc, p: LogicalPlan) =>

http://git-wip-us.apache.org/repos/asf/spark/blob/acdf45fb/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelation.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelation.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelation.scala
index 987cd74..cfffa6b 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelation.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelation.scala
@@ -38,7 +38,8 @@ object PropagateEmptyRelation extends Rule[LogicalPlan] with 
PredicateHelper {
 case _ => false
   }
 
-  private def empty(plan: LogicalPlan) = LocalRelation(plan.output, data = 
Seq.empty)
+  private def empty(plan: LogicalPlan) =
+LocalRelation(plan.output, data = Seq.empty, isStreaming = 
plan.isStreaming)
 
   def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
 case p: Union if p.children.forall(isEmptyLocalRelation) =>
@@ -65,11 +66,15 @@ object PropagateEmptyRelation extends Rule[LogicalPlan] 
with PredicateHelper {
   case _: RepartitionByExpression => empty(p)
   // An aggregate with non-empty group expression will return one output 
row per group when the
   // input to the aggregate is not empty. If the input to the aggregate is 
empty then all groups
-  // will be empty and thus the output will be empty.
+  // will be empty and thus the output will be empty. If we're working on 
batch data, we can
+  // 

spark git commit: [SPARK-21925] Update trigger interval documentation in docs with behavior change in Spark 2.2

2017-09-05 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/branch-2.2 fb1b5f08a -> 1f7c4869b


[SPARK-21925] Update trigger interval documentation in docs with behavior 
change in Spark 2.2

Forgot to update docs with behavior change.

Author: Burak Yavuz 

Closes #19138 from brkyvz/trigger-doc-fix.

(cherry picked from commit 8c954d2cd10a2cf729d2971fbeb19b2dd751a178)
Signed-off-by: Tathagata Das 


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

Branch: refs/heads/branch-2.2
Commit: 1f7c4869b811f9a05cd1fb54e168e739cde7933f
Parents: fb1b5f0
Author: Burak Yavuz 
Authored: Tue Sep 5 13:10:32 2017 -0700
Committer: Tathagata Das 
Committed: Tue Sep 5 13:10:47 2017 -0700

--
 docs/structured-streaming-programming-guide.md | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/1f7c4869/docs/structured-streaming-programming-guide.md
--
diff --git a/docs/structured-streaming-programming-guide.md 
b/docs/structured-streaming-programming-guide.md
index 8367f5a..13a6a82 100644
--- a/docs/structured-streaming-programming-guide.md
+++ b/docs/structured-streaming-programming-guide.md
@@ -1168,7 +1168,7 @@ returned through `Dataset.writeStream()`. You will have 
to specify one or more o
 
 - *Query name:* Optionally, specify a unique name of the query for 
identification.
 
-- *Trigger interval:* Optionally, specify the trigger interval. If it is not 
specified, the system will check for availability of new data as soon as the 
previous processing has completed. If a trigger time is missed because the 
previous processing has not completed, then the system will attempt to trigger 
at the next trigger point, not immediately after the processing has completed.
+- *Trigger interval:* Optionally, specify the trigger interval. If it is not 
specified, the system will check for availability of new data as soon as the 
previous processing has completed. If a trigger time is missed because the 
previous processing has not completed, then the system will trigger processing 
immediately.
 
 - *Checkpoint location:* For some output sinks where the end-to-end 
fault-tolerance can be guaranteed, specify the location where the system will 
write all the checkpoint information. This should be a directory in an 
HDFS-compatible fault-tolerant file system. The semantics of checkpointing is 
discussed in more detail in the next section.
 


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



spark git commit: [SPARK-21925] Update trigger interval documentation in docs with behavior change in Spark 2.2

2017-09-05 Thread tdas
Repository: spark
Updated Branches:
  refs/heads/master 2974406d1 -> 8c954d2cd


[SPARK-21925] Update trigger interval documentation in docs with behavior 
change in Spark 2.2

Forgot to update docs with behavior change.

Author: Burak Yavuz 

Closes #19138 from brkyvz/trigger-doc-fix.


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

Branch: refs/heads/master
Commit: 8c954d2cd10a2cf729d2971fbeb19b2dd751a178
Parents: 2974406
Author: Burak Yavuz 
Authored: Tue Sep 5 13:10:32 2017 -0700
Committer: Tathagata Das 
Committed: Tue Sep 5 13:10:32 2017 -0700

--
 docs/structured-streaming-programming-guide.md | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/8c954d2c/docs/structured-streaming-programming-guide.md
--
diff --git a/docs/structured-streaming-programming-guide.md 
b/docs/structured-streaming-programming-guide.md
index 8367f5a..13a6a82 100644
--- a/docs/structured-streaming-programming-guide.md
+++ b/docs/structured-streaming-programming-guide.md
@@ -1168,7 +1168,7 @@ returned through `Dataset.writeStream()`. You will have 
to specify one or more o
 
 - *Query name:* Optionally, specify a unique name of the query for 
identification.
 
-- *Trigger interval:* Optionally, specify the trigger interval. If it is not 
specified, the system will check for availability of new data as soon as the 
previous processing has completed. If a trigger time is missed because the 
previous processing has not completed, then the system will attempt to trigger 
at the next trigger point, not immediately after the processing has completed.
+- *Trigger interval:* Optionally, specify the trigger interval. If it is not 
specified, the system will check for availability of new data as soon as the 
previous processing has completed. If a trigger time is missed because the 
previous processing has not completed, then the system will trigger processing 
immediately.
 
 - *Checkpoint location:* For some output sinks where the end-to-end 
fault-tolerance can be guaranteed, specify the location where the system will 
write all the checkpoint information. This should be a directory in an 
HDFS-compatible fault-tolerant file system. The semantics of checkpointing is 
discussed in more detail in the next section.
 


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



  1   2   3   4   5   6   7   8   9   >