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