From cbf78370cbb898929fa5909cb2640176a28281ff Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Tue, 5 Dec 2017 10:48:20 -0800 Subject: [PATCH 01/33] add tests --- .../sql/sources/v2/BaseStreamingSink.java | 7 + .../sql/sources/v2/BaseStreamingSource.java | 17 ++ .../sql/sources/v2/ContinuousReadSupport.java | 25 +++ .../sources/v2/ContinuousWriteSupport.java | 39 +++++ .../sql/sources/v2/DataSourceV2Options.java | 8 + .../sql/sources/v2/MicroBatchReadSupport.java | 26 +++ .../sources/v2/MicroBatchWriteSupport.java | 40 +++++ .../v2/reader/ContinuousDataReader.java | 32 ++++ .../sources/v2/reader/ContinuousReader.java | 44 +++++ .../sources/v2/reader/DataSourceV2Reader.java | 3 + .../sources/v2/reader/MicroBatchReader.java | 35 ++++ .../sources/v2/writer/ContinuousWriter.java | 24 +++ .../sql/execution/streaming/Offset.scala | 8 + .../streaming/RateSourceProvider.scala | 22 ++- .../streaming/RateStreamSourceV2.scala | 102 ++++++++++++ .../ContinuousRateStreamSource.scala | 151 ++++++++++++++++++ .../streaming/RateSourceV2Suite.scala | 134 ++++++++++++++++ 17 files changed, 716 insertions(+), 1 deletion(-) create mode 100644 sql/core/src/main/java/org/apache/spark/sql/sources/v2/BaseStreamingSink.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/sources/v2/BaseStreamingSource.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousReadSupport.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousWriteSupport.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupport.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchWriteSupport.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousDataReader.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousReader.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/MicroBatchReader.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/ContinuousWriter.java create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateStreamSourceV2.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/RateSourceV2Suite.scala diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/BaseStreamingSink.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/BaseStreamingSink.java new file mode 100644 index 0000000000000..c67077ef42bec --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/BaseStreamingSink.java @@ -0,0 +1,7 @@ +package org.apache.spark.sql.sources.v2; + +/** + * The shared interface between V1 and V2 streaming sinks. + */ +public interface BaseStreamingSink { +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/BaseStreamingSource.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/BaseStreamingSource.java new file mode 100644 index 0000000000000..df9f20687d287 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/BaseStreamingSource.java @@ -0,0 +1,17 @@ +package org.apache.spark.sql.sources.v2; + +import org.apache.spark.sql.execution.streaming.Offset; + +/** + * The shared interface between V1 streaming sources and V2 streaming readers. + */ +public interface BaseStreamingSource { + /** + * Informs the source that Spark has completed processing all data for offsets less than or + * equal to `end` and will only request offsets greater than `end` in the future. + */ + void commit(Offset end); + + /** Stop this source and free any resources it has allocated. */ + void stop(); +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousReadSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousReadSupport.java new file mode 100644 index 0000000000000..4579833dbccc0 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousReadSupport.java @@ -0,0 +1,25 @@ +package org.apache.spark.sql.sources.v2; + +import java.util.Optional; + +import org.apache.spark.sql.execution.streaming.Offset; +import org.apache.spark.sql.sources.v2.reader.ContinuousReader; +import org.apache.spark.sql.sources.v2.reader.DataSourceV2Reader; +import org.apache.spark.sql.types.StructType; + +/** + * A mix-in interface for {@link DataSourceV2}. Data sources can implement this interface to + * provide data reading ability for continuous stream processing. + */ +public interface ContinuousReadSupport extends DataSourceV2 { + /** + * Creates a {@link DataSourceV2Reader} to scan the data from this data source. + * + * If this method fails (by throwing an exception), the action would fail and no Spark job was + * submitted. + * + * @param options the options for the returned data source reader, which is an immutable + * case-insensitive string-to-string map. + */ + ContinuousReader createContinuousReader(Optional schema, String checkpointLocation, DataSourceV2Options options); +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousWriteSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousWriteSupport.java new file mode 100644 index 0000000000000..37ae49489c689 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousWriteSupport.java @@ -0,0 +1,39 @@ +package org.apache.spark.sql.sources.v2; + +import java.util.Optional; + +import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.sql.sources.v2.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; + +/** + * A mix-in interface for {@link DataSourceV2}. Data sources can implement this interface to + * provide data writing ability for continuous stream processing. + */ +@InterfaceStability.Evolving +public interface ContinuousWriteSupport extends BaseStreamingSink { + + /** + * Creates an optional {@link DataSourceV2Writer} to save the data to this data source. Data + * sources can return None if there is no writing needed to be done. + * + * If this method fails (by throwing an exception), the action would fail and no Spark job was + * submitted. + * + * @param queryId A unique string for the writing query. It's possible that there are many writing + * queries running at the same time, and the returned {@link DataSourceV2Writer} + * can use this id to distinguish itself from others. + * @param schema the schema of the data to be written. + * @param mode the output mode which determines what successive batch output means to this + * source, please refer to {@link OutputMode} for more details. + * @param options the options for the returned data source writer, which is an immutable + * case-insensitive string-to-string map. + */ + Optional createContinuousWriter( + String queryId, + StructType schema, + OutputMode mode, + DataSourceV2Options options); +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/DataSourceV2Options.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/DataSourceV2Options.java index b2c908dc73a61..deaad81f93969 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/DataSourceV2Options.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/DataSourceV2Options.java @@ -36,6 +36,10 @@ private String toLowerCase(String key) { return key.toLowerCase(Locale.ROOT); } + public static DataSourceV2Options empty() { + return new DataSourceV2Options(new HashMap<>()); + } + public DataSourceV2Options(Map originalMap) { keyLowerCasedMap = new HashMap<>(originalMap.size()); for (Map.Entry entry : originalMap.entrySet()) { @@ -43,6 +47,10 @@ public DataSourceV2Options(Map originalMap) { } } + public Map asMap() { + return new HashMap<>(keyLowerCasedMap); + } + /** * Returns the option value to which the specified key is mapped, case-insensitively. */ diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupport.java new file mode 100644 index 0000000000000..d7190d0ce0388 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupport.java @@ -0,0 +1,26 @@ +package org.apache.spark.sql.sources.v2; + +import java.util.Optional; + +import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.sql.execution.streaming.Offset; +import org.apache.spark.sql.sources.v2.reader.MicroBatchReader; +import org.apache.spark.sql.types.StructType; + +/** + * A mix-in interface for {@link DataSourceV2}. Data sources can implement this interface to + * provide streaming micro-batch data reading ability. + */ +@InterfaceStability.Evolving +public interface MicroBatchReadSupport extends DataSourceV2 { + /** + * Creates a {@link MicroBatchReader} to scan a batch of data from this data source. + * + * If this method fails (by throwing an exception), the action would fail and no Spark job was + * submitted. + * + * @param options the options for the returned data source reader, which is an immutable + * case-insensitive string-to-string map. + */ + MicroBatchReader createMicroBatchReader(Optional schema, String checkpointLocation, DataSourceV2Options options); +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchWriteSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchWriteSupport.java new file mode 100644 index 0000000000000..09a75cad9704c --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchWriteSupport.java @@ -0,0 +1,40 @@ +package org.apache.spark.sql.sources.v2; + +import java.util.Optional; + +import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.sql.sources.v2.writer.DataSourceV2Writer; +import org.apache.spark.sql.streaming.OutputMode; +import org.apache.spark.sql.types.StructType; + +/** + * A mix-in interface for {@link DataSourceV2}. Data sources can implement this interface to + * provide data writing ability and save the data from a microbatch to the data source. + */ +@InterfaceStability.Evolving +public interface MicroBatchWriteSupport extends BaseStreamingSink { + + /** + * Creates an optional {@link DataSourceV2Writer} to save the data to this data source. Data + * sources can return None if there is no writing needed to be done. + * + * If this method fails (by throwing an exception), the action would fail and no Spark job was + * submitted. + * + * @param queryId A unique string for the writing query. It's possible that there are many writing + * queries running at the same time, and the returned {@link DataSourceV2Writer} + * can use this id to distinguish itself from others. + * @param batchId The numeric ID of the batch within this writing query. + * @param schema the schema of the data to be written. + * @param mode the output mode which determines what successive batch output means to this + * source, please refer to {@link OutputMode} for more details. + * @param options the options for the returned data source writer, which is an immutable + * case-insensitive string-to-string map. + */ + Optional createMicroBatchWriter( + String queryId, + long batchId, + StructType schema, + OutputMode mode, + DataSourceV2Options options); +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousDataReader.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousDataReader.java new file mode 100644 index 0000000000000..4d03e7f5d3b31 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousDataReader.java @@ -0,0 +1,32 @@ +package org.apache.spark.sql.sources.v2.reader; + +import org.apache.spark.sql.execution.streaming.Offset; +import org.apache.spark.sql.execution.streaming.PartitionOffset; + +import java.io.Closeable; +import java.io.IOException; + +/** + * A variation on {@link DataReader} for use with streaming in continuous processing mode. + */ +public interface ContinuousDataReader extends DataReader { + /** + * Proceed to next record, returning false only if the read is interrupted. + * + * @throws IOException if failure happens during disk/network IO like reading files. + */ + boolean next() throws IOException; + + /** + * Return the current record. This method should return same value until `next` is called. + */ + T get(); + + /** + * Get the offset of the next record; that is, the earliest record after the current result of + * {@link this.get}. + * + * The execution engine will use this offset as a restart checkpoint. + */ + PartitionOffset getOffset(); +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousReader.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousReader.java new file mode 100644 index 0000000000000..37d8ba08d9e98 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousReader.java @@ -0,0 +1,44 @@ +package org.apache.spark.sql.sources.v2.reader; + +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.expressions.UnsafeRow; +import org.apache.spark.sql.execution.streaming.Offset; +import org.apache.spark.sql.execution.streaming.PartitionOffset; +import org.apache.spark.sql.sources.v2.BaseStreamingSource; +import org.apache.spark.sql.types.StructType; + +import java.util.List; +import java.util.Optional; + +/** + * A mix-in interface for {@link DataSourceV2Reader}. Data source readers can implement this + * interface to allow reading in a continuous processing mode stream. + * + * Implementations must ensure each read task output is a {@link ContinuousDataReader}. + */ +public interface ContinuousReader extends BaseStreamingSource, DataSourceV2Reader { + /** + * Merge offsets coming from {@link ContinuousDataReader} instances in each partition to + * a single global offset. + */ + Offset mergeOffsets(PartitionOffset[] offsets); + + /** + * Set the desired start offset for read tasks created from this reader. + * + * @param start The initial offset to scan from. May be None, in which case scan will start from + * the beginning of the stream. + */ + void setOffset(Optional start); + + /** + * The execution engine will call this method in every epoch to determine if new read tasks need + * to be generated, which may be required if for example the underlying source system has had + * partitions added or removed. + * + * If true, the query will be shut down and restarted with a new reader. + */ + default boolean needsReconfiguration() { + return false; + } +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/DataSourceV2Reader.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/DataSourceV2Reader.java index 95ee4a8278322..9b7a5603d1735 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/DataSourceV2Reader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/DataSourceV2Reader.java @@ -46,6 +46,9 @@ * Spark first applies all operator push-down optimizations that this data source supports. Then * Spark collects information this data source reported for further optimizations. Finally Spark * issues the scan request and does the actual data reading. + * + * This reader must be able to get constructed and serve readSchema() without assuming an active + * Spark session. An active session can be assumed when creating read tasks. */ @InterfaceStability.Evolving public interface DataSourceV2Reader { diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/MicroBatchReader.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/MicroBatchReader.java new file mode 100644 index 0000000000000..3c425ea0608f1 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/MicroBatchReader.java @@ -0,0 +1,35 @@ +package org.apache.spark.sql.sources.v2.reader; + +import org.apache.spark.sql.Row; +import org.apache.spark.sql.execution.streaming.Offset; +import org.apache.spark.sql.sources.v2.BaseStreamingSource; + +import java.util.List; +import java.util.Optional; + +/** + * A mix-in interface for {@link DataSourceV2Reader}. Data source readers can implement this + * interface to indicate they allow micro-batch streaming reads. + */ +public interface MicroBatchReader extends DataSourceV2Reader, BaseStreamingSource { + /** + * Set the desired offset range for read tasks created from this reader. + * + * @param start The initial offset to scan from. If absent(), scan from the earliest available + * offset. + * @param end The last offset to include in the scan. If absent(), scan up to an + * implementation-defined inferred endpoint, such as the last available offset + * or the start offset plus a target batch size. + */ + void setOffsetRange(Optional start, Optional end); + + /** + * Returns the current start offset for this reader. + */ + Offset getStart(); + + /** + * Return the current end offset for this reader. + */ + Offset getEnd(); +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/ContinuousWriter.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/ContinuousWriter.java new file mode 100644 index 0000000000000..40d35641d4c34 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/ContinuousWriter.java @@ -0,0 +1,24 @@ +package org.apache.spark.sql.sources.v2.writer; + +import org.apache.spark.annotation.InterfaceStability; + +/** + * A {@link DataSourceV2Writer} for use with continuous stream processing. + */ +@InterfaceStability.Evolving +public interface ContinuousWriter extends DataSourceV2Writer { + /** + * Commits this writing job for the specified epoch with a list of commit messages. The commit + * messages are collected from successful data writers and are produced by + * {@link DataWriter#commit()}. + * + * If this method fails (by throwing an exception), this writing job is considered to have been + * failed, and the execution engine will attempt to call {@link #abort(WriterCommitMessage[])}. + */ + void commit(long epochId, WriterCommitMessage[] messages); + + default void commit(WriterCommitMessage[] messages) { + throw new UnsupportedOperationException( + "Commit without epoch should not be called with ContinuousWriter"); + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Offset.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Offset.scala index 4efcee0f8f9d6..30afc4389c62b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Offset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Offset.scala @@ -58,3 +58,11 @@ abstract class Offset { * that accepts a [[SerializedOffset]] for doing the conversion. */ case class SerializedOffset(override val json: String) extends Offset + +/** + * Used for per-partition offsets in continuous processing. ContinuousReader implementations will + * provide a method to merge these into a global Offset. + * + * These offsets must be serializable. + */ +trait PartitionOffset diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateSourceProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateSourceProvider.scala index 077a4778e34a8..e043e8fdb63e9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateSourceProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateSourceProvider.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution.streaming import java.io._ import java.nio.charset.StandardCharsets +import java.util.Optional import java.util.concurrent.TimeUnit import org.apache.commons.io.IOUtils @@ -28,7 +29,10 @@ import org.apache.spark.network.util.JavaUtils import org.apache.spark.sql.{AnalysisException, DataFrame, SQLContext} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils} +import org.apache.spark.sql.execution.streaming.continuous.ContinuousRateStreamReader import org.apache.spark.sql.sources.{DataSourceRegister, StreamSourceProvider} +import org.apache.spark.sql.sources.v2.{ContinuousReadSupport, DataSourceV2, DataSourceV2Options, MicroBatchReadSupport} +import org.apache.spark.sql.sources.v2.reader.{ContinuousReader, MicroBatchReader} import org.apache.spark.sql.types._ import org.apache.spark.util.{ManualClock, SystemClock} @@ -46,7 +50,8 @@ import org.apache.spark.util.{ManualClock, SystemClock} * generated rows. The source will try its best to reach `rowsPerSecond`, but the query may * be resource constrained, and `numPartitions` can be tweaked to help reach the desired speed. */ -class RateSourceProvider extends StreamSourceProvider with DataSourceRegister { +class RateSourceProvider extends StreamSourceProvider with DataSourceRegister + with DataSourceV2 with MicroBatchReadSupport with ContinuousReadSupport{ override def sourceSchema( sqlContext: SQLContext, @@ -100,6 +105,21 @@ class RateSourceProvider extends StreamSourceProvider with DataSourceRegister { params.get("useManualClock").map(_.toBoolean).getOrElse(false) // Only for testing ) } + + override def createMicroBatchReader( + schema: Optional[StructType], + checkpointLocation: String, + options: DataSourceV2Options): MicroBatchReader = { + new RateStreamV2Reader(options) + } + + override def createContinuousReader( + schema: Optional[StructType], + checkpointLocation: String, + options: DataSourceV2Options): ContinuousReader = { + new ContinuousRateStreamReader(options) + } + override def shortName(): String = "rate" } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateStreamSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateStreamSourceV2.scala new file mode 100644 index 0000000000000..09ca2dbcc098d --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateStreamSourceV2.scala @@ -0,0 +1,102 @@ +/* + * 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 java.util.Optional + +import scala.collection.JavaConverters._ + +import org.json4s.DefaultFormats +import org.json4s.jackson.Serialization + +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.execution.streaming.continuous.{ContinuousRateStreamSource, RateStreamDataReader, RateStreamReadTask} +import org.apache.spark.sql.sources.v2.DataSourceV2Options +import org.apache.spark.sql.sources.v2.reader._ +import org.apache.spark.sql.types.{LongType, StructField, StructType, TimestampType} +import org.apache.spark.util.SystemClock + +class RateStreamV2Reader(options: DataSourceV2Options) + extends MicroBatchReader { + implicit val defaultFormats: DefaultFormats = DefaultFormats + + val clock = new SystemClock + + private val numPartitions = + options.get(ContinuousRateStreamSource.NUM_PARTITIONS).orElse("5").toInt + private val rowsPerSecond = + options.get(ContinuousRateStreamSource.ROWS_PER_SECOND).orElse("6").toLong + + override def readSchema(): StructType = { + StructType( + StructField("timestamp", TimestampType, false) :: + StructField("value", LongType, false) :: Nil) + } + + val creationTimeMs = clock.getTimeMillis() + + private var start: Offset = _ + private var end: Offset = _ + + override def setOffsetRange(start: Optional[Offset], end: Optional[Offset]): Unit = { + this.start = start.orElse(LongOffset(creationTimeMs)) + this.end = end.orElse(LongOffset(clock.getTimeMillis())) + } + + override def getStart(): Offset = start + override def getEnd(): Offset = end + + override def createReadTasks(): java.util.List[ReadTask[Row]] = { + val startTime = LongOffset.convert(start).get.offset + val numSeconds = (LongOffset.convert(end).get.offset - startTime) / 1000 + val firstValue = (startTime - creationTimeMs) / 1000 + + (firstValue to firstValue + numSeconds * rowsPerSecond - 1) + .groupBy(_ % numPartitions) + .values + .map(vals => RateStreamBatchTask(vals).asInstanceOf[ReadTask[Row]]) + .toList + .asJava + } + + override def commit(end: Offset): Unit = {} + override def stop(): Unit = {} +} + +case class RateStreamBatchTask(vals: Seq[Long]) extends ReadTask[Row] { + override def createDataReader(): DataReader[Row] = new RateStreamBatchReader(vals) +} + +class RateStreamBatchReader(vals: Seq[Long]) extends DataReader[Row] { + var currentIndex = -1 + + override def next(): Boolean = { + // Return true as long as the new index is in the seq. + currentIndex += 1 + currentIndex < vals.size + } + + override def get(): Row = { + Row( + DateTimeUtils.toJavaTimestamp(DateTimeUtils.fromMillis(System.currentTimeMillis)), + vals(currentIndex)) + } + + override def close(): Unit = {} +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala new file mode 100644 index 0000000000000..4b868580772cc --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala @@ -0,0 +1,151 @@ +/* + * 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 + +import scala.collection.JavaConverters._ + +import org.json4s.DefaultFormats +import org.json4s.jackson.Serialization + +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.expressions.UnsafeRow +import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.sources.v2.{ContinuousReadSupport, DataSourceV2, DataSourceV2Options} +import org.apache.spark.sql.sources.v2.reader._ +import org.apache.spark.sql.types.{LongType, StructField, StructType, TimestampType} + +object ContinuousRateStreamSource { + val NUM_PARTITIONS = "numPartitions" + val ROWS_PER_SECOND = "rowsPerSecond" +} + +case class RateStreamOffset(partitionToStartValue: Map[Int, Long]) extends Offset { + implicit val defaultFormats: DefaultFormats = DefaultFormats + override val json = Serialization.write(partitionToStartValue) +} + +case class ContinuousRateStreamPartitionOffset(partition: Int, start: Long) extends PartitionOffset + +class ContinuousRateStreamReader(options: DataSourceV2Options) + extends ContinuousReader { + implicit val defaultFormats: DefaultFormats = DefaultFormats + + val numPartitions = options.get(ContinuousRateStreamSource.NUM_PARTITIONS).orElse("5").toInt + val rowsPerSecond = options.get(ContinuousRateStreamSource.ROWS_PER_SECOND).orElse("6").toLong + + override def mergeOffsets(offsets: Array[PartitionOffset]): Offset = { + + assert(offsets.length == numPartitions) + val tuples = offsets.map { + case ContinuousRateStreamPartitionOffset(p, s) => p -> s + } + RateStreamOffset(Map(tuples: _*)) + } + + override def readSchema(): StructType = { + StructType( + StructField("timestamp", TimestampType, false) :: + StructField("value", LongType, false) :: Nil) + } + + private var offset: java.util.Optional[Offset] = _ + + override def setOffset(offset: java.util.Optional[Offset]): Unit = { + this.offset = offset + } + + override def createReadTasks(): java.util.List[ReadTask[Row]] = { + val partitionStartMap = Option(offset.orElse(null)).map { + case o: RateStreamOffset => o.partitionToStartValue + case s: SerializedOffset => Serialization.read[Map[Int, Long]](s.json) + case _ => throw new IllegalArgumentException("invalid offset type for ContinuousRateSource") + } + if (partitionStartMap.exists(_.keySet.size > numPartitions)) { + throw new IllegalArgumentException("Start offset contained too many partitions.") + } + val perPartitionRate = rowsPerSecond.toDouble / numPartitions.toDouble + + Range(0, numPartitions).map { n => + // If the offset doesn't have a value for this partition, start from the beginning. + val start = partitionStartMap.flatMap(_.get(n)).getOrElse(0L + n) + // Have each partition advance by numPartitions each row, with starting points staggered + // by their partition index. + RateStreamReadTask(start, n, numPartitions, perPartitionRate) + .asInstanceOf[ReadTask[Row]] + }.asJava + } + + override def commit(end: Offset): Unit = {} + override def stop(): Unit = {} + +} + +case class RateStreamReadTask( + startValue: Long, partitionIndex: Int, increment: Long, rowsPerSecond: Double) + extends ReadTask[Row] { + override def createDataReader(): DataReader[Row] = + new RateStreamDataReader(startValue, partitionIndex, increment, rowsPerSecond.toLong) +} + +class RateStreamDataReader( + startValue: Long, partitionIndex: Int, increment: Long, rowsPerSecond: Long) + extends ContinuousDataReader[Row] { + private var nextReadTime = 0L + private var numReadRows = 0L + + private var currentValue = startValue + private var currentRow: Row = null + + override def next(): Boolean = { + // Set the timestamp for the first time. + if (currentRow == null) nextReadTime = System.currentTimeMillis() + 1000 + + if (numReadRows == rowsPerSecond) { + // Sleep until we reach the next second. + + try { + while (System.currentTimeMillis < nextReadTime) { + Thread.sleep(nextReadTime - System.currentTimeMillis) + } + } catch { + case _: InterruptedException => + // Someone's trying to end the task; just let them. + return false + } + numReadRows = 0 + nextReadTime += 1000 + } + + currentRow = Row( + DateTimeUtils.toJavaTimestamp(DateTimeUtils.fromMillis(System.currentTimeMillis)), + currentValue) + currentValue += increment + numReadRows += 1 + + true + } + + override def get: Row = currentRow + + override def close(): Unit = {} + + // We use the value corresponding to partition 0 as the offset. + override def getOffset(): PartitionOffset = + ContinuousRateStreamPartitionOffset(partitionIndex, currentValue) +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/RateSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/RateSourceV2Suite.scala new file mode 100644 index 0000000000000..888793da6bb82 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/RateSourceV2Suite.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.execution.streaming + +import java.util.Optional + +import scala.collection.JavaConverters._ + +import org.apache.spark.sql.Row +import org.apache.spark.sql.execution.datasources.DataSource +import org.apache.spark.sql.execution.streaming.continuous.{ContinuousRateStreamPartitionOffset, ContinuousRateStreamReader, RateStreamDataReader, RateStreamReadTask} +import org.apache.spark.sql.sources.v2.{ContinuousReadSupport, DataSourceV2Options, MicroBatchReadSupport} +import org.apache.spark.sql.streaming.StreamTest + +class RateSourceV2Suite extends StreamTest { + test("microbatch in registry") { + DataSource.lookupDataSource("rate").newInstance() match { + case ds: MicroBatchReadSupport => + val reader = ds.createMicroBatchReader(Optional.empty(), "", DataSourceV2Options.empty()) + assert(reader.isInstanceOf[RateStreamV2Reader]) + case _ => + throw new IllegalStateException("Could not find v2 read support for rate") + } + } + + test("microbatch - options propagated") { + val reader = new RateStreamV2Reader( + new DataSourceV2Options(Map("numPartitions" -> "11", "rowsPerSecond" -> "33").asJava)) + reader.setOffsetRange(Optional.empty(), + Optional.of(LongOffset(System.currentTimeMillis() + 1001))) + val tasks = reader.createReadTasks() + assert(tasks.size == 11) + tasks.asScala.foreach { + // for 1 second, size of each task is (rowsPerSecond / numPartitions) + case RateStreamBatchTask(vals) => vals.size == 3 + case _ => throw new IllegalStateException("Unexpected task type") + } + } + + test("microbatch - set offset") { + val reader = new RateStreamV2Reader(DataSourceV2Options.empty()) + reader.setOffsetRange(Optional.of(LongOffset(12345)), Optional.of(LongOffset(54321))) + assert(reader.getStart() == LongOffset(12345)) + assert(reader.getEnd() == LongOffset(54321)) + } + + test("microbatch - infer offsets") { + val reader = new RateStreamV2Reader(DataSourceV2Options.empty()) + reader.clock.waitTillTime(reader.clock.getTimeMillis() + 100) + reader.setOffsetRange(Optional.empty(), Optional.empty()) + assert(reader.getStart() == LongOffset(reader.creationTimeMs)) + assert(reader.getEnd().asInstanceOf[LongOffset].offset >= reader.creationTimeMs + 100) + } + + + test("microbatch - data read") { + val reader = new RateStreamV2Reader( + new DataSourceV2Options(Map("numPartitions" -> "11", "rowsPerSecond" -> "33").asJava)) + reader.setOffsetRange(Optional.empty(), + Optional.of(LongOffset(System.currentTimeMillis() + 1001))) + val tasks = reader.createReadTasks() + assert(tasks.size == 11) + + val readData = tasks.asScala + .map(_.createDataReader()) + .flatMap { reader => + val buf = scala.collection.mutable.ListBuffer[Row]() + while (reader.next()) buf.append(reader.get()) + buf + } + + assert(readData.map(_.getLong(1)).sorted == Range(0, 33)) + } + + test("continuous in registry") { + DataSource.lookupDataSource("rate").newInstance() match { + case ds: ContinuousReadSupport => + val reader = ds.createContinuousReader(Optional.empty(), "", DataSourceV2Options.empty()) + assert(reader.isInstanceOf[ContinuousRateStreamReader]) + case _ => + throw new IllegalStateException("Could not find v2 read support for rate") + } + } + + test("continuous data") { + val reader = new ContinuousRateStreamReader( + new DataSourceV2Options(Map("numPartitions" -> "2", "rowsPerSecond" -> "10").asJava)) + reader.setOffset(Optional.empty()) + val tasks = reader.createReadTasks() + assert(tasks.size == 2) + + val data = scala.collection.mutable.ListBuffer[Row]() + tasks.asScala.foreach { + case t: RateStreamReadTask => + val startTime = System.currentTimeMillis() + val r = t.createDataReader().asInstanceOf[RateStreamDataReader] + // The first set of (rowsPerSecond / numPartitions) should come ~immediately, but the + // next should only come after 1 second. + for (i <- 1 to 5) { + r.next() + data.append(r.get()) + assert(r.getOffset() == + ContinuousRateStreamPartitionOffset(t.partitionIndex, r.get.getLong(1) + 2)) + } + assert(System.currentTimeMillis() < startTime + 100) + for (i <- 1 to 5) { + r.next() + data.append(r.get()) + assert(r.getOffset() == + ContinuousRateStreamPartitionOffset(t.partitionIndex, r.get.getLong(1) + 2)) + } + assert(System.currentTimeMillis() > startTime + 1000) + + case _ => throw new IllegalStateException("Unexpected task type") + } + + assert(data.map(_.getLong(1)).toSeq.sorted == Range(0, 20)) + } +} From 10ac59981f69475612f7f66d733606ed947aacdd Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Tue, 5 Dec 2017 11:35:36 -0800 Subject: [PATCH 02/33] writer impl --- .../sql/execution/streaming/memoryV2.scala | 185 ++++++++++++++++++ .../streaming/MemorySinkV2Suite.scala | 83 ++++++++ 2 files changed, 268 insertions(+) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memoryV2.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MemorySinkV2Suite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memoryV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memoryV2.scala new file mode 100644 index 0000000000000..f780e4e08a216 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memoryV2.scala @@ -0,0 +1,185 @@ +/* + * 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 javax.annotation.concurrent.GuardedBy + +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer +import scala.util.control.NonFatal + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.{Append, Complete, Update} +import org.apache.spark.sql.sources.v2.{ContinuousWriteSupport, DataSourceV2, DataSourceV2Options, MicroBatchWriteSupport} +import org.apache.spark.sql.sources.v2.writer._ +import org.apache.spark.sql.streaming.OutputMode +import org.apache.spark.sql.types.StructType + +/** + * A sink that stores the results in memory. This [[Sink]] is primarily intended for use in unit + * tests and does not provide durability. + */ +class MemorySinkV2 extends DataSourceV2 + with MicroBatchWriteSupport with ContinuousWriteSupport with Logging { + + override def createMicroBatchWriter( + queryId: String, + batchId: Long, + schema: StructType, + mode: OutputMode, + options: DataSourceV2Options): java.util.Optional[DataSourceV2Writer] = { + java.util.Optional.of(new MemoryWriter(this, batchId, mode)) + } + + override def createContinuousWriter( + queryId: String, + schema: StructType, + mode: OutputMode, + options: DataSourceV2Options): java.util.Optional[ContinuousWriter] = { + java.util.Optional.of(new ContinuousMemoryWriter(this, mode)) + } + + private case class AddedData(batchId: Long, data: Array[Row]) + + /** An order list of batches that have been written to this [[Sink]]. */ + @GuardedBy("this") + private val batches = new ArrayBuffer[AddedData]() + + /** Returns all rows that are stored in this [[Sink]]. */ + def allData: Seq[Row] = synchronized { + batches.flatMap(_.data) + } + + def latestBatchId: Option[Long] = synchronized { + batches.lastOption.map(_.batchId) + } + + def latestBatchData: Seq[Row] = synchronized { + batches.lastOption.toSeq.flatten(_.data) + } + + def toDebugString: String = synchronized { + batches.map { case AddedData(batchId, data) => + val dataStr = try data.mkString(" ") catch { + case NonFatal(e) => "[Error converting to string]" + } + s"$batchId: $dataStr" + }.mkString("\n") + } + + def write(batchId: Long, outputMode: OutputMode, newRows: Array[Row]): Unit = { + val notCommitted = synchronized { + latestBatchId.isEmpty || batchId > latestBatchId.get + } + if (notCommitted) { + logDebug(s"Committing batch $batchId to $this") + outputMode match { + case Append | Update => + val rows = AddedData(batchId, newRows) + synchronized { batches += rows } + + case Complete => + val rows = AddedData(batchId, newRows) + synchronized { + batches.clear() + batches += rows + } + + case _ => + throw new IllegalArgumentException( + s"Output mode $outputMode is not supported by MemorySink") + } + } else { + logDebug(s"Skipping already committed batch: $batchId") + } + } + + def clear(): Unit = synchronized { + batches.clear() + } + + override def toString(): String = "MemorySink" +} + +case class MemoryWriterCommitMessage(partition: Int, data: Seq[Row]) extends WriterCommitMessage {} + +class MemoryWriter(sink: MemorySinkV2, batchId: Long, outputMode: OutputMode) + extends DataSourceV2Writer with Logging { + + override def createWriterFactory: MemoryWriterFactory = MemoryWriterFactory(outputMode) + + def commit(messages: Array[WriterCommitMessage]): Unit = { + val newRows = messages.flatMap { message => + // TODO remove + if (message != null) { + assert(message.isInstanceOf[MemoryWriterCommitMessage]) + message.asInstanceOf[MemoryWriterCommitMessage].data + } else { + Seq() + } + } + sink.write(batchId, outputMode, newRows) + } + + override def abort(messages: Array[WriterCommitMessage]): Unit = { + // Don't accept any of the new input. + } +} + +class ContinuousMemoryWriter(val sink: MemorySinkV2, outputMode: OutputMode) + extends ContinuousWriter { + + override def createWriterFactory: MemoryWriterFactory = MemoryWriterFactory(outputMode) + + override def commit(epochId: Long, messages: Array[WriterCommitMessage]): Unit = { + val newRows = messages.flatMap { + case message: MemoryWriterCommitMessage => message.data + case _ => Seq() + } + sink.write(epochId, outputMode, newRows) + } + + override def abort(messages: Array[WriterCommitMessage]): Unit = { + // Don't accept any of the new input. + } +} + +case class MemoryWriterFactory(outputMode: OutputMode) extends DataWriterFactory[Row] { + def createDataWriter(partitionId: Int, attemptNumber: Int): DataWriter[Row] = { + new MemoryDataWriter(partitionId, outputMode) + } +} + +class MemoryDataWriter(partition: Int, outputMode: OutputMode) + extends DataWriter[Row] with Logging { + + private val data = mutable.Buffer[Row]() + + override def write(row: Row): Unit = { + data.append(row) + } + + override def commit(): MemoryWriterCommitMessage = { + val msg = MemoryWriterCommitMessage(partition, data.clone()) + data.clear() + msg + } + + override def abort(): Unit = {} +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MemorySinkV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MemorySinkV2Suite.scala new file mode 100644 index 0000000000000..86da56548454a --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MemorySinkV2Suite.scala @@ -0,0 +1,83 @@ +/* + * 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.scalatest.BeforeAndAfter + +import org.apache.spark.sql.Row +import org.apache.spark.sql.streaming.{OutputMode, StreamTest} + +class MemorySinkV2Suite extends StreamTest with BeforeAndAfter { + test("data writer") { + val partition = 1234 + val writer = new MemoryDataWriter(partition, OutputMode.Append()) + writer.write(Row(1)) + writer.write(Row(2)) + writer.write(Row(44)) + val msg = writer.commit() + assert(msg.data.map(_.getInt(0)) == Seq(1, 2, 44)) + assert(msg.partition == partition) + + // Buffer should be cleared, so repeated commits should give empty. + assert(writer.commit().data.isEmpty) + } + + test("continuous writer") { + val sink = new MemorySinkV2 + val writer = new ContinuousMemoryWriter(sink, OutputMode.Append()) + writer.commit(0, + Array( + MemoryWriterCommitMessage(0, Seq(Row(1), Row(2))), + MemoryWriterCommitMessage(1, Seq(Row(3), Row(4))), + MemoryWriterCommitMessage(2, Seq(Row(6), Row(7))) + )) + assert(sink.latestBatchId.contains(0)) + assert(sink.latestBatchData.map(_.getInt(0)).sorted == Seq(1, 2, 3, 4, 6, 7)) + writer.commit(19, + Array( + MemoryWriterCommitMessage(3, Seq(Row(11), Row(22))), + MemoryWriterCommitMessage(0, Seq(Row(33))) + )) + assert(sink.latestBatchId.contains(19)) + assert(sink.latestBatchData.map(_.getInt(0)).sorted == Seq(11, 22, 33)) + + assert(sink.allData.map(_.getInt(0)).sorted == Seq(1, 2, 3, 4, 6, 7, 11, 22, 33)) + } + + test("microbatch writer") { + val sink = new MemorySinkV2 + val writer = new ContinuousMemoryWriter(sink, OutputMode.Append()) + new MemoryWriter(sink, 0, OutputMode.Append()).commit( + Array( + MemoryWriterCommitMessage(0, Seq(Row(1), Row(2))), + MemoryWriterCommitMessage(1, Seq(Row(3), Row(4))), + MemoryWriterCommitMessage(2, Seq(Row(6), Row(7))) + )) + assert(sink.latestBatchId.contains(0)) + assert(sink.latestBatchData.map(_.getInt(0)).sorted == Seq(1, 2, 3, 4, 6, 7)) + new MemoryWriter(sink, 19, OutputMode.Append()).commit( + Array( + MemoryWriterCommitMessage(3, Seq(Row(11), Row(22))), + MemoryWriterCommitMessage(0, Seq(Row(33))) + )) + assert(sink.latestBatchId.contains(19)) + assert(sink.latestBatchData.map(_.getInt(0)).sorted == Seq(11, 22, 33)) + + assert(sink.allData.map(_.getInt(0)).sorted == Seq(1, 2, 3, 4, 6, 7, 11, 22, 33)) + } +} From 5594f7a5770b1a39ea86003fc6aa8ef9d93bc3c1 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Tue, 5 Dec 2017 11:37:24 -0800 Subject: [PATCH 03/33] rm useless writer --- .../apache/spark/sql/execution/streaming/MemorySinkV2Suite.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MemorySinkV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MemorySinkV2Suite.scala index 86da56548454a..be4b490754986 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MemorySinkV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MemorySinkV2Suite.scala @@ -61,7 +61,6 @@ class MemorySinkV2Suite extends StreamTest with BeforeAndAfter { test("microbatch writer") { val sink = new MemorySinkV2 - val writer = new ContinuousMemoryWriter(sink, OutputMode.Append()) new MemoryWriter(sink, 0, OutputMode.Append()).commit( Array( MemoryWriterCommitMessage(0, Seq(Row(1), Row(2))), From a364fa6dca3b51cee869a8ce788d5ff52bc82830 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Tue, 5 Dec 2017 13:02:32 -0800 Subject: [PATCH 04/33] rm weird docs --- .../org/apache/spark/sql/sources/v2/ContinuousReadSupport.java | 3 --- .../apache/spark/sql/sources/v2/ContinuousWriteSupport.java | 3 --- .../org/apache/spark/sql/sources/v2/MicroBatchReadSupport.java | 3 --- .../apache/spark/sql/sources/v2/MicroBatchWriteSupport.java | 3 --- 4 files changed, 12 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousReadSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousReadSupport.java index 4579833dbccc0..232fc26b81491 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousReadSupport.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousReadSupport.java @@ -15,9 +15,6 @@ public interface ContinuousReadSupport extends DataSourceV2 { /** * Creates a {@link DataSourceV2Reader} to scan the data from this data source. * - * If this method fails (by throwing an exception), the action would fail and no Spark job was - * submitted. - * * @param options the options for the returned data source reader, which is an immutable * case-insensitive string-to-string map. */ diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousWriteSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousWriteSupport.java index 37ae49489c689..5b54d822b276f 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousWriteSupport.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousWriteSupport.java @@ -19,9 +19,6 @@ public interface ContinuousWriteSupport extends BaseStreamingSink { * Creates an optional {@link DataSourceV2Writer} to save the data to this data source. Data * sources can return None if there is no writing needed to be done. * - * If this method fails (by throwing an exception), the action would fail and no Spark job was - * submitted. - * * @param queryId A unique string for the writing query. It's possible that there are many writing * queries running at the same time, and the returned {@link DataSourceV2Writer} * can use this id to distinguish itself from others. diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupport.java index d7190d0ce0388..39d8a0ba207a4 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupport.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupport.java @@ -16,9 +16,6 @@ public interface MicroBatchReadSupport extends DataSourceV2 { /** * Creates a {@link MicroBatchReader} to scan a batch of data from this data source. * - * If this method fails (by throwing an exception), the action would fail and no Spark job was - * submitted. - * * @param options the options for the returned data source reader, which is an immutable * case-insensitive string-to-string map. */ diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchWriteSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchWriteSupport.java index 09a75cad9704c..c17f099cc9f07 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchWriteSupport.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchWriteSupport.java @@ -18,9 +18,6 @@ public interface MicroBatchWriteSupport extends BaseStreamingSink { * Creates an optional {@link DataSourceV2Writer} to save the data to this data source. Data * sources can return None if there is no writing needed to be done. * - * If this method fails (by throwing an exception), the action would fail and no Spark job was - * submitted. - * * @param queryId A unique string for the writing query. It's possible that there are many writing * queries running at the same time, and the returned {@link DataSourceV2Writer} * can use this id to distinguish itself from others. From 41d732c3aa7740f43cb40ceeb878ea28d716e411 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Tue, 5 Dec 2017 13:59:50 -0800 Subject: [PATCH 05/33] shuffle around public interfaces --- .../spark/sql/kafka010/KafkaSource.scala | 3 +- .../sql/kafka010/KafkaSourceOffset.scala | 3 +- .../spark/sql/kafka010/KafkaSourceSuite.scala | 3 +- .../sql/sources/v2/ContinuousReadSupport.java | 1 - .../sources/v2/ContinuousWriteSupport.java | 1 + .../sql/sources/v2/MicroBatchReadSupport.java | 1 - .../sources/v2/MicroBatchWriteSupport.java | 1 + .../v2/reader/ContinuousDataReader.java | 4 +- .../sources/v2/reader/ContinuousReader.java | 9 +--- .../sources/v2/reader/MicroBatchReader.java | 6 +-- .../spark/sql/sources/v2/reader/Offset.java | 54 +++++++++++++++++++ .../sources/v2/reader/PartitionOffset.java | 28 ++++++++++ .../streaming}/BaseStreamingSink.java | 2 +- .../streaming}/BaseStreamingSource.java | 4 +- .../streaming/FileStreamSource.scala | 1 + .../streaming/FileStreamSourceOffset.scala | 3 ++ .../sql/execution/streaming/LongOffset.scala | 2 + .../sql/execution/streaming/Offset.scala | 40 +------------- .../sql/execution/streaming/OffsetSeq.scala | 1 + .../execution/streaming/OffsetSeqLog.scala | 1 + .../streaming/RateSourceProvider.scala | 4 +- .../sql/execution/streaming/Source.scala | 1 + .../execution/streaming/StreamExecution.scala | 1 + .../execution/streaming/StreamProgress.scala | 2 + .../sql/execution/streaming/memory.scala | 1 + .../sql/execution/streaming/socket.scala | 1 + .../streaming/OffsetSeqLogSuite.scala | 1 + .../execution/streaming/RateSourceSuite.scala | 1 + .../sql/streaming/FileStreamSourceSuite.scala | 1 + .../spark/sql/streaming/OffsetSuite.scala | 3 +- .../spark/sql/streaming/StreamSuite.scala | 1 + .../spark/sql/streaming/StreamTest.scala | 1 + .../streaming/StreamingAggregationSuite.scala | 1 + .../StreamingQueryListenerSuite.scala | 1 + .../sql/streaming/StreamingQuerySuite.scala | 1 + .../test/DataStreamReaderWriterSuite.scala | 1 + .../sql/streaming/util/BlockingSource.scala | 3 +- 37 files changed, 129 insertions(+), 64 deletions(-) create mode 100644 sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/Offset.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionOffset.java rename sql/core/src/main/{java/org/apache/spark/sql/sources/v2 => scala/org/apache/spark/sql/execution/streaming}/BaseStreamingSink.java (68%) rename sql/core/src/main/{java/org/apache/spark/sql/sources/v2 => scala/org/apache/spark/sql/execution/streaming}/BaseStreamingSource.java (80%) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala index e9cff04ba5f2e..57be83758fa8e 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala @@ -23,8 +23,8 @@ import java.nio.charset.StandardCharsets import org.apache.commons.io.IOUtils import org.apache.kafka.common.TopicPartition - import org.apache.spark.SparkContext + import org.apache.spark.internal.Logging import org.apache.spark.scheduler.ExecutorCacheTaskLocation import org.apache.spark.sql._ @@ -32,6 +32,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.kafka010.KafkaSource._ +import org.apache.spark.sql.sources.v2.reader.Offset import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala index b5da415b3097e..6e24423df4abc 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala @@ -19,7 +19,8 @@ package org.apache.spark.sql.kafka010 import org.apache.kafka.common.TopicPartition -import org.apache.spark.sql.execution.streaming.{Offset, SerializedOffset} +import org.apache.spark.sql.execution.streaming.SerializedOffset +import org.apache.spark.sql.sources.v2.reader.Offset /** * An [[Offset]] for the [[KafkaSource]]. This one tracks all partitions of subscribed topics and 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 2034b9be07f24..7308cc69c198d 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 @@ -32,12 +32,13 @@ import org.apache.kafka.common.TopicPartition import org.scalatest.concurrent.Eventually._ 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.execution.streaming._ import org.apache.spark.sql.functions.{count, window} import org.apache.spark.sql.kafka010.KafkaSourceProvider._ +import org.apache.spark.sql.sources.v2.reader.Offset import org.apache.spark.sql.streaming.{ProcessingTime, StreamTest} import org.apache.spark.sql.streaming.util.StreamManualClock import org.apache.spark.sql.test.{SharedSQLContext, TestSparkSession} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousReadSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousReadSupport.java index 232fc26b81491..ba2b3b217aba7 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousReadSupport.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousReadSupport.java @@ -2,7 +2,6 @@ import java.util.Optional; -import org.apache.spark.sql.execution.streaming.Offset; import org.apache.spark.sql.sources.v2.reader.ContinuousReader; import org.apache.spark.sql.sources.v2.reader.DataSourceV2Reader; import org.apache.spark.sql.types.StructType; diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousWriteSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousWriteSupport.java index 5b54d822b276f..81217e1e8f257 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousWriteSupport.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousWriteSupport.java @@ -3,6 +3,7 @@ import java.util.Optional; import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.sql.execution.streaming.BaseStreamingSink; import org.apache.spark.sql.sources.v2.writer.ContinuousWriter; import org.apache.spark.sql.sources.v2.writer.DataSourceV2Writer; import org.apache.spark.sql.streaming.OutputMode; diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupport.java index 39d8a0ba207a4..7c372db7789c2 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupport.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupport.java @@ -3,7 +3,6 @@ import java.util.Optional; import org.apache.spark.annotation.InterfaceStability; -import org.apache.spark.sql.execution.streaming.Offset; import org.apache.spark.sql.sources.v2.reader.MicroBatchReader; import org.apache.spark.sql.types.StructType; diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchWriteSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchWriteSupport.java index c17f099cc9f07..1ab6b70985c0c 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchWriteSupport.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchWriteSupport.java @@ -3,6 +3,7 @@ import java.util.Optional; import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.sql.execution.streaming.BaseStreamingSink; import org.apache.spark.sql.sources.v2.writer.DataSourceV2Writer; import org.apache.spark.sql.streaming.OutputMode; import org.apache.spark.sql.types.StructType; diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousDataReader.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousDataReader.java index 4d03e7f5d3b31..17e84cb249947 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousDataReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousDataReader.java @@ -1,9 +1,7 @@ package org.apache.spark.sql.sources.v2.reader; -import org.apache.spark.sql.execution.streaming.Offset; -import org.apache.spark.sql.execution.streaming.PartitionOffset; +import org.apache.spark.sql.sources.v2.reader.PartitionOffset; -import java.io.Closeable; import java.io.IOException; /** diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousReader.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousReader.java index 37d8ba08d9e98..b3fede06a893d 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousReader.java @@ -1,13 +1,8 @@ package org.apache.spark.sql.sources.v2.reader; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.catalyst.expressions.UnsafeRow; -import org.apache.spark.sql.execution.streaming.Offset; -import org.apache.spark.sql.execution.streaming.PartitionOffset; -import org.apache.spark.sql.sources.v2.BaseStreamingSource; -import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.sources.v2.reader.PartitionOffset; +import org.apache.spark.sql.execution.streaming.BaseStreamingSource; -import java.util.List; import java.util.Optional; /** diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/MicroBatchReader.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/MicroBatchReader.java index 3c425ea0608f1..11ee4ad94cc8d 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/MicroBatchReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/MicroBatchReader.java @@ -1,10 +1,8 @@ package org.apache.spark.sql.sources.v2.reader; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.execution.streaming.Offset; -import org.apache.spark.sql.sources.v2.BaseStreamingSource; +import org.apache.spark.sql.sources.v2.reader.Offset; +import org.apache.spark.sql.execution.streaming.BaseStreamingSource; -import java.util.List; import java.util.Optional; /** diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/Offset.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/Offset.java new file mode 100644 index 0000000000000..a5d7f1bd99ae9 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/Offset.java @@ -0,0 +1,54 @@ +/* + * 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.sources.v2.reader; + +public abstract class Offset { + /** + * A JSON-serialized representation of an Offset that is + * used for saving offsets to the offset log. + * Note: We assume that equivalent/equal offsets serialize to + * identical JSON strings. + * + * @return JSON string encoding + */ + public abstract String json(); + + /** + * Equality based on JSON string representation. We leverage the + * JSON representation for normalization between the Offset's + * in memory and on disk representations. + */ + @Override + public boolean equals(Object obj) { + if (obj instanceof Offset) { + return this.json().equals(((Offset) obj).json()); + } else { + return false; + } + } + + @Override + public int hashCode() { + return this.json().hashCode(); + } + + @Override + public String toString() { + return this.json(); + } +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionOffset.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionOffset.java new file mode 100644 index 0000000000000..c1b0081d80749 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionOffset.java @@ -0,0 +1,28 @@ +/* + * 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.sources.v2.reader; + +/** + * Used for per-partition offsets in continuous processing. ContinuousReader implementations will + * provide a method to merge these into a global Offset. + * + * These offsets must be serializable. + */ +public interface PartitionOffset { + +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/BaseStreamingSink.java b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/BaseStreamingSink.java similarity index 68% rename from sql/core/src/main/java/org/apache/spark/sql/sources/v2/BaseStreamingSink.java rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/BaseStreamingSink.java index c67077ef42bec..a33096ab29175 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/BaseStreamingSink.java +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/BaseStreamingSink.java @@ -1,4 +1,4 @@ -package org.apache.spark.sql.sources.v2; +package org.apache.spark.sql.execution.streaming; /** * The shared interface between V1 and V2 streaming sinks. diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/BaseStreamingSource.java b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/BaseStreamingSource.java similarity index 80% rename from sql/core/src/main/java/org/apache/spark/sql/sources/v2/BaseStreamingSource.java rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/BaseStreamingSource.java index df9f20687d287..01c37f01c6919 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/BaseStreamingSource.java +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/BaseStreamingSource.java @@ -1,6 +1,6 @@ -package org.apache.spark.sql.sources.v2; +package org.apache.spark.sql.execution.streaming; -import org.apache.spark.sql.execution.streaming.Offset; +import org.apache.spark.sql.sources.v2.reader.Offset; /** * The shared interface between V1 streaming sources and V2 streaming readers. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala index 0debd7db84757..a33b785126765 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala @@ -27,6 +27,7 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging import org.apache.spark.sql.{DataFrame, Dataset, SparkSession} import org.apache.spark.sql.execution.datasources.{DataSource, InMemoryFileIndex, LogicalRelation} +import org.apache.spark.sql.sources.v2.reader.Offset import org.apache.spark.sql.types.StructType /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceOffset.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceOffset.scala index 06d0fe6c18c1e..431e5b99e3e98 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceOffset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceOffset.scala @@ -22,8 +22,11 @@ import scala.util.control.Exception._ import org.json4s.NoTypeHints import org.json4s.jackson.Serialization +import org.apache.spark.sql.sources.v2.reader.Offset + /** * Offset for the [[FileStreamSource]]. + * * @param logOffset Position in the [[FileStreamSourceLog]] */ case class FileStreamSourceOffset(logOffset: Long) extends Offset { 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 5f0b195fcfcb8..7ea31462ca7b0 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,6 +17,8 @@ package org.apache.spark.sql.execution.streaming +import org.apache.spark.sql.sources.v2.reader.Offset + /** * A simple offset for sources that produce a single linear stream of data. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Offset.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Offset.scala index 30afc4389c62b..73f0c6221c5c1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Offset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Offset.scala @@ -17,38 +17,8 @@ package org.apache.spark.sql.execution.streaming -/** - * An offset is a monotonically increasing metric used to track progress in the computation of a - * stream. Since offsets are retrieved from a [[Source]] by a single thread, we know the global - * ordering of two [[Offset]] instances. We do assume that if two offsets are `equal` then no - * new data has arrived. - */ -abstract class Offset { - - /** - * Equality based on JSON string representation. We leverage the - * JSON representation for normalization between the Offset's - * in memory and on disk representations. - */ - override def equals(obj: Any): Boolean = obj match { - case o: Offset => this.json == o.json - case _ => false - } - - override def hashCode(): Int = this.json.hashCode - - override def toString(): String = this.json.toString +import org.apache.spark.sql.sources.v2.reader.Offset - /** - * A JSON-serialized representation of an Offset that is - * used for saving offsets to the offset log. - * Note: We assume that equivalent/equal offsets serialize to - * identical JSON strings. - * - * @return JSON string encoding - */ - def json: String -} /** * Used when loading a JSON serialized offset from external storage. @@ -59,10 +29,4 @@ abstract class Offset { */ case class SerializedOffset(override val json: String) extends Offset -/** - * Used for per-partition offsets in continuous processing. ContinuousReader implementations will - * provide a method to merge these into a global Offset. - * - * These offsets must be serializable. - */ -trait PartitionOffset + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeq.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeq.scala index 4e0a468b962a2..dcc5935890c8d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeq.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeq.scala @@ -23,6 +23,7 @@ import org.json4s.jackson.Serialization import org.apache.spark.internal.Logging import org.apache.spark.sql.RuntimeConfig import org.apache.spark.sql.internal.SQLConf.{SHUFFLE_PARTITIONS, STATE_STORE_PROVIDER_CLASS} +import org.apache.spark.sql.sources.v2.reader.Offset /** * An ordered collection of offsets, used to track the progress of processing data from one or more diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLog.scala index e3f4abcf9f1dc..bfdbc65296165 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLog.scala @@ -24,6 +24,7 @@ import java.nio.charset.StandardCharsets._ import scala.io.{Source => IOSource} import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.sources.v2.reader.Offset /** * This class is used to log offsets to persistent files in HDFS. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateSourceProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateSourceProvider.scala index e043e8fdb63e9..50671a46599e6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateSourceProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateSourceProvider.scala @@ -31,8 +31,8 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils} import org.apache.spark.sql.execution.streaming.continuous.ContinuousRateStreamReader import org.apache.spark.sql.sources.{DataSourceRegister, StreamSourceProvider} -import org.apache.spark.sql.sources.v2.{ContinuousReadSupport, DataSourceV2, DataSourceV2Options, MicroBatchReadSupport} -import org.apache.spark.sql.sources.v2.reader.{ContinuousReader, MicroBatchReader} +import org.apache.spark.sql.sources.v2._ +import org.apache.spark.sql.sources.v2.reader.{ContinuousReader, MicroBatchReader, Offset} import org.apache.spark.sql.types._ import org.apache.spark.util.{ManualClock, SystemClock} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Source.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Source.scala index 311942f6dbd84..dbb408ffc98d8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Source.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Source.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution.streaming import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.sources.v2.reader.Offset import org.apache.spark.sql.types.StructType /** 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 406560c260f07..16063c02ce06b 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 @@ -39,6 +39,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} import org.apache.spark.sql.execution.{QueryExecution, SQLExecution} import org.apache.spark.sql.execution.command.StreamingExplainCommand import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.v2.reader.Offset import org.apache.spark.sql.streaming._ import org.apache.spark.util.{Clock, UninterruptibleThread, Utils} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamProgress.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamProgress.scala index a3f3662e6f4c9..770db401c9fd7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamProgress.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamProgress.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.execution.streaming import scala.collection.{immutable, GenTraversableOnce} +import org.apache.spark.sql.sources.v2.reader.Offset + /** * A helper class that looks like a Map[Source, Offset]. */ 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 3041d4d703cb4..db0717510a2cb 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 @@ -32,6 +32,7 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LocalRelation, Statistics} import org.apache.spark.sql.catalyst.streaming.InternalOutputModes._ import org.apache.spark.sql.execution.SQLExecution +import org.apache.spark.sql.sources.v2.reader.Offset import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types.StructType import org.apache.spark.util.Utils diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/socket.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/socket.scala index 0b22cbc46e6bf..440cae016a173 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/socket.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/socket.scala @@ -31,6 +31,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.sources.{DataSourceRegister, StreamSourceProvider} +import org.apache.spark.sql.sources.v2.reader.Offset import org.apache.spark.sql.types.{StringType, StructField, StructType, TimestampType} import org.apache.spark.unsafe.types.UTF8String diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLogSuite.scala index e6cdc063c4e9f..4868ba4e68934 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLogSuite.scala @@ -22,6 +22,7 @@ import java.io.File import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.util.stringToFile import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.v2.reader.Offset import org.apache.spark.sql.test.SharedSQLContext class OffsetSeqLogSuite extends SparkFunSuite with SharedSQLContext { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/RateSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/RateSourceSuite.scala index 03d0f63fa4d7f..ceba27b26e578 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/RateSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/RateSourceSuite.scala @@ -21,6 +21,7 @@ import java.util.concurrent.TimeUnit import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.functions._ +import org.apache.spark.sql.sources.v2.reader.Offset import org.apache.spark.sql.streaming.{StreamingQueryException, StreamTest} import org.apache.spark.util.ManualClock diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala index b6baaed1927e4..7a2d9e3728208 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala @@ -32,6 +32,7 @@ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.FileStreamSource.{FileEntry, SeenFilesMap} import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.v2.reader.Offset import org.apache.spark.sql.streaming.ExistsThrowsExceptionFileSystem._ import org.apache.spark.sql.streaming.util.StreamManualClock import org.apache.spark.sql.test.SharedSQLContext diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/OffsetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/OffsetSuite.scala index f208f9bd9b6e3..429748261f1ea 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/OffsetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/OffsetSuite.scala @@ -18,7 +18,8 @@ package org.apache.spark.sql.streaming import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.execution.streaming.{LongOffset, Offset, SerializedOffset} +import org.apache.spark.sql.execution.streaming.{LongOffset, SerializedOffset} +import org.apache.spark.sql.sources.v2.reader.Offset trait OffsetSuite extends SparkFunSuite { /** Creates test to check all the comparisons of offsets given a `one` that is less than `two`. */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala index 3d687d2214e90..8163a1f91e1ca 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala @@ -39,6 +39,7 @@ import org.apache.spark.sql.execution.streaming.state.{StateStore, StateStoreCon import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.StreamSourceProvider +import org.apache.spark.sql.sources.v2.reader.Offset import org.apache.spark.sql.streaming.util.StreamManualClock import org.apache.spark.sql.types.{IntegerType, StructField, StructType} import org.apache.spark.util.Utils 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 e68fca050571f..40accf82cbd96 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 @@ -39,6 +39,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.state.StateStore +import org.apache.spark.sql.sources.v2.reader.Offset import org.apache.spark.sql.streaming.StreamingQueryListener._ import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.util.{Clock, SystemClock, Utils} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala index 1b4d8556f6ae5..fa0313592b8e7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala @@ -34,6 +34,7 @@ import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.state.StateStore import org.apache.spark.sql.expressions.scalalang.typed import org.apache.spark.sql.functions._ +import org.apache.spark.sql.sources.v2.reader.Offset import org.apache.spark.sql.streaming.OutputMode._ import org.apache.spark.sql.streaming.util.{MockSourceProvider, StreamManualClock} import org.apache.spark.sql.types.StructType diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala index 9ff02dee288fb..fc9ac2a56c4e5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala @@ -33,6 +33,7 @@ import org.apache.spark.scheduler._ import org.apache.spark.sql.{Encoder, SparkSession} import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.v2.reader.Offset import org.apache.spark.sql.streaming.StreamingQueryListener._ import org.apache.spark.sql.streaming.util.StreamManualClock import org.apache.spark.util.JsonProtocol diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala index cc693909270f8..f813b77e3ce6b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala @@ -33,6 +33,7 @@ import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.v2.reader.Offset import org.apache.spark.sql.streaming.util.{BlockingSource, MockSourceProvider, StreamManualClock} import org.apache.spark.sql.types.StructType import org.apache.spark.util.ManualClock diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala index aa163d2211c38..952908f21ca60 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala @@ -32,6 +32,7 @@ import org.apache.spark.sql._ import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.{StreamSinkProvider, StreamSourceProvider} +import org.apache.spark.sql.sources.v2.reader.Offset import org.apache.spark.sql.streaming.{ProcessingTime => DeprecatedProcessingTime, _} import org.apache.spark.sql.streaming.Trigger._ import org.apache.spark.sql.types._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/util/BlockingSource.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/util/BlockingSource.scala index 19ab2ff13e14e..9a35f097e6e40 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/util/BlockingSource.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/util/BlockingSource.scala @@ -20,8 +20,9 @@ package org.apache.spark.sql.streaming.util import java.util.concurrent.CountDownLatch import org.apache.spark.sql.{SQLContext, _} -import org.apache.spark.sql.execution.streaming.{LongOffset, Offset, Sink, Source} +import org.apache.spark.sql.execution.streaming.{LongOffset, Sink, Source} import org.apache.spark.sql.sources.{StreamSinkProvider, StreamSourceProvider} +import org.apache.spark.sql.sources.v2.reader.Offset import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types.{IntegerType, StructField, StructType} From 60c12c29629dbe27cec4dbb62f6d4d8d4cb7f702 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Tue, 5 Dec 2017 14:02:01 -0800 Subject: [PATCH 06/33] fix imports --- .../main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala | 2 +- .../scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala index 57be83758fa8e..87f31fcc20ae6 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala @@ -23,8 +23,8 @@ import java.nio.charset.StandardCharsets import org.apache.commons.io.IOUtils import org.apache.kafka.common.TopicPartition -import org.apache.spark.SparkContext +import org.apache.spark.SparkContext import org.apache.spark.internal.Logging import org.apache.spark.scheduler.ExecutorCacheTaskLocation import org.apache.spark.sql._ 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 7308cc69c198d..9cac0e5ae7117 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 @@ -32,8 +32,8 @@ import org.apache.kafka.common.TopicPartition import org.scalatest.concurrent.Eventually._ import org.scalatest.concurrent.PatienceConfiguration.Timeout import org.scalatest.time.SpanSugar._ -import org.apache.spark.SparkContext +import org.apache.spark.SparkContext import org.apache.spark.sql.ForeachWriter import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.functions.{count, window} From 57db413cd840cf62434657e322a1c2978160d587 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Tue, 5 Dec 2017 14:40:15 -0800 Subject: [PATCH 07/33] put deserialize in reader so we don't have to port SerializedOffset --- .../apache/spark/sql/sources/v2/MicroBatchWriteSupport.java | 4 ++-- .../spark/sql/sources/v2/reader/ContinuousReader.java | 6 ++++++ .../spark/sql/sources/v2/reader/MicroBatchReader.java | 6 ++++++ 3 files changed, 14 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchWriteSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchWriteSupport.java index 1ab6b70985c0c..5286876e049bd 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchWriteSupport.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchWriteSupport.java @@ -22,7 +22,7 @@ public interface MicroBatchWriteSupport extends BaseStreamingSink { * @param queryId A unique string for the writing query. It's possible that there are many writing * queries running at the same time, and the returned {@link DataSourceV2Writer} * can use this id to distinguish itself from others. - * @param batchId The numeric ID of the batch within this writing query. + * @param epochId The numeric ID of the batch within this writing query. * @param schema the schema of the data to be written. * @param mode the output mode which determines what successive batch output means to this * source, please refer to {@link OutputMode} for more details. @@ -31,7 +31,7 @@ public interface MicroBatchWriteSupport extends BaseStreamingSink { */ Optional createMicroBatchWriter( String queryId, - long batchId, + long epochId, StructType schema, OutputMode mode, DataSourceV2Options options); diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousReader.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousReader.java index b3fede06a893d..6be2c671b1206 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousReader.java @@ -18,6 +18,12 @@ public interface ContinuousReader extends BaseStreamingSource, DataSourceV2Reade */ Offset mergeOffsets(PartitionOffset[] offsets); + /** + * Deserialize a JSON string into an Offset of the implementation-defined offset type. + * @throws IllegalArgumentException if the JSON does not encode a valid offset for this reader + */ + Offset deserialize(String json); + /** * Set the desired start offset for read tasks created from this reader. * diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/MicroBatchReader.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/MicroBatchReader.java index 11ee4ad94cc8d..fbd40a8bbdcbd 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/MicroBatchReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/MicroBatchReader.java @@ -21,6 +21,12 @@ public interface MicroBatchReader extends DataSourceV2Reader, BaseStreamingSourc */ void setOffsetRange(Optional start, Optional end); + /** + * Deserialize a JSON string into an Offset of the implementation-defined offset type. + * @throws IllegalArgumentException if the JSON does not encode a valid offset for this reader + */ + Offset deserialize(String json); + /** * Returns the current start offset for this reader. */ From 93b69762fb37c3d6a913b0cdefee21dce3bbe8bd Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Tue, 5 Dec 2017 15:51:08 -0800 Subject: [PATCH 08/33] off by one errors grr --- .../spark/sql/sources/v2/reader/ContinuousDataReader.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousDataReader.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousDataReader.java index 17e84cb249947..77f5d8657e258 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousDataReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousDataReader.java @@ -21,8 +21,7 @@ public interface ContinuousDataReader extends DataReader { T get(); /** - * Get the offset of the next record; that is, the earliest record after the current result of - * {@link this.get}. + * Get the offset of the current record. * * The execution engine will use this offset as a restart checkpoint. */ From abd20db666645f8b2da6d28b2b06690d348da59f Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Tue, 5 Dec 2017 15:55:58 -0800 Subject: [PATCH 09/33] document right semantics --- .../spark/sql/sources/v2/reader/ContinuousReader.java | 7 +++---- .../spark/sql/sources/v2/reader/MicroBatchReader.java | 4 +++- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousReader.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousReader.java index 6be2c671b1206..45b5acecc9052 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousReader.java @@ -25,10 +25,9 @@ public interface ContinuousReader extends BaseStreamingSource, DataSourceV2Reade Offset deserialize(String json); /** - * Set the desired start offset for read tasks created from this reader. - * - * @param start The initial offset to scan from. May be None, in which case scan will start from - * the beginning of the stream. + * Set the desired start offset for read tasks created from this reader. The scan will start + * from the first record after the provided offset, or from the beginning of the stream if + * Optional.empty() is provided. */ void setOffset(Optional start); diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/MicroBatchReader.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/MicroBatchReader.java index fbd40a8bbdcbd..291cc4cc24209 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/MicroBatchReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/MicroBatchReader.java @@ -11,7 +11,9 @@ */ public interface MicroBatchReader extends DataSourceV2Reader, BaseStreamingSource { /** - * Set the desired offset range for read tasks created from this reader. + * Set the desired offset range for read tasks created from this reader. Read tasks will + * generate only data within (`start`, `end`]; that is, from the first record after `start` to + * the record with offset `end`. * * @param start The initial offset to scan from. If absent(), scan from the earliest available * offset. From 3629d27ea19c5abf1e147aecb083c62fb8dca71d Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Tue, 5 Dec 2017 16:08:54 -0800 Subject: [PATCH 10/33] document checkpoint location --- .../apache/spark/sql/sources/v2/ContinuousReadSupport.java | 4 ++++ .../apache/spark/sql/sources/v2/MicroBatchReadSupport.java | 4 ++++ .../streaming/continuous/ContinuousRateStreamSource.scala | 5 ++++- 3 files changed, 12 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousReadSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousReadSupport.java index ba2b3b217aba7..9713d393fe590 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousReadSupport.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousReadSupport.java @@ -14,6 +14,10 @@ public interface ContinuousReadSupport extends DataSourceV2 { /** * Creates a {@link DataSourceV2Reader} to scan the data from this data source. * + * @param schema the user provided schema, or empty() if none was provided + * @param checkpointLocation a path to HDFS scratch space that can be used for failure recovery. + * Readers for the same logical source in the same query will be + * given the same checkpointLocation. * @param options the options for the returned data source reader, which is an immutable * case-insensitive string-to-string map. */ diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupport.java index 7c372db7789c2..72196369d38c7 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupport.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupport.java @@ -15,6 +15,10 @@ public interface MicroBatchReadSupport extends DataSourceV2 { /** * Creates a {@link MicroBatchReader} to scan a batch of data from this data source. * + * @param schema the user provided schema, or empty() if none was provided + * @param checkpointLocation a path to HDFS scratch space that can be used for failure recovery. + * Readers for the same logical source in the same query will be + * given the same checkpointLocation. * @param options the options for the returned data source reader, which is an immutable * case-insensitive string-to-string map. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala index 4b868580772cc..2a1c7914c821d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala @@ -50,7 +50,6 @@ class ContinuousRateStreamReader(options: DataSourceV2Options) val rowsPerSecond = options.get(ContinuousRateStreamSource.ROWS_PER_SECOND).orElse("6").toLong override def mergeOffsets(offsets: Array[PartitionOffset]): Offset = { - assert(offsets.length == numPartitions) val tuples = offsets.map { case ContinuousRateStreamPartitionOffset(p, s) => p -> s @@ -58,6 +57,10 @@ class ContinuousRateStreamReader(options: DataSourceV2Options) RateStreamOffset(Map(tuples: _*)) } + override def deserialize(json: String): Offset = { + RateStreamOffset(Serialization.read[Map[Int, Long]](json)) + } + override def readSchema(): StructType = { StructType( StructField("timestamp", TimestampType, false) :: From d021f317c74e28b76cb3ef81a87f00173531fde6 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Tue, 5 Dec 2017 16:43:36 -0800 Subject: [PATCH 11/33] add getStart to continuous and clarify semantics --- .../sources/v2/reader/ContinuousReader.java | 7 ++++++ .../sources/v2/reader/MicroBatchReader.java | 22 +++++++++++-------- .../ContinuousRateStreamSource.scala | 2 ++ 3 files changed, 22 insertions(+), 9 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousReader.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousReader.java index 45b5acecc9052..403105688cab4 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousReader.java @@ -31,6 +31,13 @@ public interface ContinuousReader extends BaseStreamingSource, DataSourceV2Reade */ void setOffset(Optional start); + /** + * Return the specified or inferred start offset for this reader. + * + * Should only be called after setOffset. + */ + Offset getStart(); + /** * The execution engine will call this method in every epoch to determine if new read tasks need * to be generated, which may be required if for example the underlying source system has had diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/MicroBatchReader.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/MicroBatchReader.java index 291cc4cc24209..4e7f002ec541d 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/MicroBatchReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/MicroBatchReader.java @@ -15,8 +15,8 @@ public interface MicroBatchReader extends DataSourceV2Reader, BaseStreamingSourc * generate only data within (`start`, `end`]; that is, from the first record after `start` to * the record with offset `end`. * - * @param start The initial offset to scan from. If absent(), scan from the earliest available - * offset. + * @param start The initial offset to scan from. If absent(), the reader should infer the + * earliest available offset and scan from the beginning of the stream. * @param end The last offset to include in the scan. If absent(), scan up to an * implementation-defined inferred endpoint, such as the last available offset * or the start offset plus a target batch size. @@ -24,18 +24,22 @@ public interface MicroBatchReader extends DataSourceV2Reader, BaseStreamingSourc void setOffsetRange(Optional start, Optional end); /** - * Deserialize a JSON string into an Offset of the implementation-defined offset type. - * @throws IllegalArgumentException if the JSON does not encode a valid offset for this reader + * Returns the specified or inferred start offset for this reader. + * + * Should only be called after setOffsetRange. */ - Offset deserialize(String json); + Offset getStart(); /** - * Returns the current start offset for this reader. + * Return the specified or inferred end offset for this reader. + * + * Should only be called after setOffsetRange. */ - Offset getStart(); + Offset getEnd(); /** - * Return the current end offset for this reader. + * Deserialize a JSON string into an Offset of the implementation-defined offset type. + * @throws IllegalArgumentException if the JSON does not encode a valid offset for this reader */ - Offset getEnd(); + Offset deserialize(String json); } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala index 2a1c7914c821d..ac474a2b2bfd4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala @@ -73,6 +73,8 @@ class ContinuousRateStreamReader(options: DataSourceV2Options) this.offset = offset } + override def getStart(): Offset = offset.get() + override def createReadTasks(): java.util.List[ReadTask[Row]] = { val partitionStartMap = Option(offset.orElse(null)).map { case o: RateStreamOffset => o.partitionToStartValue From 052808a87bfba9df068aa6ec8bfb173c6541d618 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Wed, 6 Dec 2017 09:23:34 -0800 Subject: [PATCH 12/33] cleanup offset set/get docs --- .../sql/sources/v2/reader/ContinuousReader.java | 8 ++++---- .../sql/sources/v2/reader/MicroBatchReader.java | 14 +++++++------- .../execution/streaming/RateStreamSourceV2.scala | 10 ++++++++-- .../continuous/ContinuousRateStreamSource.scala | 2 +- .../execution/streaming/RateSourceV2Suite.scala | 8 ++++---- 5 files changed, 24 insertions(+), 18 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousReader.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousReader.java index 403105688cab4..0856a5a1ed02e 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousReader.java @@ -26,17 +26,17 @@ public interface ContinuousReader extends BaseStreamingSource, DataSourceV2Reade /** * Set the desired start offset for read tasks created from this reader. The scan will start - * from the first record after the provided offset, or from the beginning of the stream if - * Optional.empty() is provided. + * from the first record after the provided offset, or from an implementation-defined inferred + * starting point if no offset is provided. */ void setOffset(Optional start); /** * Return the specified or inferred start offset for this reader. * - * Should only be called after setOffset. + * @throws IllegalStateException if setOffset has not been called */ - Offset getStart(); + Offset getStartOffset(); /** * The execution engine will call this method in every epoch to determine if new read tasks need diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/MicroBatchReader.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/MicroBatchReader.java index 4e7f002ec541d..4ca1b0ad746dc 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/MicroBatchReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/MicroBatchReader.java @@ -15,9 +15,9 @@ public interface MicroBatchReader extends DataSourceV2Reader, BaseStreamingSourc * generate only data within (`start`, `end`]; that is, from the first record after `start` to * the record with offset `end`. * - * @param start The initial offset to scan from. If absent(), the reader should infer the - * earliest available offset and scan from the beginning of the stream. - * @param end The last offset to include in the scan. If absent(), scan up to an + * @param start The initial offset to scan from. If not specified, scan from an + * implementation-specified start point, such as the earliest available record. + * @param end The last offset to include in the scan. If not specified, scan up to an * implementation-defined inferred endpoint, such as the last available offset * or the start offset plus a target batch size. */ @@ -26,16 +26,16 @@ public interface MicroBatchReader extends DataSourceV2Reader, BaseStreamingSourc /** * Returns the specified or inferred start offset for this reader. * - * Should only be called after setOffsetRange. + * @throws IllegalStateException if setOffsetRange has not been called */ - Offset getStart(); + Offset getStartOffset(); /** * Return the specified or inferred end offset for this reader. * - * Should only be called after setOffsetRange. + * @throws IllegalStateException if setOffsetRange has not been called */ - Offset getEnd(); + Offset getEndOffset(); /** * Deserialize a JSON string into an Offset of the implementation-defined offset type. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateStreamSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateStreamSourceV2.scala index 09ca2dbcc098d..6a34a318defad 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateStreamSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateStreamSourceV2.scala @@ -59,8 +59,14 @@ class RateStreamV2Reader(options: DataSourceV2Options) this.end = end.orElse(LongOffset(clock.getTimeMillis())) } - override def getStart(): Offset = start - override def getEnd(): Offset = end + override def getStartOffset(): Offset = { + if (start == null) throw new IllegalStateException("start offset not set") + start + } + override def getEndOffset(): Offset = { + if (end == null) throw new IllegalStateException("end offset not set") + end + } override def createReadTasks(): java.util.List[ReadTask[Row]] = { val startTime = LongOffset.convert(start).get.offset diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala index ac474a2b2bfd4..aaa4b5d664324 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala @@ -73,7 +73,7 @@ class ContinuousRateStreamReader(options: DataSourceV2Options) this.offset = offset } - override def getStart(): Offset = offset.get() + override def getStartOffset(): Offset = offset.get() override def createReadTasks(): java.util.List[ReadTask[Row]] = { val partitionStartMap = Option(offset.orElse(null)).map { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/RateSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/RateSourceV2Suite.scala index 888793da6bb82..974eba1e36bd5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/RateSourceV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/RateSourceV2Suite.scala @@ -55,16 +55,16 @@ class RateSourceV2Suite extends StreamTest { test("microbatch - set offset") { val reader = new RateStreamV2Reader(DataSourceV2Options.empty()) reader.setOffsetRange(Optional.of(LongOffset(12345)), Optional.of(LongOffset(54321))) - assert(reader.getStart() == LongOffset(12345)) - assert(reader.getEnd() == LongOffset(54321)) + assert(reader.getStartOffset() == LongOffset(12345)) + assert(reader.getEndOffset() == LongOffset(54321)) } test("microbatch - infer offsets") { val reader = new RateStreamV2Reader(DataSourceV2Options.empty()) reader.clock.waitTillTime(reader.clock.getTimeMillis() + 100) reader.setOffsetRange(Optional.empty(), Optional.empty()) - assert(reader.getStart() == LongOffset(reader.creationTimeMs)) - assert(reader.getEnd().asInstanceOf[LongOffset].offset >= reader.creationTimeMs + 100) + assert(reader.getStartOffset() == LongOffset(reader.creationTimeMs)) + assert(reader.getEndOffset().asInstanceOf[LongOffset].offset >= reader.creationTimeMs + 100) } From 7a7638bdd906ea022afa46f2a7528c49824dd275 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Wed, 6 Dec 2017 09:32:45 -0800 Subject: [PATCH 13/33] cleanup reader docs --- .../spark/sql/sources/v2/ContinuousReadSupport.java | 8 ++++---- .../spark/sql/sources/v2/ContinuousWriteSupport.java | 2 +- .../spark/sql/sources/v2/MicroBatchReadSupport.java | 6 +++--- .../spark/sql/sources/v2/MicroBatchWriteSupport.java | 5 ++++- 4 files changed, 12 insertions(+), 9 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousReadSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousReadSupport.java index 9713d393fe590..e0538178cabb3 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousReadSupport.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousReadSupport.java @@ -12,12 +12,12 @@ */ public interface ContinuousReadSupport extends DataSourceV2 { /** - * Creates a {@link DataSourceV2Reader} to scan the data from this data source. + * Creates a {@link ContinuousReader} to scan the data from this data source. * * @param schema the user provided schema, or empty() if none was provided - * @param checkpointLocation a path to HDFS scratch space that can be used for failure recovery. - * Readers for the same logical source in the same query will be - * given the same checkpointLocation. + * @param checkpointLocation a path to Hadoop FS scratch space that can be used for failure + * recovery. Readers for the same logical source in the same query + * will be given the same checkpointLocation. * @param options the options for the returned data source reader, which is an immutable * case-insensitive string-to-string map. */ diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousWriteSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousWriteSupport.java index 81217e1e8f257..7f87681769515 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousWriteSupport.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousWriteSupport.java @@ -17,7 +17,7 @@ public interface ContinuousWriteSupport extends BaseStreamingSink { /** - * Creates an optional {@link DataSourceV2Writer} to save the data to this data source. Data + * Creates an optional {@link ContinuousWriter} to save the data to this data source. Data * sources can return None if there is no writing needed to be done. * * @param queryId A unique string for the writing query. It's possible that there are many writing diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupport.java index 72196369d38c7..410b36eb4487a 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupport.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupport.java @@ -16,9 +16,9 @@ public interface MicroBatchReadSupport extends DataSourceV2 { * Creates a {@link MicroBatchReader} to scan a batch of data from this data source. * * @param schema the user provided schema, or empty() if none was provided - * @param checkpointLocation a path to HDFS scratch space that can be used for failure recovery. - * Readers for the same logical source in the same query will be - * given the same checkpointLocation. + * @param checkpointLocation a path to Hadoop FS scratch space that can be used for failure + * recovery. Readers for the same logical source in the same query + * will be given the same checkpointLocation. * @param options the options for the returned data source reader, which is an immutable * case-insensitive string-to-string map. */ diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchWriteSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchWriteSupport.java index 5286876e049bd..bdf009e855b32 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchWriteSupport.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchWriteSupport.java @@ -22,7 +22,10 @@ public interface MicroBatchWriteSupport extends BaseStreamingSink { * @param queryId A unique string for the writing query. It's possible that there are many writing * queries running at the same time, and the returned {@link DataSourceV2Writer} * can use this id to distinguish itself from others. - * @param epochId The numeric ID of the batch within this writing query. + * @param epochId The uniquenumeric ID of the batch within this writing query. This is an + * incrementing counter representing a consistent set of data; the same batch may + * be started multiple times in failure recovery scenarios, but it will always + * contain the same records. * @param schema the schema of the data to be written. * @param mode the output mode which determines what successive batch output means to this * source, please refer to {@link OutputMode} for more details. From 81efbeeca9c41a52bf2112c11f4ec4f042dc2b52 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Wed, 6 Dec 2017 10:06:44 -0800 Subject: [PATCH 14/33] explain getOffset --- .../sql/sources/v2/MicroBatchReadSupport.java | 5 ++++- .../sources/v2/reader/ContinuousDataReader.java | 16 +++------------- 2 files changed, 7 insertions(+), 14 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupport.java index 410b36eb4487a..9ed8f6fba72dc 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupport.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupport.java @@ -22,5 +22,8 @@ public interface MicroBatchReadSupport extends DataSourceV2 { * @param options the options for the returned data source reader, which is an immutable * case-insensitive string-to-string map. */ - MicroBatchReader createMicroBatchReader(Optional schema, String checkpointLocation, DataSourceV2Options options); + MicroBatchReader createMicroBatchReader( + Optional schema, + String checkpointLocation, + DataSourceV2Options options); } diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousDataReader.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousDataReader.java index 77f5d8657e258..f70add1e853d8 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousDataReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousDataReader.java @@ -8,22 +8,12 @@ * A variation on {@link DataReader} for use with streaming in continuous processing mode. */ public interface ContinuousDataReader extends DataReader { - /** - * Proceed to next record, returning false only if the read is interrupted. - * - * @throws IOException if failure happens during disk/network IO like reading files. - */ - boolean next() throws IOException; - - /** - * Return the current record. This method should return same value until `next` is called. - */ - T get(); - /** * Get the offset of the current record. * - * The execution engine will use this offset as a restart checkpoint. + * The execution engine will call this method along with get() to keep track of the current + * offset. When an epoch ends, the offset of the previous record in each partition will be saved + * as a restart checkpoint. */ PartitionOffset getOffset(); } From a9c43d99a6c27a191fcb030916c7f17a00415cc3 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Wed, 6 Dec 2017 13:17:43 -0800 Subject: [PATCH 15/33] fix fmt --- .../spark/sql/sources/v2/ContinuousWriteSupport.java | 8 ++++---- .../spark/sql/sources/v2/reader/MicroBatchReader.java | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousWriteSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousWriteSupport.java index 7f87681769515..21fedb286aa99 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousWriteSupport.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousWriteSupport.java @@ -30,8 +30,8 @@ public interface ContinuousWriteSupport extends BaseStreamingSink { * case-insensitive string-to-string map. */ Optional createContinuousWriter( - String queryId, - StructType schema, - OutputMode mode, - DataSourceV2Options options); + String queryId, + StructType schema, + OutputMode mode, + DataSourceV2Options options); } diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/MicroBatchReader.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/MicroBatchReader.java index 4ca1b0ad746dc..2c70d1ed84238 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/MicroBatchReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/MicroBatchReader.java @@ -24,7 +24,7 @@ public interface MicroBatchReader extends DataSourceV2Reader, BaseStreamingSourc void setOffsetRange(Optional start, Optional end); /** - * Returns the specified or inferred start offset for this reader. + * Returns the specified start offset for this reader. * * @throws IllegalStateException if setOffsetRange has not been called */ From 3f03f50766072dedaa7db12f58cd08e858add5e7 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Wed, 6 Dec 2017 13:24:35 -0800 Subject: [PATCH 16/33] fix doc --- .../apache/spark/sql/sources/v2/reader/MicroBatchReader.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/MicroBatchReader.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/MicroBatchReader.java index 2c70d1ed84238..d65630892093c 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/MicroBatchReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/MicroBatchReader.java @@ -18,13 +18,13 @@ public interface MicroBatchReader extends DataSourceV2Reader, BaseStreamingSourc * @param start The initial offset to scan from. If not specified, scan from an * implementation-specified start point, such as the earliest available record. * @param end The last offset to include in the scan. If not specified, scan up to an - * implementation-defined inferred endpoint, such as the last available offset + * implementation-defined endpoint, such as the last available offset * or the start offset plus a target batch size. */ void setOffsetRange(Optional start, Optional end); /** - * Returns the specified start offset for this reader. + * Returns the specified or inferred start offset for this reader. * * @throws IllegalStateException if setOffsetRange has not been called */ From 633804387baa128ddd261bceda934a22745126e8 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Wed, 6 Dec 2017 13:39:20 -0800 Subject: [PATCH 17/33] note interfaces are temporary --- .../spark/sql/execution/streaming/BaseStreamingSink.java | 3 +++ .../spark/sql/execution/streaming/BaseStreamingSource.java | 3 +++ 2 files changed, 6 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/BaseStreamingSink.java b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/BaseStreamingSink.java index a33096ab29175..5a7c8414652bc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/BaseStreamingSink.java +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/BaseStreamingSink.java @@ -2,6 +2,9 @@ /** * The shared interface between V1 and V2 streaming sinks. + * + * This is a temporary interface for compatibility during migration. It should not be implemented + * directly, and will be removed in future versions. */ public interface BaseStreamingSink { } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/BaseStreamingSource.java b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/BaseStreamingSource.java index 01c37f01c6919..d5f12595034a8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/BaseStreamingSource.java +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/BaseStreamingSource.java @@ -4,6 +4,9 @@ /** * The shared interface between V1 streaming sources and V2 streaming readers. + * + * This is a temporary interface for compatibility during migration. It should not be implemented + * directly, and will be removed in future versions. */ public interface BaseStreamingSource { /** From 8949571d08f22dc2e291d8fe4295788441b7cd1c Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Wed, 6 Dec 2017 13:43:38 -0800 Subject: [PATCH 18/33] fix wording --- .../org/apache/spark/sql/sources/v2/MicroBatchReadSupport.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupport.java index 9ed8f6fba72dc..25531076b4365 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupport.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupport.java @@ -13,7 +13,8 @@ @InterfaceStability.Evolving public interface MicroBatchReadSupport extends DataSourceV2 { /** - * Creates a {@link MicroBatchReader} to scan a batch of data from this data source. + * Creates a {@link MicroBatchReader} to read batches of data from this data source in a + * streaming query. * * @param schema the user provided schema, or empty() if none was provided * @param checkpointLocation a path to Hadoop FS scratch space that can be used for failure From df9f22400c59b7413638bba92bb8fb587bfc3109 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Wed, 6 Dec 2017 14:40:48 -0800 Subject: [PATCH 19/33] lifecycle --- .../apache/spark/sql/sources/v2/MicroBatchReadSupport.java | 5 +++++ .../spark/sql/sources/v2/reader/MicroBatchReader.java | 6 ++++-- 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupport.java index 25531076b4365..dda98677267b1 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupport.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupport.java @@ -16,6 +16,11 @@ public interface MicroBatchReadSupport extends DataSourceV2 { * Creates a {@link MicroBatchReader} to read batches of data from this data source in a * streaming query. * + * The execution engine will create a micro-batch reader at the start of a streaming query, + * alternate calls to setOffsetRange and createReadTasks for each batch to process, and then + * call stop() when the execution is complete. Note that a single query may have multiple + * executions due to restart or failure recovery. + * * @param schema the user provided schema, or empty() if none was provided * @param checkpointLocation a path to Hadoop FS scratch space that can be used for failure * recovery. Readers for the same logical source in the same query diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/MicroBatchReader.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/MicroBatchReader.java index d65630892093c..d06542a8b64fa 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/MicroBatchReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/MicroBatchReader.java @@ -24,14 +24,16 @@ public interface MicroBatchReader extends DataSourceV2Reader, BaseStreamingSourc void setOffsetRange(Optional start, Optional end); /** - * Returns the specified or inferred start offset for this reader. + * Returns the specified (if explicitly set through setOffsetRange) or inferred start offset + * for this reader. * * @throws IllegalStateException if setOffsetRange has not been called */ Offset getStartOffset(); /** - * Return the specified or inferred end offset for this reader. + * Return the specified (if explicitly set through setOffsetRange) or inferred end offset + * for this reader. * * @throws IllegalStateException if setOffsetRange has not been called */ From ddaee34e83f287297a74157cb9b0c5b3dea1e0ec Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Wed, 6 Dec 2017 14:53:50 -0800 Subject: [PATCH 20/33] fix offset semantic implementation --- .../continuous/ContinuousRateStreamSource.scala | 11 +++++------ .../sql/execution/streaming/RateSourceV2Suite.scala | 4 ++-- 2 files changed, 7 insertions(+), 8 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala index aaa4b5d664324..d897d49e59e98 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala @@ -35,7 +35,7 @@ object ContinuousRateStreamSource { val ROWS_PER_SECOND = "rowsPerSecond" } -case class RateStreamOffset(partitionToStartValue: Map[Int, Long]) extends Offset { +case class ContinuousRateStreamOffset(partitionToStartValue: Map[Int, Long]) extends Offset { implicit val defaultFormats: DefaultFormats = DefaultFormats override val json = Serialization.write(partitionToStartValue) } @@ -54,11 +54,11 @@ class ContinuousRateStreamReader(options: DataSourceV2Options) val tuples = offsets.map { case ContinuousRateStreamPartitionOffset(p, s) => p -> s } - RateStreamOffset(Map(tuples: _*)) + ContinuousRateStreamOffset(Map(tuples: _*)) } override def deserialize(json: String): Offset = { - RateStreamOffset(Serialization.read[Map[Int, Long]](json)) + ContinuousRateStreamOffset(Serialization.read[Map[Int, Long]](json)) } override def readSchema(): StructType = { @@ -77,7 +77,7 @@ class ContinuousRateStreamReader(options: DataSourceV2Options) override def createReadTasks(): java.util.List[ReadTask[Row]] = { val partitionStartMap = Option(offset.orElse(null)).map { - case o: RateStreamOffset => o.partitionToStartValue + case o: ContinuousRateStreamOffset => o.partitionToStartValue case s: SerializedOffset => Serialization.read[Map[Int, Long]](s.json) case _ => throw new IllegalArgumentException("invalid offset type for ContinuousRateSource") } @@ -150,7 +150,6 @@ class RateStreamDataReader( override def close(): Unit = {} - // We use the value corresponding to partition 0 as the offset. override def getOffset(): PartitionOffset = - ContinuousRateStreamPartitionOffset(partitionIndex, currentValue) + ContinuousRateStreamPartitionOffset(partitionIndex, currentValue - increment) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/RateSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/RateSourceV2Suite.scala index 974eba1e36bd5..4b7c811d55cad 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/RateSourceV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/RateSourceV2Suite.scala @@ -115,14 +115,14 @@ class RateSourceV2Suite extends StreamTest { r.next() data.append(r.get()) assert(r.getOffset() == - ContinuousRateStreamPartitionOffset(t.partitionIndex, r.get.getLong(1) + 2)) + ContinuousRateStreamPartitionOffset(t.partitionIndex, r.get.getLong(1))) } assert(System.currentTimeMillis() < startTime + 100) for (i <- 1 to 5) { r.next() data.append(r.get()) assert(r.getOffset() == - ContinuousRateStreamPartitionOffset(t.partitionIndex, r.get.getLong(1) + 2)) + ContinuousRateStreamPartitionOffset(t.partitionIndex, r.get.getLong(1))) } assert(System.currentTimeMillis() > startTime + 1000) From 1608444ba3c511d861287bb6b07b7348e62d48ee Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Thu, 7 Dec 2017 14:09:49 -0800 Subject: [PATCH 21/33] remove unneeded restriction --- .../apache/spark/sql/sources/v2/reader/DataSourceV2Reader.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/DataSourceV2Reader.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/DataSourceV2Reader.java index 9b7a5603d1735..95ee4a8278322 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/DataSourceV2Reader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/DataSourceV2Reader.java @@ -46,9 +46,6 @@ * Spark first applies all operator push-down optimizations that this data source supports. Then * Spark collects information this data source reported for further optimizations. Finally Spark * issues the scan request and does the actual data reading. - * - * This reader must be able to get constructed and serve readSchema() without assuming an active - * Spark session. An active session can be assumed when creating read tasks. */ @InterfaceStability.Evolving public interface DataSourceV2Reader { From 49525b472bf8162e14a1467da8dcd46201e6df49 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Thu, 7 Dec 2017 14:39:16 -0800 Subject: [PATCH 22/33] deserializeOffset --- .../spark/sql/sources/v2/reader/ContinuousReader.java | 2 +- .../spark/sql/sources/v2/reader/MicroBatchReader.java | 2 +- .../sql/execution/streaming/RateStreamSourceV2.scala | 10 ++++++++-- .../continuous/ContinuousRateStreamSource.scala | 2 +- 4 files changed, 11 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousReader.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousReader.java index 0856a5a1ed02e..81b1f170f2d4b 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousReader.java @@ -22,7 +22,7 @@ public interface ContinuousReader extends BaseStreamingSource, DataSourceV2Reade * Deserialize a JSON string into an Offset of the implementation-defined offset type. * @throws IllegalArgumentException if the JSON does not encode a valid offset for this reader */ - Offset deserialize(String json); + Offset deserializeOffset(String json); /** * Set the desired start offset for read tasks created from this reader. The scan will start diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/MicroBatchReader.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/MicroBatchReader.java index d06542a8b64fa..41aaad2f06ec9 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/MicroBatchReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/MicroBatchReader.java @@ -43,5 +43,5 @@ public interface MicroBatchReader extends DataSourceV2Reader, BaseStreamingSourc * Deserialize a JSON string into an Offset of the implementation-defined offset type. * @throws IllegalArgumentException if the JSON does not encode a valid offset for this reader */ - Offset deserialize(String json); + Offset deserializeOffset(String json); } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateStreamSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateStreamSourceV2.scala index 6a34a318defad..a7474897e6049 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateStreamSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateStreamSourceV2.scala @@ -22,11 +22,10 @@ import java.util.Optional import scala.collection.JavaConverters._ import org.json4s.DefaultFormats -import org.json4s.jackson.Serialization import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.util.DateTimeUtils -import org.apache.spark.sql.execution.streaming.continuous.{ContinuousRateStreamSource, RateStreamDataReader, RateStreamReadTask} +import org.apache.spark.sql.execution.streaming.continuous.{ContinuousRateStreamSource} import org.apache.spark.sql.sources.v2.DataSourceV2Options import org.apache.spark.sql.sources.v2.reader._ import org.apache.spark.sql.types.{LongType, StructField, StructType, TimestampType} @@ -68,6 +67,13 @@ class RateStreamV2Reader(options: DataSourceV2Options) end } + override def deserializeOffset(json: String): Offset = { + LongOffset.convert(SerializedOffset(json)).getOrElse { + throw new IllegalArgumentException(s"invalid encoded offset $json") + } + } + + override def createReadTasks(): java.util.List[ReadTask[Row]] = { val startTime = LongOffset.convert(start).get.offset val numSeconds = (LongOffset.convert(end).get.offset - startTime) / 1000 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala index d897d49e59e98..9dd87c15e773c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala @@ -57,7 +57,7 @@ class ContinuousRateStreamReader(options: DataSourceV2Options) ContinuousRateStreamOffset(Map(tuples: _*)) } - override def deserialize(json: String): Offset = { + override def deserializeOffset(json: String): Offset = { ContinuousRateStreamOffset(Serialization.read[Map[Int, Long]](json)) } From a9fbf3327ba8119f9c704829f31aa81f533bf8a4 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Thu, 7 Dec 2017 14:46:43 -0800 Subject: [PATCH 23/33] add copyright headers --- .../sql/sources/v2/ContinuousReadSupport.java | 17 +++++++++++++++++ .../sql/sources/v2/ContinuousWriteSupport.java | 17 +++++++++++++++++ .../sql/sources/v2/MicroBatchReadSupport.java | 17 +++++++++++++++++ .../sql/sources/v2/MicroBatchWriteSupport.java | 17 +++++++++++++++++ .../sources/v2/reader/ContinuousDataReader.java | 17 +++++++++++++++++ .../sql/sources/v2/reader/ContinuousReader.java | 17 +++++++++++++++++ .../sql/sources/v2/reader/MicroBatchReader.java | 17 +++++++++++++++++ .../sql/sources/v2/writer/ContinuousWriter.java | 17 +++++++++++++++++ .../execution/streaming/BaseStreamingSink.java | 17 +++++++++++++++++ .../streaming/BaseStreamingSource.java | 17 +++++++++++++++++ 10 files changed, 170 insertions(+) diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousReadSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousReadSupport.java index e0538178cabb3..ae4f85820649f 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousReadSupport.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousReadSupport.java @@ -1,3 +1,20 @@ +/* + * 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.sources.v2; import java.util.Optional; diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousWriteSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousWriteSupport.java index 21fedb286aa99..10979ac87f82d 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousWriteSupport.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousWriteSupport.java @@ -1,3 +1,20 @@ +/* + * 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.sources.v2; import java.util.Optional; diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupport.java index dda98677267b1..442cad029d211 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupport.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupport.java @@ -1,3 +1,20 @@ +/* + * 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.sources.v2; import java.util.Optional; diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchWriteSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchWriteSupport.java index bdf009e855b32..520edc3d4c606 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchWriteSupport.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchWriteSupport.java @@ -1,3 +1,20 @@ +/* + * 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.sources.v2; import java.util.Optional; diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousDataReader.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousDataReader.java index f70add1e853d8..ad3fbc7c47134 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousDataReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousDataReader.java @@ -1,3 +1,20 @@ +/* + * 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.sources.v2.reader; import org.apache.spark.sql.sources.v2.reader.PartitionOffset; diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousReader.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousReader.java index 81b1f170f2d4b..1baf82c2df762 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousReader.java @@ -1,3 +1,20 @@ +/* + * 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.sources.v2.reader; import org.apache.spark.sql.sources.v2.reader.PartitionOffset; diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/MicroBatchReader.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/MicroBatchReader.java index 41aaad2f06ec9..438e3f55b7bcf 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/MicroBatchReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/MicroBatchReader.java @@ -1,3 +1,20 @@ +/* + * 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.sources.v2.reader; import org.apache.spark.sql.sources.v2.reader.Offset; diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/ContinuousWriter.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/ContinuousWriter.java index 40d35641d4c34..618f47ed79ca5 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/ContinuousWriter.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/ContinuousWriter.java @@ -1,3 +1,20 @@ +/* + * 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.sources.v2.writer; import org.apache.spark.annotation.InterfaceStability; diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/BaseStreamingSink.java b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/BaseStreamingSink.java index 5a7c8414652bc..ac96c2765368f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/BaseStreamingSink.java +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/BaseStreamingSink.java @@ -1,3 +1,20 @@ +/* + * 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; /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/BaseStreamingSource.java b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/BaseStreamingSource.java index d5f12595034a8..3a02cbfe7afe3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/BaseStreamingSource.java +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/BaseStreamingSource.java @@ -1,3 +1,20 @@ +/* + * 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.sql.sources.v2.reader.Offset; From 22d07cd061bcdbe60a31a2c404b8bb0ac2b5f51f Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Thu, 7 Dec 2017 15:07:18 -0800 Subject: [PATCH 24/33] rebase against datasource package change --- .../spark/sql/execution/streaming/RateSourceV2Suite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/RateSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/RateSourceV2Suite.scala index 4b7c811d55cad..a701eb8691219 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/RateSourceV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/RateSourceV2Suite.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.streaming.StreamTest class RateSourceV2Suite extends StreamTest { test("microbatch in registry") { - DataSource.lookupDataSource("rate").newInstance() match { + DataSource.lookupDataSource("rate", spark.sqlContext.conf).newInstance() match { case ds: MicroBatchReadSupport => val reader = ds.createMicroBatchReader(Optional.empty(), "", DataSourceV2Options.empty()) assert(reader.isInstanceOf[RateStreamV2Reader]) @@ -88,7 +88,7 @@ class RateSourceV2Suite extends StreamTest { } test("continuous in registry") { - DataSource.lookupDataSource("rate").newInstance() match { + DataSource.lookupDataSource("rate", spark.sqlContext.conf).newInstance() match { case ds: ContinuousReadSupport => val reader = ds.createContinuousReader(Optional.empty(), "", DataSourceV2Options.empty()) assert(reader.isInstanceOf[ContinuousRateStreamReader]) From 0b68873fea35923deacdf9395e097d9f8828c77b Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Fri, 8 Dec 2017 12:19:32 -0800 Subject: [PATCH 25/33] refer properly to sink --- .../apache/spark/sql/sources/v2/ContinuousWriteSupport.java | 4 ++-- .../apache/spark/sql/sources/v2/MicroBatchWriteSupport.java | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousWriteSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousWriteSupport.java index 10979ac87f82d..362d5f52b4d00 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousWriteSupport.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousWriteSupport.java @@ -41,8 +41,8 @@ public interface ContinuousWriteSupport extends BaseStreamingSink { * queries running at the same time, and the returned {@link DataSourceV2Writer} * can use this id to distinguish itself from others. * @param schema the schema of the data to be written. - * @param mode the output mode which determines what successive batch output means to this - * source, please refer to {@link OutputMode} for more details. + * @param mode the output mode which determines what successive epoch output means to this + * sink, please refer to {@link OutputMode} for more details. * @param options the options for the returned data source writer, which is an immutable * case-insensitive string-to-string map. */ diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchWriteSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchWriteSupport.java index 520edc3d4c606..63640779b955c 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchWriteSupport.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchWriteSupport.java @@ -45,7 +45,7 @@ public interface MicroBatchWriteSupport extends BaseStreamingSink { * contain the same records. * @param schema the schema of the data to be written. * @param mode the output mode which determines what successive batch output means to this - * source, please refer to {@link OutputMode} for more details. + * sink, please refer to {@link OutputMode} for more details. * @param options the options for the returned data source writer, which is an immutable * case-insensitive string-to-string map. */ From f924a8a2a1e9dde8d10cfc399b7978ecd3184461 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Fri, 8 Dec 2017 15:03:36 -0800 Subject: [PATCH 26/33] widen tolerance interval in continuous rate source test --- .../spark/sql/execution/streaming/RateSourceV2Suite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/RateSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/RateSourceV2Suite.scala index a701eb8691219..baab346430295 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/RateSourceV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/RateSourceV2Suite.scala @@ -124,7 +124,7 @@ class RateSourceV2Suite extends StreamTest { assert(r.getOffset() == ContinuousRateStreamPartitionOffset(t.partitionIndex, r.get.getLong(1))) } - assert(System.currentTimeMillis() > startTime + 1000) + assert(System.currentTimeMillis() > startTime + 900) case _ => throw new IllegalStateException("Unexpected task type") } From 4d166ded90b071332c42704070e98e581fa92042 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Fri, 8 Dec 2017 15:04:24 -0800 Subject: [PATCH 27/33] widen tolerance interval --- .../spark/sql/execution/streaming/RateSourceV2Suite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/RateSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/RateSourceV2Suite.scala index baab346430295..d55cfbb53451b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/RateSourceV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/RateSourceV2Suite.scala @@ -124,7 +124,7 @@ class RateSourceV2Suite extends StreamTest { assert(r.getOffset() == ContinuousRateStreamPartitionOffset(t.partitionIndex, r.get.getLong(1))) } - assert(System.currentTimeMillis() > startTime + 900) + assert(System.currentTimeMillis() >= startTime + 1000) case _ => throw new IllegalStateException("Unexpected task type") } From 7c46b333f314c032caa5a6b838c91f2027e2ab93 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Mon, 11 Dec 2017 12:49:21 -0800 Subject: [PATCH 28/33] clarify start offset --- .../spark/sql/sources/v2/reader/ContinuousDataReader.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousDataReader.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousDataReader.java index ad3fbc7c47134..11b99a93f1494 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousDataReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousDataReader.java @@ -26,7 +26,7 @@ */ public interface ContinuousDataReader extends DataReader { /** - * Get the offset of the current record. + * Get the offset of the current record, or the start offset if no records have been read. * * The execution engine will call this method along with get() to keep track of the current * offset. When an epoch ends, the offset of the previous record in each partition will be saved From 8a2a4f15d7f8635ddb4fbd1ca0585c71d2b2a5e0 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Tue, 12 Dec 2017 16:58:40 -0800 Subject: [PATCH 29/33] add docs --- .../spark/sql/sources/v2/reader/Offset.java | 6 ++++++ .../sources/v2/reader/PartitionOffset.java | 4 +++- .../ContinuousRateStreamSource.scala | 20 +++++++++---------- 3 files changed, 19 insertions(+), 11 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/Offset.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/Offset.java index a5d7f1bd99ae9..1ebd35356f1a3 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/Offset.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/Offset.java @@ -17,6 +17,12 @@ package org.apache.spark.sql.sources.v2.reader; +/** + * An abstract representation of progress through a [[MicroBatchReader]] or [[ContinuousReader]]. + * During execution, Offsets provided by the data source implementation will be logged and used as + * restart checkpoints. Sources should provide an Offset implementation which they can use to + * reconstruct the stream position where the offset was taken. + */ public abstract class Offset { /** * A JSON-serialized representation of an Offset that is diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionOffset.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionOffset.java index c1b0081d80749..07826b6688476 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionOffset.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionOffset.java @@ -17,12 +17,14 @@ package org.apache.spark.sql.sources.v2.reader; +import java.io.Serializable; + /** * Used for per-partition offsets in continuous processing. ContinuousReader implementations will * provide a method to merge these into a global Offset. * * These offsets must be serializable. */ -public interface PartitionOffset { +public interface PartitionOffset extends Serializable { } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala index 9dd87c15e773c..1abfcce2f02fe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala @@ -61,11 +61,7 @@ class ContinuousRateStreamReader(options: DataSourceV2Options) ContinuousRateStreamOffset(Serialization.read[Map[Int, Long]](json)) } - override def readSchema(): StructType = { - StructType( - StructField("timestamp", TimestampType, false) :: - StructField("value", LongType, false) :: Nil) - } + override def readSchema(): StructType = RateSourceProvider.SCHEMA private var offset: java.util.Optional[Offset] = _ @@ -77,12 +73,16 @@ class ContinuousRateStreamReader(options: DataSourceV2Options) override def createReadTasks(): java.util.List[ReadTask[Row]] = { val partitionStartMap = Option(offset.orElse(null)).map { - case o: ContinuousRateStreamOffset => o.partitionToStartValue - case s: SerializedOffset => Serialization.read[Map[Int, Long]](s.json) - case _ => throw new IllegalArgumentException("invalid offset type for ContinuousRateSource") + case off: ContinuousRateStreamOffset => off.partitionToStartValue + case off => + throw new IllegalArgumentException( + s"invalid offset type ${off.getClass()} for ContinuousRateSource") } - if (partitionStartMap.exists(_.keySet.size > numPartitions)) { - throw new IllegalArgumentException("Start offset contained too many partitions.") + if (partitionStartMap.exists(_.keySet.size != numPartitions)) { + throw new IllegalArgumentException( + s"The previous run contained ${partitionStartMap.get.keySet.size} partitions, but" + + s" $numPartitions partitions are currently configured. The numPartitions option" + + " cannot be changed.") } val perPartitionRate = rowsPerSecond.toDouble / numPartitions.toDouble From 8809bf9e575195cc4352b5ac6e27a124a91583f6 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Tue, 12 Dec 2017 17:12:02 -0800 Subject: [PATCH 30/33] fix problems --- .../continuous/ContinuousRateStreamSource.scala | 7 ++++--- .../spark/sql/execution/streaming/memoryV2.scala | 11 ++--------- 2 files changed, 6 insertions(+), 12 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala index 1abfcce2f02fe..06200ed6b4dc9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala @@ -88,7 +88,8 @@ class ContinuousRateStreamReader(options: DataSourceV2Options) Range(0, numPartitions).map { n => // If the offset doesn't have a value for this partition, start from the beginning. - val start = partitionStartMap.flatMap(_.get(n)).getOrElse(0L + n) + // Start offset is exclusive, so we actually pass negative values when we want to start at 0. + val start = partitionStartMap.flatMap(_.get(n)).getOrElse(0L + n - numPartitions) // Have each partition advance by numPartitions each row, with starting points staggered // by their partition index. RateStreamReadTask(start, n, numPartitions, perPartitionRate) @@ -137,10 +138,10 @@ class RateStreamDataReader( nextReadTime += 1000 } + currentValue += increment currentRow = Row( DateTimeUtils.toJavaTimestamp(DateTimeUtils.fromMillis(System.currentTimeMillis)), currentValue) - currentValue += increment numReadRows += 1 true @@ -151,5 +152,5 @@ class RateStreamDataReader( override def close(): Unit = {} override def getOffset(): PartitionOffset = - ContinuousRateStreamPartitionOffset(partitionIndex, currentValue - increment) + ContinuousRateStreamPartitionOffset(partitionIndex, currentValue) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memoryV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memoryV2.scala index f780e4e08a216..437040cc12472 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memoryV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memoryV2.scala @@ -125,14 +125,8 @@ class MemoryWriter(sink: MemorySinkV2, batchId: Long, outputMode: OutputMode) override def createWriterFactory: MemoryWriterFactory = MemoryWriterFactory(outputMode) def commit(messages: Array[WriterCommitMessage]): Unit = { - val newRows = messages.flatMap { message => - // TODO remove - if (message != null) { - assert(message.isInstanceOf[MemoryWriterCommitMessage]) - message.asInstanceOf[MemoryWriterCommitMessage].data - } else { - Seq() - } + val newRows = messages.flatMap { + case message: MemoryWriterCommitMessage => message.data } sink.write(batchId, outputMode, newRows) } @@ -150,7 +144,6 @@ class ContinuousMemoryWriter(val sink: MemorySinkV2, outputMode: OutputMode) override def commit(epochId: Long, messages: Array[WriterCommitMessage]): Unit = { val newRows = messages.flatMap { case message: MemoryWriterCommitMessage => message.data - case _ => Seq() } sink.write(epochId, outputMode, newRows) } From 0974ac3031284b9cfbaaad89bd4ed088de753243 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Wed, 13 Dec 2017 10:17:06 -0800 Subject: [PATCH 31/33] even timestamps --- .../ContinuousRateStreamSource.scala | 34 ++++++++----------- .../streaming/RateSourceV2Suite.scala | 20 ++++------- 2 files changed, 21 insertions(+), 33 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala index 06200ed6b4dc9..155ae0f644725 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala @@ -106,43 +106,37 @@ case class RateStreamReadTask( startValue: Long, partitionIndex: Int, increment: Long, rowsPerSecond: Double) extends ReadTask[Row] { override def createDataReader(): DataReader[Row] = - new RateStreamDataReader(startValue, partitionIndex, increment, rowsPerSecond.toLong) + new RateStreamDataReader(startValue, partitionIndex, increment, rowsPerSecond) } class RateStreamDataReader( - startValue: Long, partitionIndex: Int, increment: Long, rowsPerSecond: Long) + startValue: Long, partitionIndex: Int, increment: Long, rowsPerSecond: Double) extends ContinuousDataReader[Row] { - private var nextReadTime = 0L - private var numReadRows = 0L + private var nextReadTime: Long = _ + private val readTimeIncrement: Long = (1000 / rowsPerSecond).toLong private var currentValue = startValue private var currentRow: Row = null override def next(): Boolean = { // Set the timestamp for the first time. - if (currentRow == null) nextReadTime = System.currentTimeMillis() + 1000 - - if (numReadRows == rowsPerSecond) { - // Sleep until we reach the next second. - - try { - while (System.currentTimeMillis < nextReadTime) { - Thread.sleep(nextReadTime - System.currentTimeMillis) - } - } catch { - case _: InterruptedException => - // Someone's trying to end the task; just let them. - return false + if (currentRow == null) nextReadTime = System.currentTimeMillis() + + try { + while (System.currentTimeMillis < nextReadTime) { + Thread.sleep(nextReadTime - System.currentTimeMillis) } - numReadRows = 0 - nextReadTime += 1000 + } catch { + case _: InterruptedException => + // Someone's trying to end the task; just let them. + return false } + nextReadTime += readTimeIncrement currentValue += increment currentRow = Row( DateTimeUtils.toJavaTimestamp(DateTimeUtils.fromMillis(System.currentTimeMillis)), currentValue) - numReadRows += 1 true } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/RateSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/RateSourceV2Suite.scala index d55cfbb53451b..bc99ccd328562 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/RateSourceV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/RateSourceV2Suite.scala @@ -99,7 +99,7 @@ class RateSourceV2Suite extends StreamTest { test("continuous data") { val reader = new ContinuousRateStreamReader( - new DataSourceV2Options(Map("numPartitions" -> "2", "rowsPerSecond" -> "10").asJava)) + new DataSourceV2Options(Map("numPartitions" -> "2", "rowsPerSecond" -> "20").asJava)) reader.setOffset(Optional.empty()) val tasks = reader.createReadTasks() assert(tasks.size == 2) @@ -107,28 +107,22 @@ class RateSourceV2Suite extends StreamTest { val data = scala.collection.mutable.ListBuffer[Row]() tasks.asScala.foreach { case t: RateStreamReadTask => + // Read the first 11 rows. Each partition should be outputting 10 rows per second, so + // the 11th row should come 1 second (within a confidence interval) after the first. val startTime = System.currentTimeMillis() val r = t.createDataReader().asInstanceOf[RateStreamDataReader] - // The first set of (rowsPerSecond / numPartitions) should come ~immediately, but the - // next should only come after 1 second. - for (i <- 1 to 5) { + for (_ <- 1 to 11) { r.next() data.append(r.get()) assert(r.getOffset() == ContinuousRateStreamPartitionOffset(t.partitionIndex, r.get.getLong(1))) } - assert(System.currentTimeMillis() < startTime + 100) - for (i <- 1 to 5) { - r.next() - data.append(r.get()) - assert(r.getOffset() == - ContinuousRateStreamPartitionOffset(t.partitionIndex, r.get.getLong(1))) - } - assert(System.currentTimeMillis() >= startTime + 1000) + assert(System.currentTimeMillis() < startTime + 1100) + assert(System.currentTimeMillis() > startTime + 900) case _ => throw new IllegalStateException("Unexpected task type") } - assert(data.map(_.getLong(1)).toSeq.sorted == Range(0, 20)) + assert(data.map(_.getLong(1)).toSeq.sorted == Range(0, 22)) } } From c1a632266e0f7fbb2a792d3a8f4e15667d380710 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Wed, 13 Dec 2017 15:28:13 -0800 Subject: [PATCH 32/33] unify offsets --- .../streaming/RateStreamOffset.scala | 29 ++++++ .../streaming/RateStreamSourceV2.scala | 98 ++++++++++++++----- ...mSource.scala => RateStreamSourceV2.scala} | 80 +++++++-------- .../streaming/RateSourceV2Suite.scala | 77 ++++++++++----- 4 files changed, 195 insertions(+), 89 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateStreamOffset.scala rename sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/{ContinuousRateStreamSource.scala => RateStreamSourceV2.scala} (61%) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateStreamOffset.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateStreamOffset.scala new file mode 100644 index 0000000000000..13679dfbe446b --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateStreamOffset.scala @@ -0,0 +1,29 @@ +/* + * 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.json4s.DefaultFormats +import org.json4s.jackson.Serialization + +import org.apache.spark.sql.sources.v2.reader.Offset + +case class RateStreamOffset(partitionToValueAndRunTimeMs: Map[Int, (Long, Long)]) + extends Offset { + implicit val defaultFormats: DefaultFormats = DefaultFormats + override val json = Serialization.write(partitionToValueAndRunTimeMs) +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateStreamSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateStreamSourceV2.scala index a7474897e6049..102551c238bfb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateStreamSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateStreamSourceV2.scala @@ -20,12 +20,13 @@ package org.apache.spark.sql.execution.streaming import java.util.Optional import scala.collection.JavaConverters._ +import scala.collection.mutable import org.json4s.DefaultFormats +import org.json4s.jackson.Serialization import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.util.DateTimeUtils -import org.apache.spark.sql.execution.streaming.continuous.{ContinuousRateStreamSource} import org.apache.spark.sql.sources.v2.DataSourceV2Options import org.apache.spark.sql.sources.v2.reader._ import org.apache.spark.sql.types.{LongType, StructField, StructType, TimestampType} @@ -38,9 +39,14 @@ class RateStreamV2Reader(options: DataSourceV2Options) val clock = new SystemClock private val numPartitions = - options.get(ContinuousRateStreamSource.NUM_PARTITIONS).orElse("5").toInt + options.get(RateStreamSourceV2.NUM_PARTITIONS).orElse("5").toInt private val rowsPerSecond = - options.get(ContinuousRateStreamSource.ROWS_PER_SECOND).orElse("6").toLong + options.get(RateStreamSourceV2.ROWS_PER_SECOND).orElse("6").toLong + + // The interval (in milliseconds) between rows in each partition. + // e.g. if there are 4 global rows per second, and 2 partitions, each partition + // should output rows every (1000 * 2 / 4) = 500 ms. + private val msPerPartitionBetweenRows = (1000 * numPartitions) / rowsPerSecond override def readSchema(): StructType = { StructType( @@ -50,12 +56,33 @@ class RateStreamV2Reader(options: DataSourceV2Options) val creationTimeMs = clock.getTimeMillis() - private var start: Offset = _ - private var end: Offset = _ + private var start: RateStreamOffset = _ + private var end: RateStreamOffset = _ override def setOffsetRange(start: Optional[Offset], end: Optional[Offset]): Unit = { - this.start = start.orElse(LongOffset(creationTimeMs)) - this.end = end.orElse(LongOffset(clock.getTimeMillis())) + this.start = start.orElse( + RateStreamSourceV2.createInitialOffset(numPartitions, creationTimeMs)) + .asInstanceOf[RateStreamOffset] + + this.end = end.orElse { + val currentTime = clock.getTimeMillis() + RateStreamOffset( + this.start.partitionToValueAndRunTimeMs.map { + case startOffset @ (part, (currentVal, currentReadTime)) => + // Calculate the number of rows we should advance in this partition (based on the + // current time), and output a corresponding offset. + val readInterval = currentTime - currentReadTime + val numNewRows = readInterval / msPerPartitionBetweenRows + if (numNewRows <= 0) { + startOffset + } else { + (part, + (currentVal + (numNewRows * numPartitions), + currentReadTime + (numNewRows * msPerPartitionBetweenRows))) + } + } + ) + }.asInstanceOf[RateStreamOffset] } override def getStartOffset(): Offset = { @@ -68,34 +95,38 @@ class RateStreamV2Reader(options: DataSourceV2Options) } override def deserializeOffset(json: String): Offset = { - LongOffset.convert(SerializedOffset(json)).getOrElse { - throw new IllegalArgumentException(s"invalid encoded offset $json") - } + RateStreamOffset(Serialization.read[Map[Int, (Long, Long)]](json)) } - override def createReadTasks(): java.util.List[ReadTask[Row]] = { - val startTime = LongOffset.convert(start).get.offset - val numSeconds = (LongOffset.convert(end).get.offset - startTime) / 1000 - val firstValue = (startTime - creationTimeMs) / 1000 - - (firstValue to firstValue + numSeconds * rowsPerSecond - 1) - .groupBy(_ % numPartitions) - .values - .map(vals => RateStreamBatchTask(vals).asInstanceOf[ReadTask[Row]]) - .toList - .asJava + val startMap = start.partitionToValueAndRunTimeMs + val endMap = end.partitionToValueAndRunTimeMs + endMap.keys.toSeq.map { part => + val (endVal, _) = endMap(part) + val (startVal, startTimeMs) = startMap(part) + + val packedRows = mutable.ListBuffer[(Long, Long)]() + var outVal = startVal + numPartitions + var outTimeMs = startTimeMs + msPerPartitionBetweenRows + while (outVal <= endVal) { + packedRows.append((outTimeMs, outVal)) + outVal += numPartitions + outTimeMs += msPerPartitionBetweenRows + } + + RateStreamBatchTask(packedRows).asInstanceOf[ReadTask[Row]] + }.toList.asJava } override def commit(end: Offset): Unit = {} override def stop(): Unit = {} } -case class RateStreamBatchTask(vals: Seq[Long]) extends ReadTask[Row] { +case class RateStreamBatchTask(vals: Seq[(Long, Long)]) extends ReadTask[Row] { override def createDataReader(): DataReader[Row] = new RateStreamBatchReader(vals) } -class RateStreamBatchReader(vals: Seq[Long]) extends DataReader[Row] { +class RateStreamBatchReader(vals: Seq[(Long, Long)]) extends DataReader[Row] { var currentIndex = -1 override def next(): Boolean = { @@ -106,9 +137,26 @@ class RateStreamBatchReader(vals: Seq[Long]) extends DataReader[Row] { override def get(): Row = { Row( - DateTimeUtils.toJavaTimestamp(DateTimeUtils.fromMillis(System.currentTimeMillis)), - vals(currentIndex)) + DateTimeUtils.toJavaTimestamp(DateTimeUtils.fromMillis(vals(currentIndex)._1)), + vals(currentIndex)._2) } override def close(): Unit = {} } + +object RateStreamSourceV2 { + val NUM_PARTITIONS = "numPartitions" + val ROWS_PER_SECOND = "rowsPerSecond" + + private[sql] def createInitialOffset(numPartitions: Int, creationTimeMs: Long) = { + RateStreamOffset( + Range(0, numPartitions).map { i => + // Note that the starting offset is exclusive, so we have to decrement the starting value + // by the increment that will later be applied. The first row output in each + // partition will have a value equal to the partition index. + (i, + ((i - numPartitions).toLong, + creationTimeMs)) + }.toMap) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/RateStreamSourceV2.scala similarity index 61% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/RateStreamSourceV2.scala index 155ae0f644725..77fc26730e52c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/RateStreamSourceV2.scala @@ -30,69 +30,65 @@ import org.apache.spark.sql.sources.v2.{ContinuousReadSupport, DataSourceV2, Dat import org.apache.spark.sql.sources.v2.reader._ import org.apache.spark.sql.types.{LongType, StructField, StructType, TimestampType} -object ContinuousRateStreamSource { - val NUM_PARTITIONS = "numPartitions" - val ROWS_PER_SECOND = "rowsPerSecond" -} - -case class ContinuousRateStreamOffset(partitionToStartValue: Map[Int, Long]) extends Offset { - implicit val defaultFormats: DefaultFormats = DefaultFormats - override val json = Serialization.write(partitionToStartValue) -} - -case class ContinuousRateStreamPartitionOffset(partition: Int, start: Long) extends PartitionOffset +case class ContinuousRateStreamPartitionOffset( + partition: Int, currentValue: Long, currentTimeMs: Long) extends PartitionOffset class ContinuousRateStreamReader(options: DataSourceV2Options) extends ContinuousReader { implicit val defaultFormats: DefaultFormats = DefaultFormats - val numPartitions = options.get(ContinuousRateStreamSource.NUM_PARTITIONS).orElse("5").toInt - val rowsPerSecond = options.get(ContinuousRateStreamSource.ROWS_PER_SECOND).orElse("6").toLong + val creationTime = System.currentTimeMillis() + + val numPartitions = options.get(RateStreamSourceV2.NUM_PARTITIONS).orElse("5").toInt + val rowsPerSecond = options.get(RateStreamSourceV2.ROWS_PER_SECOND).orElse("6").toLong + val perPartitionRate = rowsPerSecond.toDouble / numPartitions.toDouble override def mergeOffsets(offsets: Array[PartitionOffset]): Offset = { assert(offsets.length == numPartitions) val tuples = offsets.map { - case ContinuousRateStreamPartitionOffset(p, s) => p -> s + case ContinuousRateStreamPartitionOffset(i, currVal, nextRead) => (i, (currVal, nextRead)) } - ContinuousRateStreamOffset(Map(tuples: _*)) + RateStreamOffset(Map(tuples: _*)) } override def deserializeOffset(json: String): Offset = { - ContinuousRateStreamOffset(Serialization.read[Map[Int, Long]](json)) + RateStreamOffset(Serialization.read[Map[Int, (Long, Long)]](json)) } override def readSchema(): StructType = RateSourceProvider.SCHEMA - private var offset: java.util.Optional[Offset] = _ + private var offset: Offset = _ override def setOffset(offset: java.util.Optional[Offset]): Unit = { - this.offset = offset + this.offset = offset.orElse(RateStreamSourceV2.createInitialOffset(numPartitions, creationTime)) } - override def getStartOffset(): Offset = offset.get() + override def getStartOffset(): Offset = offset override def createReadTasks(): java.util.List[ReadTask[Row]] = { - val partitionStartMap = Option(offset.orElse(null)).map { - case off: ContinuousRateStreamOffset => off.partitionToStartValue + val partitionStartMap = offset match { + case off: RateStreamOffset => off.partitionToValueAndRunTimeMs case off => throw new IllegalArgumentException( s"invalid offset type ${off.getClass()} for ContinuousRateSource") } - if (partitionStartMap.exists(_.keySet.size != numPartitions)) { + if (partitionStartMap.keySet.size != numPartitions) { throw new IllegalArgumentException( - s"The previous run contained ${partitionStartMap.get.keySet.size} partitions, but" + + s"The previous run contained ${partitionStartMap.keySet.size} partitions, but" + s" $numPartitions partitions are currently configured. The numPartitions option" + " cannot be changed.") } - val perPartitionRate = rowsPerSecond.toDouble / numPartitions.toDouble - Range(0, numPartitions).map { n => - // If the offset doesn't have a value for this partition, start from the beginning. - // Start offset is exclusive, so we actually pass negative values when we want to start at 0. - val start = partitionStartMap.flatMap(_.get(n)).getOrElse(0L + n - numPartitions) + Range(0, numPartitions).map { i => + val start = partitionStartMap(i) // Have each partition advance by numPartitions each row, with starting points staggered // by their partition index. - RateStreamReadTask(start, n, numPartitions, perPartitionRate) + RateStreamReadTask( + start._1, // starting row value + start._2, // starting time in ms + i, + numPartitions, + perPartitionRate) .asInstanceOf[ReadTask[Row]] }.asJava } @@ -103,24 +99,32 @@ class ContinuousRateStreamReader(options: DataSourceV2Options) } case class RateStreamReadTask( - startValue: Long, partitionIndex: Int, increment: Long, rowsPerSecond: Double) + startValue: Long, + startTimeMs: Long, + partitionIndex: Int, + increment: Long, + rowsPerSecond: Double) extends ReadTask[Row] { override def createDataReader(): DataReader[Row] = - new RateStreamDataReader(startValue, partitionIndex, increment, rowsPerSecond) + new RateStreamDataReader(startValue, startTimeMs, partitionIndex, increment, rowsPerSecond) } class RateStreamDataReader( - startValue: Long, partitionIndex: Int, increment: Long, rowsPerSecond: Double) + startValue: Long, + startTimeMs: Long, + partitionIndex: Int, + increment: Long, + rowsPerSecond: Double) extends ContinuousDataReader[Row] { - private var nextReadTime: Long = _ + private var nextReadTime: Long = startTimeMs private val readTimeIncrement: Long = (1000 / rowsPerSecond).toLong private var currentValue = startValue private var currentRow: Row = null override def next(): Boolean = { - // Set the timestamp for the first time. - if (currentRow == null) nextReadTime = System.currentTimeMillis() + currentValue += increment + nextReadTime += readTimeIncrement try { while (System.currentTimeMillis < nextReadTime) { @@ -131,11 +135,9 @@ class RateStreamDataReader( // Someone's trying to end the task; just let them. return false } - nextReadTime += readTimeIncrement - currentValue += increment currentRow = Row( - DateTimeUtils.toJavaTimestamp(DateTimeUtils.fromMillis(System.currentTimeMillis)), + DateTimeUtils.toJavaTimestamp(DateTimeUtils.fromMillis(nextReadTime)), currentValue) true @@ -146,5 +148,5 @@ class RateStreamDataReader( override def close(): Unit = {} override def getOffset(): PartitionOffset = - ContinuousRateStreamPartitionOffset(partitionIndex, currentValue) + ContinuousRateStreamPartitionOffset(partitionIndex, currentValue, nextReadTime) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/RateSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/RateSourceV2Suite.scala index bc99ccd328562..ef801ceb1310c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/RateSourceV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/RateSourceV2Suite.scala @@ -23,7 +23,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.sql.Row import org.apache.spark.sql.execution.datasources.DataSource -import org.apache.spark.sql.execution.streaming.continuous.{ContinuousRateStreamPartitionOffset, ContinuousRateStreamReader, RateStreamDataReader, RateStreamReadTask} +import org.apache.spark.sql.execution.streaming.continuous._ import org.apache.spark.sql.sources.v2.{ContinuousReadSupport, DataSourceV2Options, MicroBatchReadSupport} import org.apache.spark.sql.streaming.StreamTest @@ -38,41 +38,65 @@ class RateSourceV2Suite extends StreamTest { } } - test("microbatch - options propagated") { + test("microbatch - numPartitions propagated") { val reader = new RateStreamV2Reader( new DataSourceV2Options(Map("numPartitions" -> "11", "rowsPerSecond" -> "33").asJava)) - reader.setOffsetRange(Optional.empty(), - Optional.of(LongOffset(System.currentTimeMillis() + 1001))) + reader.setOffsetRange(Optional.empty(), Optional.empty()) val tasks = reader.createReadTasks() assert(tasks.size == 11) - tasks.asScala.foreach { - // for 1 second, size of each task is (rowsPerSecond / numPartitions) - case RateStreamBatchTask(vals) => vals.size == 3 - case _ => throw new IllegalStateException("Unexpected task type") - } } test("microbatch - set offset") { val reader = new RateStreamV2Reader(DataSourceV2Options.empty()) - reader.setOffsetRange(Optional.of(LongOffset(12345)), Optional.of(LongOffset(54321))) - assert(reader.getStartOffset() == LongOffset(12345)) - assert(reader.getEndOffset() == LongOffset(54321)) + val startOffset = RateStreamOffset(Map((0, (0, 1000)))) + val endOffset = RateStreamOffset(Map((0, (0, 2000)))) + reader.setOffsetRange(Optional.of(startOffset), Optional.of(endOffset)) + assert(reader.getStartOffset() == startOffset) + assert(reader.getEndOffset() == endOffset) } test("microbatch - infer offsets") { - val reader = new RateStreamV2Reader(DataSourceV2Options.empty()) + val reader = new RateStreamV2Reader( + new DataSourceV2Options(Map("numPartitions" -> "1", "rowsPerSecond" -> "100").asJava)) reader.clock.waitTillTime(reader.clock.getTimeMillis() + 100) reader.setOffsetRange(Optional.empty(), Optional.empty()) - assert(reader.getStartOffset() == LongOffset(reader.creationTimeMs)) - assert(reader.getEndOffset().asInstanceOf[LongOffset].offset >= reader.creationTimeMs + 100) + reader.getStartOffset() match { + case r: RateStreamOffset => + assert(r.partitionToValueAndRunTimeMs(0)._2 == reader.creationTimeMs) + case _ => throw new IllegalStateException("unexpected offset type") + } + reader.getEndOffset() match { + case r: RateStreamOffset => + // End offset may be a bit beyond 100 ms/9 rows after creation if the wait lasted + // longer than 100ms. It should never be early. + assert(r.partitionToValueAndRunTimeMs(0)._1 >= 9) + assert(r.partitionToValueAndRunTimeMs(0)._2 >= reader.creationTimeMs + 100) + + case _ => throw new IllegalStateException("unexpected offset type") + } } + test("microbatch - predetermined batch size") { + val reader = new RateStreamV2Reader( + new DataSourceV2Options(Map("numPartitions" -> "1", "rowsPerSecond" -> "20").asJava)) + val startOffset = RateStreamOffset(Map((0, (0, 1000)))) + val endOffset = RateStreamOffset(Map((0, (20, 2000)))) + reader.setOffsetRange(Optional.of(startOffset), Optional.of(endOffset)) + val tasks = reader.createReadTasks() + assert(tasks.size == 1) + assert(tasks.get(0).asInstanceOf[RateStreamBatchTask].vals.size == 20) + } test("microbatch - data read") { val reader = new RateStreamV2Reader( new DataSourceV2Options(Map("numPartitions" -> "11", "rowsPerSecond" -> "33").asJava)) - reader.setOffsetRange(Optional.empty(), - Optional.of(LongOffset(System.currentTimeMillis() + 1001))) + val startOffset = RateStreamSourceV2.createInitialOffset(11, reader.creationTimeMs) + val endOffset = RateStreamOffset(startOffset.partitionToValueAndRunTimeMs.toSeq.map { + case (part, (currentVal, currentReadTime)) => + (part, (currentVal + 33, currentReadTime + 1000)) + }.toMap) + + reader.setOffsetRange(Optional.of(startOffset), Optional.of(endOffset)) val tasks = reader.createReadTasks() assert(tasks.size == 11) @@ -107,22 +131,25 @@ class RateSourceV2Suite extends StreamTest { val data = scala.collection.mutable.ListBuffer[Row]() tasks.asScala.foreach { case t: RateStreamReadTask => - // Read the first 11 rows. Each partition should be outputting 10 rows per second, so - // the 11th row should come 1 second (within a confidence interval) after the first. - val startTime = System.currentTimeMillis() + val startTimeMs = reader.getStartOffset() + .asInstanceOf[RateStreamOffset] + .partitionToValueAndRunTimeMs(t.partitionIndex) + ._2 val r = t.createDataReader().asInstanceOf[RateStreamDataReader] - for (_ <- 1 to 11) { + for (rowIndex <- 0 to 9) { r.next() data.append(r.get()) assert(r.getOffset() == - ContinuousRateStreamPartitionOffset(t.partitionIndex, r.get.getLong(1))) + ContinuousRateStreamPartitionOffset( + t.partitionIndex, + t.partitionIndex + rowIndex * 2, + startTimeMs + (rowIndex + 1) * 100)) } - assert(System.currentTimeMillis() < startTime + 1100) - assert(System.currentTimeMillis() > startTime + 900) + assert(System.currentTimeMillis() >= startTimeMs + 1000) case _ => throw new IllegalStateException("Unexpected task type") } - assert(data.map(_.getLong(1)).toSeq.sorted == Range(0, 22)) + assert(data.map(_.getLong(1)).toSeq.sorted == Range(0, 20)) } } From 3cb6ceec48b34523ac0c58e39a4825d629b56938 Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Wed, 13 Dec 2017 15:29:01 -0800 Subject: [PATCH 33/33] undo spurious rename --- ...{RateStreamSourceV2.scala => ContinuousRateStreamSource.scala} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/{RateStreamSourceV2.scala => ContinuousRateStreamSource.scala} (100%) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/RateStreamSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala similarity index 100% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/RateStreamSourceV2.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala