[GitHub] spark pull request #17141: [SPARK-19800][SS][WIP] Implement one kind of stre...

2017-06-08 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/spark/pull/17141


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #17141: [SPARK-19800][SS][WIP] Implement one kind of stre...

2017-03-02 Thread uncleGen
Github user uncleGen commented on a diff in the pull request:

https://github.com/apache/spark/pull/17141#discussion_r104073813
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/streaming/ReservoirSampleSuit.scala
 ---
@@ -0,0 +1,134 @@
+/*
+ * 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 org.scalatest.BeforeAndAfterAll
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.catalyst.streaming.InternalOutputModes._
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.execution.streaming.state.StateStore
+
+class ReservoirSampleSuit extends StateStoreMetricsTest with 
BeforeAndAfterAll {
+
+  import testImplicits._
+
+  override def afterAll(): Unit = {
+super.afterAll()
+StateStore.stop()
+  }
+
+  test("streaming reservoir sample: reservoir size is larger than stream 
data size - update mode") {
+val inputData = MemoryStream[String]
+val result = inputData.toDS().reservoir(4)
+
+testStream(result, Update)(
+  AddData(inputData, "a", "b"),
+  CheckAnswer(Row("a"), Row("b")),
+  AddData(inputData, "a"),
+  CheckAnswer(Row("a"), Row("b"), Row("a"))
+)
+  }
+
+  test("streaming reservoir sample: reservoir size is less than stream 
data size - update mode") {
+val inputData = MemoryStream[String]
+val result = inputData.toDS().reservoir(1)
+
+testStream(result, Update)(
+  AddData(inputData, "a", "a"),
+  CheckLastBatch(Row("a")),
+  AddData(inputData, "b", "b", "b", "b", "b", "b", "b", "b"),
+  CheckLastBatch(Row("b"))
+)
+  }
+
+  test("streaming reservoir sample with aggregation - update mode") {
+val inputData = MemoryStream[String]
+val result = inputData.toDS().reservoir(3).groupBy("value").count()
+
+testStream(result, Update)(
+  AddData(inputData, "a"),
+  CheckAnswer(Row("a", 1)),
+  AddData(inputData, "b"),
+  CheckAnswer(Row("a", 1), Row("b", 1))
+)
+  }
+
+  test("streaming reservoir sample with watermark") {
+val inputData = MemoryStream[Int]
+val result = inputData.toDS()
+  .withColumn("eventTime", $"value".cast("timestamp"))
+  .withWatermark("eventTime", "10 seconds")
+  .reservoir(10)
+  .select($"eventTime".cast("long").as[Long])
+
+testStream(result, Update)(
+  AddData(inputData, (1 to 1).flatMap(_ => (11 to 15)): _*),
+  CheckLastBatch(11 to 15: _*),
+  AddData(inputData, 25), // Advance watermark to 15 seconds
+  CheckLastBatch(25),
+  AddData(inputData, 25), // Drop states less than watermark
+  CheckLastBatch(25),
+  AddData(inputData, 10), // Should not emit anything as data less 
than watermark
+  CheckLastBatch(),
+  AddData(inputData, 45), // Advance watermark to 35 seconds
+  CheckLastBatch(45),
+  AddData(inputData, 25), // Should not emit anything as data less 
than watermark
+  CheckLastBatch()
+)
+  }
+
+  test("streaming reservoir sample with aggregation - complete mode") {
+val inputData = MemoryStream[(String, Int)]
+val result = inputData.toDS().select($"_1" as "key", $"_2" as "value")
+  .reservoir(3).groupBy("key").max("value")
+
+testStream(result, Complete)(
+  AddData(inputData, ("a", 1)),
+  CheckAnswer(Row("a", 1)),
+  AddData(inputData, ("b", 2)),
+  CheckAnswer(Row("a", 1), Row("b", 2)),
+  StopStream,
+  StartStream(),
+  AddData(inputData, ("a", 10)),
+  CheckAnswer(Row("a", 10), Row("b", 2)),
+  AddData(inputData, (1 to 10).map(e => ("c", 100)): _*),
+  CheckAnswer(Row("a", 10), Row("b", 2), Row("c", 100))
+)
+  }
+
+  test("batch reservoir sample") {
+val df = spark.createDataset(Array(1, 2, 3, 

[GitHub] spark pull request #17141: [SPARK-19800][SS][WIP] Implement one kind of stre...

2017-03-02 Thread uncleGen
Github user uncleGen commented on a diff in the pull request:

https://github.com/apache/spark/pull/17141#discussion_r104073535
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala
 ---
@@ -397,3 +402,110 @@ object StreamingDeduplicateExec {
   private val EMPTY_ROW =
 
UnsafeProjection.create(Array[DataType](NullType)).apply(InternalRow.apply(null))
 }
+
+/**
+ * Physical operator for executing streaming Sampling.
+ *
+ * @param k random sample k elements.
+ */
+case class StreamingReservoirSampleExec(
+keyExpressions: Seq[Attribute],
+child: SparkPlan,
+k: Int,
+stateId: Option[OperatorStateId] = None,
+eventTimeWatermark: Option[Long] = None,
+outputMode: Option[OutputMode] = None)
+  extends UnaryExecNode with StateStoreWriter with WatermarkSupport {
+
+  override def requiredChildDistribution: Seq[Distribution] =
+  ClusteredDistribution(keyExpressions) :: Nil
+
+  private val enc = Encoders.STRING.asInstanceOf[ExpressionEncoder[String]]
+  private val NUM_RECORDS_IN_PARTITION = 
enc.toRow("NUM_RECORDS_IN_PARTITION")
+.asInstanceOf[UnsafeRow]
+
+  override protected def doExecute(): RDD[InternalRow] = {
+metrics
+val fieldTypes = (keyExpressions.map(_.dataType) ++ 
Seq(LongType)).toArray
+val withSumFieldTypes = (keyExpressions.map(_.dataType) ++ 
Seq(LongType)).toArray
+
+child.execute().mapPartitionsWithStateStore(
+  getStateId.checkpointLocation,
+  getStateId.operatorId,
+  getStateId.batchId,
+  keyExpressions.toStructType,
+  child.output.toStructType,
+  sqlContext.sessionState,
+  Some(sqlContext.streams.stateStoreCoordinator)) { (store, iter) =>
+
+  val numRecordsInPart = store.get(NUM_RECORDS_IN_PARTITION).map(value 
=> {
+value.get(0, LongType).asInstanceOf[Long]
+  }).getOrElse(0L)
+
+  val seed = Random.nextLong()
+  val rand = new XORShiftRandom(seed)
+  var numSamples = numRecordsInPart
+  var count = 0
+
+  val baseIterator = watermarkPredicate match {
+case Some(predicate) => iter.filter((row: InternalRow) => 
!predicate.eval(row))
+case None => iter
+  }
+
+  baseIterator.foreach { r =>
+count += 1
+if (numSamples < k) {
+  numSamples += 1
+  store.put(enc.toRow(numSamples.toString).asInstanceOf[UnsafeRow],
+r.asInstanceOf[UnsafeRow])
+} else {
+  val randomIdx = (rand.nextDouble() * (numRecordsInPart + 
count)).toLong
+  if (randomIdx <= k) {
+val replacementIdx = 
enc.toRow(randomIdx.toString).asInstanceOf[UnsafeRow]
+store.put(replacementIdx, r.asInstanceOf[UnsafeRow])
+  }
+}
+  }
+
+  val numRecordsTillNow = 
UnsafeProjection.create(Array[DataType](LongType))
+.apply(InternalRow.apply(numRecordsInPart + count))
+  store.put(NUM_RECORDS_IN_PARTITION, numRecordsTillNow)
+  store.commit()
+
+  outputMode match {
+case Some(Complete) =>
+  CompletionIterator[InternalRow, Iterator[InternalRow]](
+store.iterator().filter(kv => {
+  
!kv._1.asInstanceOf[UnsafeRow].equals(NUM_RECORDS_IN_PARTITION)
+}).map(kv => {
+  
UnsafeProjection.create(withSumFieldTypes).apply(InternalRow.fromSeq(
+new JoinedRow(kv._2, numRecordsTillNow)
+  .toSeq(withSumFieldTypes)))
+}), {})
+case Some(Update) =>
+  CompletionIterator[InternalRow, Iterator[InternalRow]](
+store.updates()
+  .filter(update => 
!update.key.equals(NUM_RECORDS_IN_PARTITION))
+  .map(update => {
+
UnsafeProjection.create(withSumFieldTypes).apply(InternalRow.fromSeq(
+new JoinedRow(update.value, numRecordsTillNow)
+  .toSeq(withSumFieldTypes)))
--- End diff --

same


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #17141: [SPARK-19800][SS][WIP] Implement one kind of stre...

2017-03-02 Thread uncleGen
Github user uncleGen commented on a diff in the pull request:

https://github.com/apache/spark/pull/17141#discussion_r104073674
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala
 ---
@@ -397,3 +402,110 @@ object StreamingDeduplicateExec {
   private val EMPTY_ROW =
 
UnsafeProjection.create(Array[DataType](NullType)).apply(InternalRow.apply(null))
 }
+
+/**
+ * Physical operator for executing streaming Sampling.
+ *
+ * @param k random sample k elements.
+ */
+case class StreamingReservoirSampleExec(
+keyExpressions: Seq[Attribute],
+child: SparkPlan,
+k: Int,
+stateId: Option[OperatorStateId] = None,
+eventTimeWatermark: Option[Long] = None,
+outputMode: Option[OutputMode] = None)
+  extends UnaryExecNode with StateStoreWriter with WatermarkSupport {
+
+  override def requiredChildDistribution: Seq[Distribution] =
+  ClusteredDistribution(keyExpressions) :: Nil
+
+  private val enc = Encoders.STRING.asInstanceOf[ExpressionEncoder[String]]
+  private val NUM_RECORDS_IN_PARTITION = 
enc.toRow("NUM_RECORDS_IN_PARTITION")
+.asInstanceOf[UnsafeRow]
+
+  override protected def doExecute(): RDD[InternalRow] = {
+metrics
+val fieldTypes = (keyExpressions.map(_.dataType) ++ 
Seq(LongType)).toArray
+val withSumFieldTypes = (keyExpressions.map(_.dataType) ++ 
Seq(LongType)).toArray
+
+child.execute().mapPartitionsWithStateStore(
+  getStateId.checkpointLocation,
+  getStateId.operatorId,
+  getStateId.batchId,
+  keyExpressions.toStructType,
+  child.output.toStructType,
+  sqlContext.sessionState,
+  Some(sqlContext.streams.stateStoreCoordinator)) { (store, iter) =>
+
+  val numRecordsInPart = store.get(NUM_RECORDS_IN_PARTITION).map(value 
=> {
+value.get(0, LongType).asInstanceOf[Long]
+  }).getOrElse(0L)
+
+  val seed = Random.nextLong()
+  val rand = new XORShiftRandom(seed)
+  var numSamples = numRecordsInPart
+  var count = 0
+
+  val baseIterator = watermarkPredicate match {
+case Some(predicate) => iter.filter((row: InternalRow) => 
!predicate.eval(row))
+case None => iter
+  }
+
+  baseIterator.foreach { r =>
+count += 1
+if (numSamples < k) {
+  numSamples += 1
+  store.put(enc.toRow(numSamples.toString).asInstanceOf[UnsafeRow],
+r.asInstanceOf[UnsafeRow])
+} else {
+  val randomIdx = (rand.nextDouble() * (numRecordsInPart + 
count)).toLong
+  if (randomIdx <= k) {
+val replacementIdx = 
enc.toRow(randomIdx.toString).asInstanceOf[UnsafeRow]
+store.put(replacementIdx, r.asInstanceOf[UnsafeRow])
+  }
+}
+  }
--- End diff --

In partiton, we just need to do once normal (without weight) reservoir 
sampling.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #17141: [SPARK-19800][SS][WIP] Implement one kind of stre...

2017-03-02 Thread uncleGen
Github user uncleGen commented on a diff in the pull request:

https://github.com/apache/spark/pull/17141#discussion_r104073607
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala
 ---
@@ -397,3 +402,110 @@ object StreamingDeduplicateExec {
   private val EMPTY_ROW =
 
UnsafeProjection.create(Array[DataType](NullType)).apply(InternalRow.apply(null))
 }
+
+/**
+ * Physical operator for executing streaming Sampling.
+ *
+ * @param k random sample k elements.
+ */
+case class StreamingReservoirSampleExec(
+keyExpressions: Seq[Attribute],
+child: SparkPlan,
+k: Int,
+stateId: Option[OperatorStateId] = None,
+eventTimeWatermark: Option[Long] = None,
+outputMode: Option[OutputMode] = None)
+  extends UnaryExecNode with StateStoreWriter with WatermarkSupport {
+
+  override def requiredChildDistribution: Seq[Distribution] =
+  ClusteredDistribution(keyExpressions) :: Nil
+
+  private val enc = Encoders.STRING.asInstanceOf[ExpressionEncoder[String]]
+  private val NUM_RECORDS_IN_PARTITION = 
enc.toRow("NUM_RECORDS_IN_PARTITION")
+.asInstanceOf[UnsafeRow]
+
+  override protected def doExecute(): RDD[InternalRow] = {
+metrics
+val fieldTypes = (keyExpressions.map(_.dataType) ++ 
Seq(LongType)).toArray
+val withSumFieldTypes = (keyExpressions.map(_.dataType) ++ 
Seq(LongType)).toArray
+
+child.execute().mapPartitionsWithStateStore(
+  getStateId.checkpointLocation,
+  getStateId.operatorId,
+  getStateId.batchId,
+  keyExpressions.toStructType,
+  child.output.toStructType,
+  sqlContext.sessionState,
+  Some(sqlContext.streams.stateStoreCoordinator)) { (store, iter) =>
+
+  val numRecordsInPart = store.get(NUM_RECORDS_IN_PARTITION).map(value 
=> {
+value.get(0, LongType).asInstanceOf[Long]
+  }).getOrElse(0L)
+
+  val seed = Random.nextLong()
+  val rand = new XORShiftRandom(seed)
+  var numSamples = numRecordsInPart
+  var count = 0
+
+  val baseIterator = watermarkPredicate match {
+case Some(predicate) => iter.filter((row: InternalRow) => 
!predicate.eval(row))
+case None => iter
+  }
+
+  baseIterator.foreach { r =>
+count += 1
+if (numSamples < k) {
+  numSamples += 1
+  store.put(enc.toRow(numSamples.toString).asInstanceOf[UnsafeRow],
+r.asInstanceOf[UnsafeRow])
+} else {
+  val randomIdx = (rand.nextDouble() * (numRecordsInPart + 
count)).toLong
+  if (randomIdx <= k) {
+val replacementIdx = 
enc.toRow(randomIdx.toString).asInstanceOf[UnsafeRow]
+store.put(replacementIdx, r.asInstanceOf[UnsafeRow])
+  }
+}
+  }
+
+  val numRecordsTillNow = 
UnsafeProjection.create(Array[DataType](LongType))
+.apply(InternalRow.apply(numRecordsInPart + count))
+  store.put(NUM_RECORDS_IN_PARTITION, numRecordsTillNow)
+  store.commit()
+
+  outputMode match {
+case Some(Complete) =>
+  CompletionIterator[InternalRow, Iterator[InternalRow]](
+store.iterator().filter(kv => {
+  
!kv._1.asInstanceOf[UnsafeRow].equals(NUM_RECORDS_IN_PARTITION)
+}).map(kv => {
+  
UnsafeProjection.create(withSumFieldTypes).apply(InternalRow.fromSeq(
+new JoinedRow(kv._2, numRecordsTillNow)
+  .toSeq(withSumFieldTypes)))
+}), {})
+case Some(Update) =>
+  CompletionIterator[InternalRow, Iterator[InternalRow]](
+store.updates()
+  .filter(update => 
!update.key.equals(NUM_RECORDS_IN_PARTITION))
+  .map(update => {
+
UnsafeProjection.create(withSumFieldTypes).apply(InternalRow.fromSeq(
+new JoinedRow(update.value, numRecordsTillNow)
+  .toSeq(withSumFieldTypes)))
+  }), {})
+case _ =>
+  throw new UnsupportedOperationException(s"Invalid output mode: 
$outputMode " +
+s"for streaming sampling.")
+  }
+}.repartition(1).mapPartitions(it => {
+  SamplingUtils.reservoirSampleWithWeight(
+it.map(item => (item, item.getLong(keyExpressions.size))), k)
+.map(row =>
+  UnsafeProjection.create(fieldTypes)
+.apply(InternalRow.fromSeq(row.toSeq(fieldTypes)))
+).iterator
+})
+  }
--- End diff --

here, we do once global weight reservoir sampling.


---
If your project is set up for it, you can reply to 

[GitHub] spark pull request #17141: [SPARK-19800][SS][WIP] Implement one kind of stre...

2017-03-02 Thread uncleGen
Github user uncleGen commented on a diff in the pull request:

https://github.com/apache/spark/pull/17141#discussion_r104073516
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala
 ---
@@ -397,3 +402,110 @@ object StreamingDeduplicateExec {
   private val EMPTY_ROW =
 
UnsafeProjection.create(Array[DataType](NullType)).apply(InternalRow.apply(null))
 }
+
+/**
+ * Physical operator for executing streaming Sampling.
+ *
+ * @param k random sample k elements.
+ */
+case class StreamingReservoirSampleExec(
+keyExpressions: Seq[Attribute],
+child: SparkPlan,
+k: Int,
+stateId: Option[OperatorStateId] = None,
+eventTimeWatermark: Option[Long] = None,
+outputMode: Option[OutputMode] = None)
+  extends UnaryExecNode with StateStoreWriter with WatermarkSupport {
+
+  override def requiredChildDistribution: Seq[Distribution] =
+  ClusteredDistribution(keyExpressions) :: Nil
+
+  private val enc = Encoders.STRING.asInstanceOf[ExpressionEncoder[String]]
+  private val NUM_RECORDS_IN_PARTITION = 
enc.toRow("NUM_RECORDS_IN_PARTITION")
+.asInstanceOf[UnsafeRow]
+
+  override protected def doExecute(): RDD[InternalRow] = {
+metrics
+val fieldTypes = (keyExpressions.map(_.dataType) ++ 
Seq(LongType)).toArray
+val withSumFieldTypes = (keyExpressions.map(_.dataType) ++ 
Seq(LongType)).toArray
+
+child.execute().mapPartitionsWithStateStore(
+  getStateId.checkpointLocation,
+  getStateId.operatorId,
+  getStateId.batchId,
+  keyExpressions.toStructType,
+  child.output.toStructType,
+  sqlContext.sessionState,
+  Some(sqlContext.streams.stateStoreCoordinator)) { (store, iter) =>
+
+  val numRecordsInPart = store.get(NUM_RECORDS_IN_PARTITION).map(value 
=> {
+value.get(0, LongType).asInstanceOf[Long]
+  }).getOrElse(0L)
+
+  val seed = Random.nextLong()
+  val rand = new XORShiftRandom(seed)
+  var numSamples = numRecordsInPart
+  var count = 0
+
+  val baseIterator = watermarkPredicate match {
+case Some(predicate) => iter.filter((row: InternalRow) => 
!predicate.eval(row))
+case None => iter
+  }
+
+  baseIterator.foreach { r =>
+count += 1
+if (numSamples < k) {
+  numSamples += 1
+  store.put(enc.toRow(numSamples.toString).asInstanceOf[UnsafeRow],
+r.asInstanceOf[UnsafeRow])
+} else {
+  val randomIdx = (rand.nextDouble() * (numRecordsInPart + 
count)).toLong
+  if (randomIdx <= k) {
+val replacementIdx = 
enc.toRow(randomIdx.toString).asInstanceOf[UnsafeRow]
+store.put(replacementIdx, r.asInstanceOf[UnsafeRow])
+  }
+}
+  }
+
+  val numRecordsTillNow = 
UnsafeProjection.create(Array[DataType](LongType))
+.apply(InternalRow.apply(numRecordsInPart + count))
+  store.put(NUM_RECORDS_IN_PARTITION, numRecordsTillNow)
+  store.commit()
+
+  outputMode match {
+case Some(Complete) =>
+  CompletionIterator[InternalRow, Iterator[InternalRow]](
+store.iterator().filter(kv => {
+  
!kv._1.asInstanceOf[UnsafeRow].equals(NUM_RECORDS_IN_PARTITION)
+}).map(kv => {
+  
UnsafeProjection.create(withSumFieldTypes).apply(InternalRow.fromSeq(
+new JoinedRow(kv._2, numRecordsTillNow)
+  .toSeq(withSumFieldTypes)))
+}), {})
--- End diff --

Here, we transfer the row to (row, numRecordsTillNow), and 
`numRecordsTillNow` is used to calculate the weight of item.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #17141: [SPARK-19800][SS][WIP] Implement one kind of stre...

2017-03-02 Thread uncleGen
Github user uncleGen commented on a diff in the pull request:

https://github.com/apache/spark/pull/17141#discussion_r104073290
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala
 ---
@@ -397,3 +402,110 @@ object StreamingDeduplicateExec {
   private val EMPTY_ROW =
 
UnsafeProjection.create(Array[DataType](NullType)).apply(InternalRow.apply(null))
 }
+
+/**
+ * Physical operator for executing streaming Sampling.
+ *
+ * @param k random sample k elements.
+ */
+case class StreamingReservoirSampleExec(
+keyExpressions: Seq[Attribute],
+child: SparkPlan,
+k: Int,
+stateId: Option[OperatorStateId] = None,
+eventTimeWatermark: Option[Long] = None,
+outputMode: Option[OutputMode] = None)
+  extends UnaryExecNode with StateStoreWriter with WatermarkSupport {
+
+  override def requiredChildDistribution: Seq[Distribution] =
+  ClusteredDistribution(keyExpressions) :: Nil
+
+  private val enc = Encoders.STRING.asInstanceOf[ExpressionEncoder[String]]
+  private val NUM_RECORDS_IN_PARTITION = 
enc.toRow("NUM_RECORDS_IN_PARTITION")
+.asInstanceOf[UnsafeRow]
+
--- End diff --

`NUM_RECORDS_IN_PARTITION ` calculate the total number of records in 
current partiton, and update at the end of sample.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #17141: [SPARK-19800][SS][WIP] Implement one kind of stre...

2017-03-02 Thread uncleGen
GitHub user uncleGen opened a pull request:

https://github.com/apache/spark/pull/17141

[SPARK-19800][SS][WIP] Implement one kind of streaming sampling - reservoir 
sampling

## What changes were proposed in this pull request?

This pr adds a special streaming sample operator to support `sample`. It 
add a new evolving operator `reservoir`, and introduce new logical plan 
`ReservoirSample` and two physical plan `StreamingReservoirSampleExec` and 
`ReservoirSampleExec`. 

The following cases are supported:

- batch table reservoir sampling
- stream table reservoir sampling with/without aggregation and watermark in 
Update/Complete output mode

Not supported cases:

- reservoir sampling in Append output mode

Followups:

- move `reservoir` into `sample` operator

## How was this patch tested?

add new unit tests.


You can merge this pull request into a Git repository by running:

$ git pull https://github.com/uncleGen/spark sampling

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/spark/pull/17141.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #17141






---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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